You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by wa...@apache.org on 2023/02/15 16:12:13 UTC

[incubator-devlake] branch main updated: feat: add increment & timeFilter for bitbucket (#4373)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 1674a7cf6 feat: add increment & timeFilter for bitbucket (#4373)
1674a7cf6 is described below

commit 1674a7cf64af505bd37333e4e0a740467e6f000a
Author: Likyh <ya...@meri.co>
AuthorDate: Thu Feb 16 00:12:07 2023 +0800

    feat: add increment & timeFilter for bitbucket (#4373)
    
    * feat: add increment & timeFilter for bitbucket
    
    * feat: add bp/tr/scope api for bitbucket
    
    * feat: support bitbucket transformation rule in config-ui
    
    * feat: add remote api for bitbucket
    
    * feat: copy common miller columns
    
    * fix: update remote api
    
    * fix: fix enrich options if repo not exists
    
    * feat: reduct the size of response body and fix all column to proper value
    
    * fix: fix some small bugs
    
    * fix: fix for review
    
    * fix: fix e2e test1
    
    * fix: fix e2e test
    
    * fix: fix for e2e test
    
    * fix: fix for e2e test
    
    * feat: update with miller column
    
    * fix: fix for review
    
    * fix: fix for review
    
    * fix: fix for review
---
 .../models/domainlayer/devops/cicd_pipeline.go     |   8 +-
 backend/plugins/bitbucket/api/blueprint.go         | 181 --------------
 .../plugins/bitbucket/api/blueprint_V200_test.go   | 172 +++++++++++++
 backend/plugins/bitbucket/api/blueprint_test.go    | 136 ----------
 backend/plugins/bitbucket/api/blueprint_v200.go    | 196 +++++++++++++++
 .../plugins/{gitlab => bitbucket}/api/remote.go    | 257 ++++++++-----------
 backend/plugins/bitbucket/api/scope.go             | 213 ++++++++++++++++
 .../plugins/bitbucket/api/transformation_rule.go   | 132 ++++++++++
 backend/plugins/bitbucket/bitbucket.go             |  10 +-
 backend/plugins/bitbucket/e2e/account_test.go      |   7 +-
 backend/plugins/bitbucket/e2e/comment_test.go      |  15 +-
 backend/plugins/bitbucket/e2e/issue_test.go        |  16 +-
 backend/plugins/bitbucket/e2e/pipelines_test.go    |   7 +-
 backend/plugins/bitbucket/e2e/pr_test.go           |  10 +-
 .../_raw_bitbucket_api_issue_comments.csv          |  56 ++---
 .../e2e/raw_tables/_raw_bitbucket_api_issues.csv   |  60 ++---
 .../raw_tables/_raw_bitbucket_api_pipelines.csv    |  20 +-
 .../_raw_bitbucket_api_pull_requests.csv           | 114 ++++-----
 .../_raw_bitbucket_api_pullrequest_comments.csv    |  36 +--
 .../raw_tables/_raw_bitbucket_api_repositories.csv |   2 +-
 backend/plugins/bitbucket/e2e/repo_test.go         |  35 ++-
 .../snapshot_tables/_tool_bitbucket_accounts.csv   |   2 +-
 .../_tool_bitbucket_accounts_in_comments.csv       |   2 +-
 .../_tool_bitbucket_accounts_in_issue.csv          |   2 +-
 .../_tool_bitbucket_accounts_in_pr.csv             |   2 +-
 .../_tool_bitbucket_issue_comments.csv             |  56 ++---
 .../e2e/snapshot_tables/_tool_bitbucket_issues.csv |  68 ++---
 .../snapshot_tables/_tool_bitbucket_pipelines.csv  |  20 +-
 .../_tool_bitbucket_pull_request_comments.csv      |  36 +--
 .../_tool_bitbucket_pull_requests.csv              | 114 ++++-----
 .../e2e/snapshot_tables/_tool_bitbucket_repos.csv  |   4 +-
 .../bitbucket/e2e/snapshot_tables/board_issues.csv |  60 ++---
 .../bitbucket/e2e/snapshot_tables/boards.csv       |   2 +-
 .../e2e/snapshot_tables/issue_comments.csv         |  56 ++---
 .../bitbucket/e2e/snapshot_tables/issues.csv       |  66 ++---
 .../e2e/snapshot_tables/pull_request_comments.csv  |  38 +--
 .../e2e/snapshot_tables/pull_requests.csv          | 220 ++++++++---------
 .../bitbucket/e2e/snapshot_tables/repos.csv        |   2 +-
 backend/plugins/bitbucket/impl/impl.go             | 123 ++++++++-
 backend/plugins/bitbucket/models/connection.go     |   7 -
 backend/plugins/bitbucket/models/issue.go          |   3 +-
 .../20230206_add_scope_and_transformation.go       | 205 +++++++++++++++
 .../archived/transformation_rule.go                |  41 +++
 .../bitbucket/models/migrationscripts/register.go  |   1 +
 backend/plugins/bitbucket/models/pr.go             |   2 +-
 backend/plugins/bitbucket/models/repo.go           |  22 +-
 .../bitbucket/models/transformation_rule.go        |  41 +++
 .../plugins/bitbucket/tasks/account_convertor.go   |  16 +-
 .../plugins/bitbucket/tasks/account_extractor.go   |  14 +-
 backend/plugins/bitbucket/tasks/api_common.go      |  66 +++--
 .../plugins/bitbucket/tasks/commit_collector.go    |   2 +-
 .../plugins/bitbucket/tasks/commit_convertor.go    |  28 +--
 .../plugins/bitbucket/tasks/commit_extractor.go    |  23 +-
 .../bitbucket/tasks/deployment_collector.go        |  12 +-
 .../bitbucket/tasks/deployment_convertor.go        |  16 +-
 .../bitbucket/tasks/deployment_extractor.go        |  44 ++--
 backend/plugins/bitbucket/tasks/issue_collector.go |  31 ++-
 .../bitbucket/tasks/issue_comment_collector.go     |  29 ++-
 .../bitbucket/tasks/issue_comment_convertor.go     |  23 +-
 .../bitbucket/tasks/issue_comment_extractor.go     |  21 +-
 backend/plugins/bitbucket/tasks/issue_convertor.go |  26 +-
 backend/plugins/bitbucket/tasks/issue_extractor.go |  58 ++---
 .../plugins/bitbucket/tasks/pipeline_collector.go  |  28 ++-
 .../plugins/bitbucket/tasks/pipeline_convertor.go  |  16 +-
 .../plugins/bitbucket/tasks/pipeline_extractor.go  |  57 ++---
 backend/plugins/bitbucket/tasks/pr_collector.go    |  29 ++-
 .../bitbucket/tasks/pr_comment_collector.go        |  27 +-
 .../bitbucket/tasks/pr_comment_convertor.go        |  20 +-
 .../bitbucket/tasks/pr_comment_extractor.go        |  33 +--
 .../plugins/bitbucket/tasks/pr_commit_collector.go |  37 ++-
 .../plugins/bitbucket/tasks/pr_commit_convertor.go |  20 +-
 .../plugins/bitbucket/tasks/pr_commit_extractor.go |  51 +---
 backend/plugins/bitbucket/tasks/pr_convertor.go    |  29 +--
 backend/plugins/bitbucket/tasks/pr_extractor.go    |  63 ++---
 backend/plugins/bitbucket/tasks/repo_collector.go  |  63 -----
 backend/plugins/bitbucket/tasks/repo_convertor.go  | 112 +++++++--
 backend/plugins/bitbucket/tasks/repo_extractor.go  | 125 ----------
 backend/plugins/bitbucket/tasks/task_data.go       |  56 +++--
 backend/plugins/gitlab/api/remote.go               |  14 +-
 .../src/pages/pipeline/components/task/index.tsx   |   5 +-
 .../src/pages/transformation/detail/index.tsx      |   5 +
 .../data-scope-list/use-data-scope-list.ts         |   6 +
 .../src/plugins/components/data-scope/index.tsx    |   5 +
 .../components/data-scope/use-data-scope.ts        |   2 +
 .../plugins/components/transformation/index.tsx    |   5 +
 .../transformation/use-transformation.ts           |   2 +
 .../src/plugins/register/bitbucket/config.tsx      |  14 +-
 .../src/plugins/register/bitbucket/data-scope.tsx  |  49 ++++
 config-ui/src/plugins/register/bitbucket/index.ts  |   2 +
 config-ui/src/plugins/register/bitbucket/styled.ts |  89 +++++++
 .../plugins/register/bitbucket/transformation.tsx  | 274 +++++++++++++++++++++
 .../register/bitbucket/{index.ts => types.ts}      |   7 +-
 92 files changed, 2737 insertions(+), 1795 deletions(-)

diff --git a/backend/core/models/domainlayer/devops/cicd_pipeline.go b/backend/core/models/domainlayer/devops/cicd_pipeline.go
index f1132ade0..ce603911c 100644
--- a/backend/core/models/domainlayer/devops/cicd_pipeline.go
+++ b/backend/core/models/domainlayer/devops/cicd_pipeline.go
@@ -84,8 +84,8 @@ func GetResult(rule *ResultRule, input interface{}) string {
 			return ABORT
 		}
 	}
-	for _, abort := range rule.Manual {
-		if abort == input {
+	for _, manual := range rule.Manual {
+		if manual == input {
 			return MANUAL
 		}
 	}
@@ -104,8 +104,8 @@ func GetStatus(rule *StatusRule, input interface{}) string {
 			return FAILURE
 		}
 	}
-	for _, abort := range rule.Manual {
-		if abort == input {
+	for _, manual := range rule.Manual {
+		if manual == input {
 			return MANUAL
 		}
 	}
diff --git a/backend/plugins/bitbucket/api/blueprint.go b/backend/plugins/bitbucket/api/blueprint.go
deleted file mode 100644
index 076c5a6e2..000000000
--- a/backend/plugins/bitbucket/api/blueprint.go
+++ /dev/null
@@ -1,181 +0,0 @@
-/*
-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 api
-
-import (
-	"context"
-	"encoding/json"
-	"fmt"
-	"io"
-	"net/http"
-	"net/url"
-	"path"
-
-	"github.com/apache/incubator-devlake/core/errors"
-	"github.com/apache/incubator-devlake/core/models/domainlayer/didgen"
-	"github.com/apache/incubator-devlake/core/plugin"
-	"github.com/apache/incubator-devlake/core/utils"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	aha "github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
-	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
-	"github.com/apache/incubator-devlake/plugins/bitbucket/tasks"
-)
-
-func MakePipelinePlan(subtaskMetas []plugin.SubTaskMeta, connectionId uint64, scope []*plugin.BlueprintScopeV100) (plugin.PipelinePlan, errors.Error) {
-	connection := new(models.BitbucketConnection)
-	err := connectionHelper.FirstById(connection, connectionId)
-	if err != nil {
-		return nil, err
-	}
-	apiClient, err := api.NewApiClientFromConnection(context.TODO(), basicRes, connection)
-	if err != nil {
-		return nil, err
-	}
-	return makePipelinePlan(subtaskMetas, scope, apiClient, connection)
-}
-
-func makePipelinePlan(
-	subtaskMetas []plugin.SubTaskMeta,
-	scope []*plugin.BlueprintScopeV100,
-	apiClient aha.ApiClientAbstract,
-	connection *models.BitbucketConnection,
-) (plugin.PipelinePlan, errors.Error) {
-	var err errors.Error
-	plan := make(plugin.PipelinePlan, len(scope))
-	var repo *tasks.BitbucketApiRepo
-	for i, scopeElem := range scope {
-		// handle taskOptions and transformationRules, by dumping them to taskOptions
-		transformationRules := make(map[string]interface{})
-		if len(scopeElem.Transformation) > 0 {
-			err = errors.Convert(json.Unmarshal(scopeElem.Transformation, &transformationRules))
-			if err != nil {
-				return nil, err
-			}
-		}
-		// refdiff
-		if refdiffRules, ok := transformationRules["refdiff"]; ok && refdiffRules != nil {
-			// add a new task to next stage
-			j := i + 1
-			if j == len(plan) {
-				plan = append(plan, nil)
-			}
-			plan[j] = plugin.PipelineStage{
-				{
-					Plugin:  "refdiff",
-					Options: refdiffRules.(map[string]interface{}),
-				},
-			}
-			// remove it from bitbucket transformationRules
-			delete(transformationRules, "refdiff")
-		}
-		// construct task options for bitbucket
-		options := make(map[string]interface{})
-		err = errors.Convert(json.Unmarshal(scopeElem.Options, &options))
-		if err != nil {
-			return nil, err
-		}
-		options["connectionId"] = connection.ID
-		options["transformationRules"] = transformationRules
-		// make sure task options is valid
-		op, err := tasks.DecodeAndValidateTaskOptions(options)
-		if err != nil {
-			return nil, err
-		}
-		memorizedGetApiRepo := func() (*tasks.BitbucketApiRepo, errors.Error) {
-			if repo == nil {
-				repo, err = getApiRepo(op, apiClient)
-			}
-			return repo, err
-		}
-		// construct subtasks
-		subtasks, err := api.MakePipelinePlanSubtasks(subtaskMetas, scopeElem.Entities)
-		if err != nil {
-			return nil, err
-		}
-		stage := plan[i]
-		if stage == nil {
-			stage = plugin.PipelineStage{}
-		}
-		stage = append(stage, &plugin.PipelineTask{
-			Plugin:   "bitbucket",
-			Subtasks: subtasks,
-			Options:  options,
-		})
-		// collect git data by gitextractor if CODE was requested
-		if utils.StringsContains(scopeElem.Entities, plugin.DOMAIN_TYPE_CODE) {
-			repo, err = memorizedGetApiRepo()
-			if err != nil {
-				return nil, err
-			}
-			originalUrl := ""
-			for _, u := range repo.Links.Clone {
-				if u.Name == "https" {
-					originalUrl = u.Href
-				}
-			}
-			cloneUrl, err := errors.Convert01(url.Parse(originalUrl))
-			if err != nil {
-				return nil, err
-			}
-			cloneUrl.User = url.UserPassword(connection.Username, connection.Password)
-			stage = append(stage, &plugin.PipelineTask{
-				Plugin: "gitextractor",
-				Options: map[string]interface{}{
-					"url":    cloneUrl.String(),
-					"repoId": didgen.NewDomainIdGenerator(&models.BitbucketRepo{}).Generate(connection.ID, fmt.Sprintf("%s/%s", op.Owner, op.Repo)),
-				},
-			})
-
-		}
-		plan[i] = stage
-		repo = nil
-	}
-	return plan, nil
-}
-
-func getApiRepo(
-	op *tasks.BitbucketOptions,
-	apiClient aha.ApiClientAbstract,
-) (*tasks.BitbucketApiRepo, errors.Error) {
-	res, err := apiClient.Get(path.Join("repositories", op.Owner, op.Repo), nil, nil)
-	if err != nil {
-		return nil, err
-	}
-	defer res.Body.Close()
-	if res.StatusCode != http.StatusOK {
-		return nil, errors.Default.New(fmt.Sprintf(
-			"unexpected status code when requesting repo detail %d %s",
-			res.StatusCode, res.Request.URL.String(),
-		))
-	}
-	body, err := errors.Convert01(io.ReadAll(res.Body))
-	if err != nil {
-		return nil, err
-	}
-	apiRepo := new(tasks.BitbucketApiRepo)
-	err = errors.Convert(json.Unmarshal(body, apiRepo))
-	if err != nil {
-		return nil, err
-	}
-	for _, u := range apiRepo.Links.Clone {
-		if u.Name == "https" {
-			return apiRepo, nil
-		}
-	}
-	return nil, errors.Default.New("no clone url")
-}
diff --git a/backend/plugins/bitbucket/api/blueprint_V200_test.go b/backend/plugins/bitbucket/api/blueprint_V200_test.go
new file mode 100644
index 000000000..95064186d
--- /dev/null
+++ b/backend/plugins/bitbucket/api/blueprint_V200_test.go
@@ -0,0 +1,172 @@
+/*
+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 api
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-devlake/core/models/common"
+	"github.com/apache/incubator-devlake/core/models/domainlayer"
+	"github.com/apache/incubator-devlake/core/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/core/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/core/plugin"
+	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	mockcontext "github.com/apache/incubator-devlake/mocks/core/context"
+	mockdal "github.com/apache/incubator-devlake/mocks/core/dal"
+	mockplugin "github.com/apache/incubator-devlake/mocks/core/plugin"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/mock"
+)
+
+func TestMakeDataSourcePipelinePlanV200(t *testing.T) {
+	connection := &models.BitbucketConnection{
+		BaseConnection: helper.BaseConnection{
+			Name: "bitbucket-test",
+			Model: common.Model{
+				ID: 1,
+			},
+		},
+		BitbucketConn: models.BitbucketConn{
+			RestConnection: helper.RestConnection{
+				Endpoint:         "https://api.bitbucket.org/2.0/",
+				Proxy:            "",
+				RateLimitPerHour: 0,
+			},
+			BasicAuth: helper.BasicAuth{
+				Username: "Username",
+				Password: "Password",
+			},
+		},
+	}
+	mockMeta := mockplugin.NewPluginMeta(t)
+	mockMeta.On("RootPkgPath").Return("github.com/apache/incubator-devlake/plugins/bitbucket")
+	err := plugin.RegisterPlugin("bitbucket", mockMeta)
+	assert.Nil(t, err)
+	// Refresh Global Variables and set the sql mock
+	basicRes = NewMockBasicRes()
+	bs := &plugin.BlueprintScopeV200{
+		Entities: []string{"CODE", "TICKET"},
+		Id:       "1",
+	}
+	bpScopes := make([]*plugin.BlueprintScopeV200, 0)
+	bpScopes = append(bpScopes, bs)
+	syncPolicy := &plugin.BlueprintSyncPolicy{}
+
+	plan := make(plugin.PipelinePlan, len(bpScopes))
+	plan, err = makeDataSourcePipelinePlanV200(nil, plan, bpScopes, connection, syncPolicy)
+	assert.Nil(t, err)
+	basicRes = NewMockBasicRes()
+	scopes, err := makeScopesV200(bpScopes, connection)
+	assert.Nil(t, err)
+
+	expectPlan := plugin.PipelinePlan{
+		plugin.PipelineStage{
+			{
+				Plugin:   "bitbucket",
+				Subtasks: []string{},
+				Options: map[string]interface{}{
+					"fullName":     "likyh/likyhphp",
+					"connectionId": uint64(1),
+				},
+			},
+			{
+				Plugin: "gitextractor",
+				Options: map[string]interface{}{
+					"proxy":  "",
+					"repoId": "bitbucket:BitbucketRepo:1:likyh/likyhphp",
+					"url":    "https://git:Password@this_is_cloneUrl",
+				},
+			},
+		},
+		plugin.PipelineStage{
+			{
+				Plugin: "refdiff",
+				Options: map[string]interface{}{
+					"repoId":      "bitbucket:BitbucketRepo:1:likyh/likyhphp",
+					"tagsLimit":   10,
+					"tagsOrder":   "reverse semver",
+					"tagsPattern": "pattern",
+				},
+			},
+		},
+	}
+	assert.Equal(t, expectPlan, plan)
+	expectScopes := make([]plugin.Scope, 0)
+	scopeRepo := &code.Repo{
+		DomainEntity: domainlayer.DomainEntity{
+			Id: "bitbucket:BitbucketRepo:1:likyh/likyhphp",
+		},
+		Name: "test/testRepo",
+	}
+
+	scopeTicket := &ticket.Board{
+		DomainEntity: domainlayer.DomainEntity{
+			Id: "bitbucket:BitbucketRepo:1:likyh/likyhphp",
+		},
+		Name:        "test/testRepo",
+		Description: "",
+		Url:         "",
+		CreatedDate: nil,
+		Type:        "",
+	}
+
+	expectScopes = append(expectScopes, scopeRepo, scopeTicket)
+	assert.Equal(t, expectScopes, scopes)
+}
+
+// NewMockBasicRes FIXME ...
+func NewMockBasicRes() *mockcontext.BasicRes {
+	testBitbucketRepo := &models.BitbucketRepo{
+		ConnectionId:         1,
+		BitbucketId:          "likyh/likyhphp",
+		Name:                 "test/testRepo",
+		CloneUrl:             "https://this_is_cloneUrl",
+		TransformationRuleId: 1,
+	}
+
+	testTransformationRule := &models.BitbucketTransformationRule{
+		Model: common.Model{
+			ID: 1,
+		},
+		Name:            "Bitbucket transformation rule",
+		IssueStatusTodo: "new,open,wantfix",
+		Refdiff: map[string]interface{}{
+			"tagsPattern": "pattern",
+			"tagsLimit":   10,
+			"tagsOrder":   "reverse semver",
+		},
+	}
+	mockRes := new(mockcontext.BasicRes)
+	mockDal := new(mockdal.Dal)
+
+	mockDal.On("First", mock.Anything, mock.Anything).Run(func(args mock.Arguments) {
+		dst := args.Get(0).(*models.BitbucketRepo)
+		*dst = *testBitbucketRepo
+	}).Return(nil).Once()
+
+	mockDal.On("First", mock.Anything, mock.Anything).Run(func(args mock.Arguments) {
+		dst := args.Get(0).(*models.BitbucketTransformationRule)
+		*dst = *testTransformationRule
+	}).Return(nil).Once()
+
+	mockRes.On("GetDal").Return(mockDal)
+	mockRes.On("GetConfig", mock.Anything).Return("")
+
+	return mockRes
+}
diff --git a/backend/plugins/bitbucket/api/blueprint_test.go b/backend/plugins/bitbucket/api/blueprint_test.go
deleted file mode 100644
index 6db0aaa1e..000000000
--- a/backend/plugins/bitbucket/api/blueprint_test.go
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
-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 api
-
-import (
-	"bytes"
-	"encoding/json"
-	"io"
-	"net/http"
-	"path"
-	"testing"
-
-	"github.com/apache/incubator-devlake/core/models/common"
-	"github.com/apache/incubator-devlake/core/plugin"
-	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	mockplugin "github.com/apache/incubator-devlake/mocks/core/plugin"
-	mockaha "github.com/apache/incubator-devlake/mocks/helpers/pluginhelper/api/apihelperabstract"
-	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
-	"github.com/apache/incubator-devlake/plugins/bitbucket/tasks"
-	"github.com/stretchr/testify/assert"
-	"github.com/stretchr/testify/mock"
-)
-
-func TestMakePipelinePlan(t *testing.T) {
-	connection := &models.BitbucketConnection{
-		BaseConnection: helper.BaseConnection{
-			Name: "github-test",
-			Model: common.Model{
-				ID: 1,
-			},
-		},
-		BitbucketConn: models.BitbucketConn{
-			RestConnection: helper.RestConnection{
-				Endpoint:         "https://TestBitBucket/",
-				Proxy:            "",
-				RateLimitPerHour: 0,
-			},
-			BasicAuth: helper.BasicAuth{
-				Username: "Username",
-				Password: "Password",
-			},
-		},
-	}
-
-	mockApiCLient := mockaha.NewApiClientAbstract(t)
-	repo := &tasks.BitbucketApiRepo{
-		Links: struct {
-			Clone []struct {
-				Href string `json:"href"`
-				Name string `json:"name"`
-			} `json:"clone"`
-			Self struct {
-				Href string `json:"href"`
-			} `json:"self"`
-			Html struct {
-				Href string `json:"href"`
-			} `json:"html"`
-		}{
-			Clone: []struct {
-				Href string `json:"href"`
-				Name string `json:"name"`
-			}{
-				{
-					Href: "https://bitbucket.org/thenicetgp/lake.git",
-					Name: "https",
-				},
-			},
-			Self: struct {
-				Href string `json:"href"`
-			}{},
-			Html: struct {
-				Href string `json:"href"`
-			}{},
-		},
-	}
-	js, err := json.Marshal(repo)
-	assert.Nil(t, err)
-	res := &http.Response{}
-	res.Body = io.NopCloser(bytes.NewBuffer(js))
-	res.StatusCode = http.StatusOK
-	mockApiCLient.On("Get", path.Join("repositories", "thenicetgp", "lake"), mock.Anything, mock.Anything).Return(res, nil)
-	mockMeta := mockplugin.NewPluginMeta(t)
-	mockMeta.On("RootPkgPath").Return("github.com/apache/incubator-devlake/plugins/bitbucket")
-	err = plugin.RegisterPlugin("bitbucket", mockMeta)
-	scope := &plugin.BlueprintScopeV100{
-		Entities: []string{plugin.DOMAIN_TYPE_CODE, plugin.DOMAIN_TYPE_TICKET, plugin.DOMAIN_TYPE_CODE_REVIEW, plugin.DOMAIN_TYPE_CROSS},
-		Options: []byte(`{
-                            "owner": "thenicetgp",
-                            "repo": "lake"
-                        }`),
-		Transformation: nil,
-	}
-	scopes := make([]*plugin.BlueprintScopeV100, 0)
-	scopes = append(scopes, scope)
-	assert.Nil(t, err)
-	plan, err := makePipelinePlan(nil, scopes, mockApiCLient, connection)
-	assert.Nil(t, err)
-
-	expectPlan := plugin.PipelinePlan{
-		plugin.PipelineStage{
-			{
-				Plugin:   "bitbucket",
-				Subtasks: []string{},
-				Options: map[string]interface{}{
-					"connectionId":        uint64(1),
-					"owner":               "thenicetgp",
-					"repo":                "lake",
-					"transformationRules": map[string]interface{}{},
-				},
-			},
-			{
-				Plugin: "gitextractor",
-				Options: map[string]interface{}{
-					"repoId": "bitbucket:BitbucketRepo:1:thenicetgp/lake",
-					"url":    "https://Username:Password@bitbucket.org/thenicetgp/lake.git",
-				},
-			},
-		},
-	}
-	assert.Equal(t, expectPlan, plan)
-}
diff --git a/backend/plugins/bitbucket/api/blueprint_v200.go b/backend/plugins/bitbucket/api/blueprint_v200.go
new file mode 100644
index 000000000..92a338fb7
--- /dev/null
+++ b/backend/plugins/bitbucket/api/blueprint_v200.go
@@ -0,0 +1,196 @@
+/*
+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 api
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/core/dal"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/models/domainlayer"
+	"github.com/apache/incubator-devlake/core/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/core/models/domainlayer/devops"
+	"github.com/apache/incubator-devlake/core/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/core/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/core/utils"
+	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/tasks"
+	"github.com/go-playground/validator/v10"
+	"net/url"
+	"time"
+)
+
+func MakeDataSourcePipelinePlanV200(subtaskMetas []plugin.SubTaskMeta, connectionId uint64, bpScopes []*plugin.BlueprintScopeV200, syncPolicy *plugin.BlueprintSyncPolicy) (plugin.PipelinePlan, []plugin.Scope, errors.Error) {
+	connectionHelper := helper.NewConnectionHelper(basicRes, validator.New())
+	// get the connection info for url
+	connection := &models.BitbucketConnection{}
+	err := connectionHelper.FirstById(connection, connectionId)
+	if err != nil {
+		return nil, nil, err
+	}
+
+	plan := make(plugin.PipelinePlan, len(bpScopes))
+	plan, err = makeDataSourcePipelinePlanV200(subtaskMetas, plan, bpScopes, connection, syncPolicy)
+	if err != nil {
+		return nil, nil, err
+	}
+	scopes, err := makeScopesV200(bpScopes, connection)
+	if err != nil {
+		return nil, nil, err
+	}
+
+	return plan, scopes, nil
+}
+
+func makeDataSourcePipelinePlanV200(
+	subtaskMetas []plugin.SubTaskMeta,
+	plan plugin.PipelinePlan,
+	bpScopes []*plugin.BlueprintScopeV200,
+	connection *models.BitbucketConnection,
+	syncPolicy *plugin.BlueprintSyncPolicy,
+) (plugin.PipelinePlan, errors.Error) {
+	var err errors.Error
+	for i, bpScope := range bpScopes {
+		stage := plan[i]
+		if stage == nil {
+			stage = plugin.PipelineStage{}
+		}
+		repo := &models.BitbucketRepo{}
+		// get repo from db
+		err = basicRes.GetDal().First(repo, dal.Where(`connection_id = ? AND bitbucket_id = ?`, connection.ID, bpScope.Id))
+		if err != nil {
+			return nil, errors.Default.Wrap(err, fmt.Sprintf("fail to find repo %s", bpScope.Id))
+		}
+		transformationRule := &models.BitbucketTransformationRule{}
+		// get transformation rules from db
+		db := basicRes.GetDal()
+		err = db.First(transformationRule, dal.Where(`id = ?`, repo.TransformationRuleId))
+		if err != nil && !db.IsErrorNotFound(err) {
+			return nil, err
+		}
+		// refdiff
+		if transformationRule != nil && transformationRule.Refdiff != nil {
+			// add a new task to next stage
+			j := i + 1
+			if j == len(plan) {
+				plan = append(plan, nil)
+			}
+			refdiffOp := transformationRule.Refdiff
+			refdiffOp["repoId"] = didgen.NewDomainIdGenerator(&models.BitbucketRepo{}).Generate(connection.ID, repo.BitbucketId)
+			plan[j] = plugin.PipelineStage{
+				{
+					Plugin:  "refdiff",
+					Options: refdiffOp,
+				},
+			}
+			transformationRule.Refdiff = nil
+		}
+
+		// construct task options for bitbucket
+		op := &tasks.BitbucketOptions{
+			ConnectionId: repo.ConnectionId,
+			FullName:     repo.BitbucketId,
+		}
+		if syncPolicy.CreatedDateAfter != nil {
+			op.CreatedDateAfter = syncPolicy.CreatedDateAfter.Format(time.RFC3339)
+		}
+		options, err := tasks.EncodeTaskOptions(op)
+		if err != nil {
+			return nil, err
+		}
+
+		subtasks, err := helper.MakePipelinePlanSubtasks(subtaskMetas, bpScope.Entities)
+		if err != nil {
+			return nil, err
+		}
+		stage = append(stage, &plugin.PipelineTask{
+			Plugin:   "bitbucket",
+			Subtasks: subtasks,
+			Options:  options,
+		})
+		if err != nil {
+			return nil, err
+		}
+
+		// add gitex stage
+		if utils.StringsContains(bpScope.Entities, plugin.DOMAIN_TYPE_CODE) {
+			cloneUrl, err := errors.Convert01(url.Parse(repo.CloneUrl))
+			if err != nil {
+				return nil, err
+			}
+			cloneUrl.User = url.UserPassword("git", connection.Password)
+			stage = append(stage, &plugin.PipelineTask{
+				Plugin: "gitextractor",
+				Options: map[string]interface{}{
+					"url":    cloneUrl.String(),
+					"repoId": didgen.NewDomainIdGenerator(&models.BitbucketRepo{}).Generate(connection.ID, repo.BitbucketId),
+					"proxy":  connection.Proxy,
+				},
+			})
+
+		}
+		plan[i] = stage
+	}
+	return plan, nil
+}
+
+func makeScopesV200(bpScopes []*plugin.BlueprintScopeV200, connection *models.BitbucketConnection) ([]plugin.Scope, errors.Error) {
+	scopes := make([]plugin.Scope, 0)
+	for _, bpScope := range bpScopes {
+		repo := &models.BitbucketRepo{}
+		// get repo from db
+		err := basicRes.GetDal().First(repo, dal.Where(`connection_id = ? AND bitbucket_id = ?`, connection.ID, bpScope.Id))
+		if err != nil {
+			return nil, errors.Default.Wrap(err, fmt.Sprintf("fail to find repo%s", bpScope.Id))
+		}
+		if utils.StringsContains(bpScope.Entities, plugin.DOMAIN_TYPE_CODE_REVIEW) ||
+			utils.StringsContains(bpScope.Entities, plugin.DOMAIN_TYPE_CODE) ||
+			utils.StringsContains(bpScope.Entities, plugin.DOMAIN_TYPE_CROSS) {
+			// if we don't need to collect gitex, we need to add repo to scopes here
+			scopeRepo := &code.Repo{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: didgen.NewDomainIdGenerator(&models.BitbucketRepo{}).Generate(connection.ID, repo.BitbucketId),
+				},
+				Name: repo.Name,
+			}
+			scopes = append(scopes, scopeRepo)
+		}
+		// add cicd_scope to scopes
+		if utils.StringsContains(bpScope.Entities, plugin.DOMAIN_TYPE_CICD) {
+			scopeCICD := &devops.CicdScope{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: didgen.NewDomainIdGenerator(&models.BitbucketRepo{}).Generate(connection.ID, repo.BitbucketId),
+				},
+				Name: repo.Name,
+			}
+			scopes = append(scopes, scopeCICD)
+		}
+		// add board to scopes
+		if utils.StringsContains(bpScope.Entities, plugin.DOMAIN_TYPE_TICKET) {
+			scopeTicket := &ticket.Board{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: didgen.NewDomainIdGenerator(&models.BitbucketRepo{}).Generate(connection.ID, repo.BitbucketId),
+				},
+				Name: repo.Name,
+			}
+			scopes = append(scopes, scopeTicket)
+		}
+	}
+	return scopes, nil
+}
diff --git a/backend/plugins/gitlab/api/remote.go b/backend/plugins/bitbucket/api/remote.go
similarity index 60%
copy from backend/plugins/gitlab/api/remote.go
copy to backend/plugins/bitbucket/api/remote.go
index 4478158ba..9fc116842 100644
--- a/backend/plugins/gitlab/api/remote.go
+++ b/backend/plugins/bitbucket/api/remote.go
@@ -22,15 +22,15 @@ import (
 	"encoding/base64"
 	"encoding/json"
 	"fmt"
-	"net/http"
-	"net/url"
-	"strconv"
-
 	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/apache/incubator-devlake/core/plugin"
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	"github.com/apache/incubator-devlake/plugins/gitlab/models"
-	"github.com/apache/incubator-devlake/plugins/gitlab/tasks"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/tasks"
+	"net/http"
+	"net/url"
+	"strconv"
+	"strings"
 )
 
 type RemoteScopesChild struct {
@@ -53,36 +53,45 @@ type SearchRemoteScopesOutput struct {
 }
 
 type PageData struct {
-	Page    int    `json:"page"`
-	PerPage int    `json:"per_page"`
-	Tag     string `json:"tag"`
+	Page    int `json:"page"`
+	PerPage int `json:"per_page"`
 }
 
-type GroupResponse struct {
-	Id                   int    `json:"id"`
-	WebUrl               string `json:"web_url"`
-	Name                 string `json:"name"`
-	Path                 string `json:"path"`
-	Description          string `json:"description"`
-	Visibility           string `json:"visibility"`
-	LfsEnabled           bool   `json:"lfs_enabled"`
-	AvatarUrl            string `json:"avatar_url"`
-	RequestAccessEnabled bool   `json:"request_access_enabled"`
-	FullName             string `json:"full_name"`
-	FullPath             string `json:"full_path"`
-	ParentId             *int   `json:"parent_id"`
-	LdapCN               string `json:"ldap_cn"`
-	LdapAccess           string `json:"ldap_access"`
+type WorkspaceResponse struct {
+	Pagelen int `json:"pagelen"`
+	Page    int `json:"page"`
+	Size    int `json:"size"`
+	Values  []struct {
+		//Type       string `json:"type"`
+		//Permission string `json:"permission"`
+		//LastAccessed time.Time `json:"last_accessed"`
+		//AddedOn      time.Time `json:"added_on"`
+		Workspace WorkspaceItem `json:"workspace"`
+	} `json:"values"`
 }
 
-const GitlabRemoteScopesPerPage int = 100
-const TypeProject string = "scope"
+type WorkspaceItem struct {
+	//Type string `json:"type"`
+	//Uuid string `json:"uuid"`
+	Slug string `json:"slug"`
+	Name string `json:"name"`
+}
+
+type ReposResponse struct {
+	Pagelen int                      `json:"pagelen"`
+	Page    int                      `json:"page"`
+	Size    int                      `json:"size"`
+	Values  []tasks.BitbucketApiRepo `json:"values"`
+}
+
+const RemoteScopesPerPage int = 100
+const TypeScope string = "scope"
 const TypeGroup string = "group"
 
 // RemoteScopes list all available scope for users
 // @Summary list all available scope for users
 // @Description list all available scope for users
-// @Tags plugins/gitlab
+// @Tags plugins/bitbucket
 // @Accept application/json
 // @Param connectionId path int false "connection ID"
 // @Param groupId query string false "group ID"
@@ -90,14 +99,14 @@ const TypeGroup string = "group"
 // @Success 200  {object} RemoteScopesOutput
 // @Failure 400  {object} shared.ApiBody "Bad Request"
 // @Failure 500  {object} shared.ApiBody "Internal Error"
-// @Router /plugins/gitlab/connections/{connectionId}/remote-scopes [GET]
+// @Router /plugins/bitbucket/connections/{connectionId}/remote-scopes [GET]
 func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
 	connectionId, _ := extractParam(input.Params)
 	if connectionId == 0 {
 		return nil, errors.BadInput.New("invalid connectionId")
 	}
 
-	connection := &models.GitlabConnection{}
+	connection := &models.BitbucketConnection{}
 	err := connectionHelper.First(connection, input.Params)
 	if err != nil {
 		return nil, err
@@ -117,7 +126,7 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 	gid := groupId[0]
 	pageData, err := GetPageDataFromPageToken(pageToken[0])
 	if err != nil {
-		return nil, errors.BadInput.New("failed to get paget token")
+		return nil, errors.BadInput.New("failed to get page token")
 	}
 
 	// create api client
@@ -126,114 +135,74 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 		return nil, err
 	}
 
+	query, err := GetQueryFromPageData(pageData)
+	if err != nil {
+		return nil, err
+	}
+
 	var res *http.Response
 	outputBody := &RemoteScopesOutput{}
 
 	// list groups part
-	if pageData.Tag == TypeGroup {
-		query, err := GetQueryFromPageData(pageData)
+	if gid == "" {
+		query.Set("sort", "workspace.slug")
+		query.Set("fields", "values.workspace.slug,values.workspace.name,pagelen,page,size")
+		res, err = apiClient.Get("/user/permissions/workspaces", query, nil)
 		if err != nil {
 			return nil, err
 		}
 
-		if gid == "" {
-			query.Set("top_level_only", "true")
-			res, err = apiClient.Get("groups", query, nil)
-		} else {
-			res, err = apiClient.Get(fmt.Sprintf("groups/%s/subgroups", gid), query, nil)
-		}
+		resBody := &WorkspaceResponse{}
+		err = api.UnmarshalResponse(res, resBody)
 		if err != nil {
 			return nil, err
 		}
 
-		resBody := []GroupResponse{}
-		err = api.UnmarshalResponse(res, &resBody)
-
 		// append group to output
-		for _, group := range resBody {
+		for _, group := range resBody.Values {
 			child := RemoteScopesChild{
 				Type: TypeGroup,
-				Id:   strconv.Itoa(group.Id),
-				Name: group.Name,
+				Id:   group.Workspace.Slug,
+				Name: group.Workspace.Name,
 				// don't need to save group into data
 				Data: nil,
 			}
-
-			// ignore not top_level
-			if group.ParentId == nil {
-				if gid != "" {
-					continue
-				}
-			} else {
-				if strconv.Itoa(*group.ParentId) != gid {
-					continue
-				}
-			}
-
-			// ignore self
-			if gid == child.Id {
-				continue
-			}
-
-			child.ParentId = &gid
-			if *child.ParentId == "" {
-				child.ParentId = nil
-			}
-
 			outputBody.Children = append(outputBody.Children, child)
 		}
 
 		// check groups count
-		if err != nil {
-			return nil, err
-		}
-		if len(resBody) < pageData.PerPage {
-			pageData.Tag = TypeProject
-			pageData.Page = 1
-			pageData.PerPage = pageData.PerPage - len(resBody)
+		if resBody.Size < pageData.PerPage {
+			pageData = nil
 		}
-	}
-
-	// list projects part
-	if pageData.Tag == TypeProject {
-		query, err := GetQueryFromPageData(pageData)
+	} else {
+		query.Set("sort", "name")
+		query.Set("fields", "values.name,values.full_name,values.language,values.description,values.owner.username,values.created_on,values.updated_on,values.links.clone,values.links.self,pagelen,page,size")
+		// list projects part
+		res, err = apiClient.Get(fmt.Sprintf("/repositories/%s", gid), query, nil)
 		if err != nil {
 			return nil, err
 		}
-		if gid == "" {
-			res, err = apiClient.Get(fmt.Sprintf("users/%d/projects", apiClient.GetData(models.GitlabApiClientData_UserId)), query, nil)
-		} else {
-			query.Set("with_shared", "false")
-			res, err = apiClient.Get(fmt.Sprintf("/groups/%s/projects", gid), query, nil)
-		}
+
+		resBody := &ReposResponse{}
+		err = api.UnmarshalResponse(res, resBody)
 		if err != nil {
 			return nil, err
 		}
 
-		resBody := []tasks.GitlabApiProject{}
-		err = api.UnmarshalResponse(res, &resBody)
-
-		// append project to output
-		for _, project := range resBody {
+		// append repo to output
+		for _, repo := range resBody.Values {
 			child := RemoteScopesChild{
-				Type: TypeProject,
-				Id:   strconv.Itoa(project.GitlabId),
-				Name: project.Name,
-				Data: tasks.ConvertProject(&project),
+				Type: TypeScope,
+				Id:   repo.FullName,
+				Name: repo.Name,
+				Data: tasks.ConvertApiRepoToScope(&repo, connection.ID),
 			}
 			child.ParentId = &gid
-			if *child.ParentId == "" {
-				child.ParentId = nil
-			}
-
 			outputBody.Children = append(outputBody.Children, child)
 		}
 
-		// check project count
-		if err != nil {
-			return nil, err
-		}
-		if len(resBody) < pageData.PerPage {
+		// check repo count
+		if resBody.Size < pageData.PerPage {
 			pageData = nil
 		}
 	}
@@ -242,8 +211,6 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 	outputBody.NextPageToken = ""
 	if pageData != nil {
 		pageData.Page += 1
-		pageData.PerPage = GitlabRemoteScopesPerPage
-
 		outputBody.NextPageToken, err = GetPageTokenFromPageData(pageData)
 		if err != nil {
 			return nil, err
@@ -256,23 +223,23 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 // SearchRemoteScopes use the Search API and only return project
 // @Summary use the Search API and only return project
 // @Description use the Search API and only return project
-// @Tags plugins/gitlab
+// @Tags plugins/bitbucket
 // @Accept application/json
 // @Param connectionId path int false "connection ID"
-// @Param search query string false "group ID"
+// @Param search query string false "search"
 // @Param page query int false "page number"
 // @Param pageSize query int false "page size per page"
 // @Success 200  {object} SearchRemoteScopesOutput
 // @Failure 400  {object} shared.ApiBody "Bad Request"
 // @Failure 500  {object} shared.ApiBody "Internal Error"
-// @Router /plugins/gitlab/connections/{connectionId}/search-remote-scopes [GET]
+// @Router /plugins/bitbucket/connections/{connectionId}/search-remote-scopes [GET]
 func SearchRemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
 	connectionId, _ := extractParam(input.Params)
 	if connectionId == 0 {
 		return nil, errors.BadInput.New("invalid connectionId")
 	}
 
-	connection := &models.GitlabConnection{}
+	connection := &models.BitbucketConnection{}
 	err := connectionHelper.First(connection, input.Params)
 	if err != nil {
 		return nil, err
@@ -282,28 +249,26 @@ func SearchRemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutp
 	if !ok || len(search) == 0 {
 		search = []string{""}
 	}
+	s := search[0]
 
-	var p int
-	var err1 error
+	p := 1
 	page, ok := input.Query["page"]
-	if !ok || len(page) == 0 {
-		p = 1
-	} else {
-		p, err1 = strconv.Atoi(page[0])
+	if ok && len(page) != 0 {
+		p, err = errors.Convert01(strconv.Atoi(page[0]))
 		if err != nil {
-			return nil, errors.BadInput.Wrap(err1, fmt.Sprintf("failed to Atoi page:%s", page[0]))
+			return nil, errors.BadInput.Wrap(err, fmt.Sprintf("failed to Atoi page:%s", page[0]))
 		}
 	}
-	var ps int
+
+	ps := RemoteScopesPerPage
 	pageSize, ok := input.Query["pageSize"]
-	if !ok || len(pageSize) == 0 {
-		ps = GitlabRemoteScopesPerPage
-	} else {
-		ps, err1 = strconv.Atoi(pageSize[0])
-		if err1 != nil {
-			return nil, errors.BadInput.Wrap(err1, fmt.Sprintf("failed to Atoi pageSize:%s", pageSize[0]))
+	if ok && len(pageSize) != 0 {
+		ps, err = errors.Convert01(strconv.Atoi(pageSize[0]))
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, fmt.Sprintf("failed to Atoi pageSize:%s", pageSize[0]))
 		}
 	}
+
 	// create api client
 	apiClient, err := api.NewApiClientFromConnection(context.TODO(), basicRes, connection)
 	if err != nil {
@@ -311,34 +276,42 @@ func SearchRemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutp
 	}
 
 	// set query
-	query, err := GetQueryForSearchProject(search[0], p, ps)
+	query, err := GetQueryFromPageData(&PageData{p, ps})
 	if err != nil {
 		return nil, err
 	}
 
 	// request search
-	res, err := apiClient.Get("search", query, nil)
+	query.Set("sort", "name")
+	query.Set("fields", "values.name,values.full_name,values.language,values.description,values.owner.username,values.created_on,values.updated_on,values.links.clone,values.links.self,pagelen,page,size")
+
+	gid := ``
+	if strings.Contains(s, `/`) {
+		gid = strings.Split(s, `/`)[0]
+		s = strings.Split(s, `/`)[0]
+	}
+	query.Set("q", fmt.Sprintf(`name~"%s"`, s))
+	// list repos part
+	res, err := apiClient.Get(fmt.Sprintf("/repositories/%s", gid), query, nil)
 	if err != nil {
 		return nil, err
 	}
-	resBody := []tasks.GitlabApiProject{}
+	resBody := &ReposResponse{}
 	err = api.UnmarshalResponse(res, &resBody)
 	if err != nil {
 		return nil, err
 	}
 
-	outputBody := &SearchRemoteScopesOutput{}
-
-	// append project to output
-	for _, project := range resBody {
+	// set repos return
+	outputBody := &SearchRemoteScopesOutput{Children: []RemoteScopesChild{}}
+	for _, repo := range resBody.Values {
 		child := RemoteScopesChild{
-			Type:     TypeProject,
-			Id:       strconv.Itoa(project.GitlabId),
+			Type:     TypeScope,
+			Id:       repo.FullName,
 			ParentId: nil,
-			Name:     project.PathWithNamespace,
-			Data:     tasks.ConvertProject(&project),
+			Name:     repo.Name,
+			Data:     tasks.ConvertApiRepoToScope(&repo, connection.ID),
 		}
-
 		outputBody.Children = append(outputBody.Children, child)
 	}
 
@@ -363,8 +336,7 @@ func GetPageDataFromPageToken(pageToken string) (*PageData, errors.Error) {
 	if pageToken == "" {
 		return &PageData{
 			Page:    1,
-			PerPage: GitlabRemoteScopesPerPage,
-			Tag:     "group",
+			PerPage: RemoteScopesPerPage,
 		}, nil
 	}
 
@@ -386,17 +358,6 @@ func GetPageDataFromPageToken(pageToken string) (*PageData, errors.Error) {
 func GetQueryFromPageData(pageData *PageData) (url.Values, errors.Error) {
 	query := url.Values{}
 	query.Set("page", fmt.Sprintf("%v", pageData.Page))
-	query.Set("per_page", fmt.Sprintf("%v", pageData.PerPage))
-	return query, nil
-}
-
-func GetQueryForSearchProject(search string, page int, perPage int) (url.Values, errors.Error) {
-	query, err := GetQueryFromPageData(&PageData{Page: page, PerPage: perPage})
-	if err != nil {
-		return nil, err
-	}
-	query.Set("search", search)
-	query.Set("scope", "projects")
-
+	query.Set("pagelen", fmt.Sprintf("%v", pageData.PerPage))
 	return query, nil
 }
diff --git a/backend/plugins/bitbucket/api/scope.go b/backend/plugins/bitbucket/api/scope.go
new file mode 100644
index 000000000..e9053e56c
--- /dev/null
+++ b/backend/plugins/bitbucket/api/scope.go
@@ -0,0 +1,213 @@
+/*
+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 api
+
+import (
+	"github.com/apache/incubator-devlake/core/dal"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/mitchellh/mapstructure"
+	"net/http"
+	"strconv"
+	"strings"
+)
+
+type apiRepo struct {
+	models.BitbucketRepo
+	TransformationRuleName string `json:"transformationRuleName,omitempty"`
+}
+
+type req struct {
+	Data []*models.BitbucketRepo `json:"data"`
+}
+
+// PutScope create or update repo
+// @Summary create or update repo
+// @Description Create or update repo
+// @Tags plugins/bitbucket
+// @Accept application/json
+// @Param connectionId path int true "connection ID"
+// @Param scope body req true "json"
+// @Success 200  {object} []models.BitbucketRepo
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/connections/{connectionId}/scopes [PUT]
+func PutScope(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connectionId, _ := extractParam(input.Params)
+	if connectionId == 0 {
+		return nil, errors.BadInput.New("invalid connectionId")
+	}
+	var repos req
+	err := errors.Convert(mapstructure.Decode(input.Body, &repos))
+	if err != nil {
+		return nil, errors.BadInput.Wrap(err, "decoding repo error")
+	}
+	keeper := make(map[string]struct{})
+	for _, repo := range repos.Data {
+		if _, ok := keeper[repo.BitbucketId]; ok {
+			return nil, errors.BadInput.New("duplicated item")
+		} else {
+			keeper[repo.BitbucketId] = struct{}{}
+		}
+		repo.ConnectionId = connectionId
+		err = verifyRepo(repo)
+		if err != nil {
+			return nil, err
+		}
+	}
+	err = basicRes.GetDal().CreateOrUpdate(repos.Data)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving BitbucketRepo")
+	}
+	return &plugin.ApiResourceOutput{Body: repos.Data, Status: http.StatusOK}, nil
+}
+
+// UpdateScope patch to repo
+// @Summary patch to repo
+// @Description patch to repo
+// @Tags plugins/bitbucket
+// @Accept application/json
+// @Param connectionId path int true "connection ID"
+// @Param repoId path string true "repo ID"
+// @Param scope body models.BitbucketRepo true "json"
+// @Success 200  {object} models.BitbucketRepo
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/connections/{connectionId}/scopes/{repoId} [PATCH]
+func UpdateScope(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connectionId, repoId := extractParam(input.Params)
+	if connectionId == 0 || repoId == "" {
+		return nil, errors.BadInput.New("invalid connectionId or repoId")
+	}
+	var repo models.BitbucketRepo
+	err := basicRes.GetDal().First(&repo, dal.Where("connection_id = ? AND bitbucket_id = ?", connectionId, repoId))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "getting Repo error")
+	}
+	err = api.DecodeMapStruct(input.Body, &repo)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "patch repo error")
+	}
+	err = verifyRepo(&repo)
+	if err != nil {
+		return nil, err
+	}
+	err = basicRes.GetDal().Update(repo)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving BitbucketRepo")
+	}
+	return &plugin.ApiResourceOutput{Body: repo, Status: http.StatusOK}, nil
+}
+
+// GetScopeList get repos
+// @Summary get repos
+// @Description get repos
+// @Tags plugins/bitbucket
+// @Param connectionId path int true "connection ID"
+// @Param pageSize query int false "page size, default 50"
+// @Param page query int false "page size, default 1"
+// @Success 200  {object} []apiRepo
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/connections/{connectionId}/scopes/ [GET]
+func GetScopeList(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	var repos []models.BitbucketRepo
+	connectionId, _ := extractParam(input.Params)
+	if connectionId == 0 {
+		return nil, errors.BadInput.New("invalid path params")
+	}
+	limit, offset := api.GetLimitOffset(input.Query, "pageSize", "page")
+	err := basicRes.GetDal().All(&repos, dal.Where("connection_id = ?", connectionId), dal.Limit(limit), dal.Offset(offset))
+	if err != nil {
+		return nil, err
+	}
+	var ruleIds []uint64
+	for _, repo := range repos {
+		if repo.TransformationRuleId > 0 {
+			ruleIds = append(ruleIds, repo.TransformationRuleId)
+		}
+	}
+	var rules []models.BitbucketTransformationRule
+	if len(ruleIds) > 0 {
+		err = basicRes.GetDal().All(&rules, dal.Where("id IN (?)", ruleIds))
+		if err != nil {
+			return nil, err
+		}
+	}
+	names := make(map[uint64]string)
+	for _, rule := range rules {
+		names[rule.ID] = rule.Name
+	}
+	var apiRepos []apiRepo
+	for _, repo := range repos {
+		apiRepos = append(apiRepos, apiRepo{repo, names[repo.TransformationRuleId]})
+	}
+	return &plugin.ApiResourceOutput{Body: apiRepos, Status: http.StatusOK}, nil
+}
+
+// GetScope get one repo
+// @Summary get one repo
+// @Description get one repo
+// @Tags plugins/bitbucket
+// @Param connectionId path int true "connection ID"
+// @Param repoId path string true "repo ID"
+// @Success 200  {object} apiRepo
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/connections/{connectionId}/scopes/{repoId} [GET]
+func GetScope(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	var repo models.BitbucketRepo
+	connectionId, repoId := extractParam(input.Params)
+	if connectionId == 0 || repoId == "" {
+		return nil, errors.BadInput.New("invalid connectionId or repoId")
+	}
+	db := basicRes.GetDal()
+	err := db.First(&repo, dal.Where("connection_id = ? AND bitbucket_id = ?", connectionId, repoId))
+	if db.IsErrorNotFound(err) {
+		return nil, errors.NotFound.New("record not found")
+	}
+	if err != nil {
+		return nil, err
+	}
+	var rule models.BitbucketTransformationRule
+	if repo.TransformationRuleId > 0 {
+		err = basicRes.GetDal().First(&rule, dal.Where("id = ?", repo.TransformationRuleId))
+		if err != nil {
+			return nil, err
+		}
+	}
+	return &plugin.ApiResourceOutput{Body: apiRepo{repo, rule.Name}, Status: http.StatusOK}, nil
+}
+
+func extractParam(params map[string]string) (uint64, string) {
+	connectionId, _ := strconv.ParseUint(params["connectionId"], 10, 64)
+	fullName := strings.TrimLeft(params["repoId"], "/")
+	return connectionId, fullName
+}
+
+func verifyRepo(repo *models.BitbucketRepo) errors.Error {
+	if repo.ConnectionId == 0 {
+		return errors.BadInput.New("invalid connectionId")
+	}
+	if repo.BitbucketId == `` {
+		return errors.BadInput.New("invalid bitbucket ID or full name")
+	}
+	return nil
+}
diff --git a/backend/plugins/bitbucket/api/transformation_rule.go b/backend/plugins/bitbucket/api/transformation_rule.go
new file mode 100644
index 000000000..2faa5168c
--- /dev/null
+++ b/backend/plugins/bitbucket/api/transformation_rule.go
@@ -0,0 +1,132 @@
+/*
+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 api
+
+import (
+	"github.com/apache/incubator-devlake/core/dal"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"net/http"
+	"strconv"
+)
+
+// CreateTransformationRule create transformation rule for Bitbucket
+// @Summary create transformation rule for Bitbucket
+// @Description create transformation rule for Bitbucket
+// @Tags plugins/bitbucket
+// @Accept application/json
+// @Param transformationRule body models.BitbucketTransformationRule true "transformation rule"
+// @Success 200  {object} models.BitbucketTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/transformation_rules [POST]
+func CreateTransformationRule(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	var rule models.BitbucketTransformationRule
+	err := api.Decode(input.Body, &rule, vld)
+	if err != nil {
+		return nil, errors.BadInput.Wrap(err, "error in decoding transformation rule")
+	}
+	err = basicRes.GetDal().Create(&rule)
+	if err != nil {
+		if basicRes.GetDal().IsDuplicationError(err) {
+			return nil, errors.BadInput.New("there was a transformation rule with the same name, please choose another name")
+		}
+		return nil, errors.BadInput.Wrap(err, "error on saving TransformationRule")
+	}
+	return &plugin.ApiResourceOutput{Body: rule, Status: http.StatusOK}, nil
+}
+
+// UpdateTransformationRule update transformation rule for Bitbucket
+// @Summary update transformation rule for Bitbucket
+// @Description update transformation rule for Bitbucket
+// @Tags plugins/bitbucket
+// @Accept application/json
+// @Param id path int true "id"
+// @Param transformationRule body models.BitbucketTransformationRule true "transformation rule"
+// @Success 200  {object} models.BitbucketTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/transformation_rules/{id} [PATCH]
+func UpdateTransformationRule(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	transformationRuleId, e := strconv.ParseUint(input.Params["id"], 10, 64)
+	if e != nil {
+		return nil, errors.Default.Wrap(e, "the transformation rule ID should be an integer")
+	}
+	var old models.BitbucketTransformationRule
+	err := basicRes.GetDal().First(&old, dal.Where("id = ?", transformationRuleId))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving TransformationRule")
+	}
+	err = api.DecodeMapStruct(input.Body, &old)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error decoding map into transformationRule")
+	}
+	old.ID = transformationRuleId
+	err = basicRes.GetDal().Update(&old, dal.Where("id = ?", transformationRuleId))
+	if err != nil {
+		if basicRes.GetDal().IsDuplicationError(err) {
+			return nil, errors.BadInput.New("there was a transformation rule with the same name, please choose another name")
+		}
+		return nil, errors.BadInput.Wrap(err, "error on saving TransformationRule")
+	}
+	return &plugin.ApiResourceOutput{Body: old, Status: http.StatusOK}, nil
+}
+
+// GetTransformationRule return one transformation rule
+// @Summary return one transformation rule
+// @Description return one transformation rule
+// @Tags plugins/bitbucket
+// @Param id path int true "id"
+// @Success 200  {object} models.BitbucketTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/transformation_rules/{id} [GET]
+func GetTransformationRule(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	transformationRuleId, err := strconv.ParseUint(input.Params["id"], 10, 64)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "the transformation rule ID should be an integer")
+	}
+	var rule models.BitbucketTransformationRule
+	err = basicRes.GetDal().First(&rule, dal.Where("id = ?", transformationRuleId))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on get TransformationRule")
+	}
+	return &plugin.ApiResourceOutput{Body: rule, Status: http.StatusOK}, nil
+}
+
+// GetTransformationRuleList return all transformation rules
+// @Summary return all transformation rules
+// @Description return all transformation rules
+// @Tags plugins/bitbucket
+// @Param pageSize query int false "page size, default 50"
+// @Param page query int false "page size, default 1"
+// @Success 200  {object} []models.BitbucketTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/bitbucket/transformation_rules [GET]
+func GetTransformationRuleList(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	var rules []models.BitbucketTransformationRule
+	limit, offset := api.GetLimitOffset(input.Query, "pageSize", "page")
+	err := basicRes.GetDal().All(&rules, dal.Limit(limit), dal.Offset(offset))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on get TransformationRule list")
+	}
+	return &plugin.ApiResourceOutput{Body: rules, Status: http.StatusOK}, nil
+}
diff --git a/backend/plugins/bitbucket/bitbucket.go b/backend/plugins/bitbucket/bitbucket.go
index fce007540..fb0f7fdb5 100644
--- a/backend/plugins/bitbucket/bitbucket.go
+++ b/backend/plugins/bitbucket/bitbucket.go
@@ -30,17 +30,17 @@ var PluginEntry impl.Bitbucket //nolint
 func main() {
 	cmd := &cobra.Command{Use: "bitbucket"}
 	connectionId := cmd.Flags().Uint64P("connectionId", "c", 0, "bitbucket connection id")
-	owner := cmd.Flags().StringP("owner", "o", "", "bitbucket owner")
-	repo := cmd.Flags().StringP("repo", "r", "", "bitbucket repo")
+	fullName := cmd.Flags().StringP("fullName", "n", "", "bitbucket id: owner/repo")
+	createdDateAfter := cmd.Flags().StringP("createdDateAfter", "a", "", "collect data that are created after specified time, ie 2006-05-06T07:08:09Z")
 	_ = cmd.MarkFlagRequired("connectionId")
 	_ = cmd.MarkFlagRequired("owner")
 	_ = cmd.MarkFlagRequired("repo")
 
 	cmd.Run = func(cmd *cobra.Command, args []string) {
 		runner.DirectRun(cmd, args, PluginEntry, map[string]interface{}{
-			"connectionId": *connectionId,
-			"owner":        *owner,
-			"repo":         *repo,
+			"connectionId":     *connectionId,
+			"fullName":         *fullName,
+			"createdDateAfter": *createdDateAfter,
 		})
 	}
 	runner.RunCmd(cmd)
diff --git a/backend/plugins/bitbucket/e2e/account_test.go b/backend/plugins/bitbucket/e2e/account_test.go
index 676a805f0..8e7a9f71f 100644
--- a/backend/plugins/bitbucket/e2e/account_test.go
+++ b/backend/plugins/bitbucket/e2e/account_test.go
@@ -31,16 +31,11 @@ func TestAccountDataFlow(t *testing.T) {
 	var plugin impl.Bitbucket
 	dataflowTester := e2ehelper.NewDataFlowTester(t, "bitbucket", plugin)
 
-	bitbucketRepository := &models.BitbucketRepo{
-		BitbucketId: "panjf2000/ants",
-	}
 	taskData := &tasks.BitbucketTaskData{
 		Options: &tasks.BitbucketOptions{
 			ConnectionId: 1,
-			Owner:        "panjf2000",
-			Repo:         "ants",
+			FullName:     "likyh/likyhphp",
 		},
-		Repo: bitbucketRepository,
 	}
 
 	// import raw data table
diff --git a/backend/plugins/bitbucket/e2e/comment_test.go b/backend/plugins/bitbucket/e2e/comment_test.go
index 7b57fe224..526c7360f 100644
--- a/backend/plugins/bitbucket/e2e/comment_test.go
+++ b/backend/plugins/bitbucket/e2e/comment_test.go
@@ -31,21 +31,16 @@ func TestCommentDataFlow(t *testing.T) {
 	var plugin impl.Bitbucket
 	dataflowTester := e2ehelper.NewDataFlowTester(t, "bitbucket", plugin)
 
-	bitbucketRepository := &models.BitbucketRepo{
-		BitbucketId: "panjf2000/ants",
-	}
 	taskData := &tasks.BitbucketTaskData{
 		Options: &tasks.BitbucketOptions{
 			ConnectionId: 1,
-			Owner:        "panjf2000",
-			Repo:         "ants",
-			TransformationRules: models.TransformationRules{
-				IssueStatusTODO:       []string{"new", "open"},
-				IssueStatusINPROGRESS: []string{"on hold"},
-				IssueStatusDONE:       []string{"closed"},
+			FullName:     "likyh/likyhphp",
+			BitbucketTransformationRule: &models.BitbucketTransformationRule{
+				IssueStatusTodo:       "new,open",
+				IssueStatusInProgress: "on hold",
+				IssueStatusDone:       "closed",
 			},
 		},
-		Repo: bitbucketRepository,
 	}
 
 	// import raw data table
diff --git a/backend/plugins/bitbucket/e2e/issue_test.go b/backend/plugins/bitbucket/e2e/issue_test.go
index 981e09f8d..ed8746546 100644
--- a/backend/plugins/bitbucket/e2e/issue_test.go
+++ b/backend/plugins/bitbucket/e2e/issue_test.go
@@ -30,21 +30,16 @@ func TestIssueDataFlow(t *testing.T) {
 	var plugin impl.Bitbucket
 	dataflowTester := e2ehelper.NewDataFlowTester(t, "bitbucket", plugin)
 
-	bitbucketRepository := &models.BitbucketRepo{
-		BitbucketId: "panjf2000/ants",
-	}
 	taskData := &tasks.BitbucketTaskData{
 		Options: &tasks.BitbucketOptions{
 			ConnectionId: 1,
-			Owner:        "panjf2000",
-			Repo:         "ants",
-			TransformationRules: models.TransformationRules{
-				IssueStatusTODO:       []string{"new", "open", "wantfix"},
-				IssueStatusINPROGRESS: []string{"on hold", "resolved"},
-				IssueStatusDONE:       []string{"closed", "invalid"},
+			FullName:     "likyh/likyhphp",
+			BitbucketTransformationRule: &models.BitbucketTransformationRule{
+				IssueStatusTodo:       "new,open,wantfix",
+				IssueStatusInProgress: "on hold,resolved",
+				IssueStatusDone:       "closed,invalid",
 			},
 		},
-		Repo: bitbucketRepository,
 	}
 
 	// import raw data table
@@ -62,6 +57,7 @@ func TestIssueDataFlow(t *testing.T) {
 			"bitbucket_id",
 			"repo_id",
 			"number",
+			"std_state",
 			"state",
 			"title",
 			"body",
diff --git a/backend/plugins/bitbucket/e2e/pipelines_test.go b/backend/plugins/bitbucket/e2e/pipelines_test.go
index 351c60bd8..0be93e409 100644
--- a/backend/plugins/bitbucket/e2e/pipelines_test.go
+++ b/backend/plugins/bitbucket/e2e/pipelines_test.go
@@ -32,16 +32,11 @@ 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",
+			FullName:     "likyh/likyhphp",
 		},
-		Repo: bitbucketRepository,
 	}
 	// import raw data table
 	dataflowTester.ImportCsvIntoRawTable("./raw_tables/_raw_bitbucket_api_pipelines.csv", "_raw_bitbucket_api_pipelines")
diff --git a/backend/plugins/bitbucket/e2e/pr_test.go b/backend/plugins/bitbucket/e2e/pr_test.go
index e0d07f91b..1fb23e0d4 100644
--- a/backend/plugins/bitbucket/e2e/pr_test.go
+++ b/backend/plugins/bitbucket/e2e/pr_test.go
@@ -30,17 +30,11 @@ func TestPrDataFlow(t *testing.T) {
 	var plugin impl.Bitbucket
 	dataflowTester := e2ehelper.NewDataFlowTester(t, "bitbucket", plugin)
 
-	bitbucketRepository := &models.BitbucketRepo{
-		BitbucketId: "panjf2000/ants",
-	}
 	taskData := &tasks.BitbucketTaskData{
 		Options: &tasks.BitbucketOptions{
-			ConnectionId:        1,
-			Owner:               "panjf2000",
-			Repo:                "ants",
-			TransformationRules: models.TransformationRules{},
+			ConnectionId: 1,
+			FullName:     "likyh/likyhphp",
 		},
-		Repo: bitbucketRepository,
 	}
 
 	// import raw data table
diff --git a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issue_comments.csv b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issue_comments.csv
index 44b43ef98..60f3dd24f 100644
--- a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issue_comments.csv
+++ b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issue_comments.csv
@@ -1,29 +1,29 @@
 id,params,data,url,input,created_at
-1,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63860152, ""created_on"": ""2022-07-17T07:16:30.352685+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.grav [...]
-2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63860153, ""created_on"": ""2022-07-17T07:17:01.991973+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""this is a comment test for issue"", ""markup"": ""markdown"", ""html"": ""<p>this is a comment test for issue</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-429 [...]
-3,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63860241, ""created_on"": ""2022-07-17T09:11:42.759529+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.grav [...]
-4,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63961973, ""created_on"": ""2022-08-12T13:51:25.882209+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.grav [...]
-5,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63961995, ""created_on"": ""2022-08-12T13:55:58.125226+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.grav [...]
-6,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963442, ""created_on"": ""2022-08-13T06:17:23.165014+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""gvoeaoivae"", ""markup"": ""markdown"", ""html"": ""<p>gvoeaoivae</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hr [...]
-7,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963443, ""created_on"": ""2022-08-13T06:17:28.683792+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""vboeaorboaneo"", ""markup"": ""markdown"", ""html"": ""<p>vboeaorboaneo</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": [...]
-8,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963444, ""created_on"": ""2022-08-13T06:17:36.685718+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""beoiroierbieiaoboere"", ""markup"": ""markdown"", ""html"": ""<p>beoiroierbieiaoboere</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D"" [...]
-9,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963445, ""created_on"": ""2022-08-13T06:17:42.086856+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""boetboenavnin"", ""markup"": ""markdown"", ""html"": ""<p>boetboenavnin</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": [...]
-10,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963446, ""created_on"": ""2022-08-13T06:17:47.634459+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""bbenoirbnavbeboa"", ""markup"": ""markdown"", ""html"": ""<p>bbenoirbnavbeboa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""av [...]
-11,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963447, ""created_on"": ""2022-08-13T06:17:55.851366+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""lboehoraebow"", ""markup"": ""markdown"", ""html"": ""<p>lboehoraebow</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"":  [...]
-12,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963449, ""created_on"": ""2022-08-13T06:18:01.927576+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""ebeobnoerioaeiow"", ""markup"": ""markdown"", ""html"": ""<p>ebeobnoerioaeiow</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""av [...]
-13,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963450, ""created_on"": ""2022-08-13T06:18:11.146999+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""uttrberoiea"", ""markup"": ""markdown"", ""html"": ""<p>uttrberoiea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
-14,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963451, ""created_on"": ""2022-08-13T06:18:19.376928+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""obfbdfonwea"", ""markup"": ""markdown"", ""html"": ""<p>obfbdfonwea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
-15,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963452, ""created_on"": ""2022-08-13T06:18:26.505035+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""bjeorjboewaonwea"", ""markup"": ""markdown"", ""html"": ""<p>bjeorjboewaonwea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""av [...]
-16,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963453, ""created_on"": ""2022-08-13T06:18:33.109491+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""brtoeorbnqqe"", ""markup"": ""markdown"", ""html"": ""<p>brtoeorbnqqe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"":  [...]
-17,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963454, ""created_on"": ""2022-08-13T06:18:39.542749+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""mmboeroi"", ""markup"": ""markdown"", ""html"": ""<p>mmboeroi</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href" [...]
-18,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963455, ""created_on"": ""2022-08-13T06:18:46.158468+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""lqooaqjoe"", ""markup"": ""markdown"", ""html"": ""<p>lqooaqjoe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
-19,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963456, ""created_on"": ""2022-08-13T06:18:52.858029+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""vajoieiowaeovw"", ""markup"": ""markdown"", ""html"": ""<p>vajoieiowaeovw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar [...]
-20,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963457, ""created_on"": ""2022-08-13T06:18:58.979282+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""ieiojiwea"", ""markup"": ""markdown"", ""html"": ""<p>ieiojiwea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
-21,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963458, ""created_on"": ""2022-08-13T06:19:05.666753+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""beraowheuovhaewa"", ""markup"": ""markdown"", ""html"": ""<p>beraowheuovhaewa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""av [...]
-22,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963459, ""created_on"": ""2022-08-13T06:19:15.956726+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""oeorjeorb"", ""markup"": ""markdown"", ""html"": ""<p>oeorjeorb</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
-23,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963460, ""created_on"": ""2022-08-13T06:19:23.294249+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""iocowevoanewww"", ""markup"": ""markdown"", ""html"": ""<p>iocowevoanewww</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar [...]
-24,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963461, ""created_on"": ""2022-08-13T06:19:31.457341+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""cowoehvowevaobnebiejaiweaw"", ""markup"": ""markdown"", ""html"": ""<p>cowoehvowevaobnebiejaiweaw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-2182 [...]
-25,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963462, ""created_on"": ""2022-08-13T06:19:38.433512+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""vbleklawi"", ""markup"": ""markdown"", ""html"": ""<p>vbleklawi</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
-26,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963463, ""created_on"": ""2022-08-13T06:19:45.160531+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""qowjoveawev"", ""markup"": ""markdown"", ""html"": ""<p>qowjoveawev</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
-27,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963464, ""created_on"": ""2022-08-13T06:19:54.365641+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""ciojboieria"", ""markup"": ""markdown"", ""html"": ""<p>ciojboieria</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
-28,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue_comment"", ""id"": 63963465, ""created_on"": ""2022-08-13T06:20:01.329125+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""cwiwhoervowwwqe"", ""markup"": ""markdown"", ""html"": ""<p>cwiwhoervowwwqe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avat [...]
+1,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63860152, ""created_on"": ""2022-07-17T07:16:30.352685+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.gravatar.com/a [...]
+2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63860153, ""created_on"": ""2022-07-17T07:17:01.991973+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""this is a comment test for issue"", ""markup"": ""markdown"", ""html"": ""<p>this is a comment test for issue</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-218 [...]
+3,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63860241, ""created_on"": ""2022-07-17T09:11:42.759529+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.gravatar.com/a [...]
+4,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63961973, ""created_on"": ""2022-08-12T13:51:25.882209+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.gravatar.com/a [...]
+5,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63961995, ""created_on"": ""2022-08-12T13:55:58.125226+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": null, ""markup"": ""markdown"", ""html"": """"}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://secure.gravatar.com/a [...]
+6,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963442, ""created_on"": ""2022-08-13T06:17:23.165014+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""gvoeaoivae"", ""markup"": ""markdown"", ""html"": ""<p>gvoeaoivae</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""ht [...]
+7,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963443, ""created_on"": ""2022-08-13T06:17:28.683792+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""vboeaorboaneo"", ""markup"": ""markdown"", ""html"": ""<p>vboeaorboaneo</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"" [...]
+8,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963444, ""created_on"": ""2022-08-13T06:17:36.685718+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""beoiroierbieiaoboere"", ""markup"": ""markdown"", ""html"": ""<p>beoiroierbieiaoboere</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avata [...]
+9,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963445, ""created_on"": ""2022-08-13T06:17:42.086856+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""boetboenavnin"", ""markup"": ""markdown"", ""html"": ""<p>boetboenavnin</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"" [...]
+10,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963446, ""created_on"": ""2022-08-13T06:17:47.634459+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""bbenoirbnavbeboa"", ""markup"": ""markdown"", ""html"": ""<p>bbenoirbnavbeboa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
+11,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963447, ""created_on"": ""2022-08-13T06:17:55.851366+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""lboehoraebow"", ""markup"": ""markdown"", ""html"": ""<p>lboehoraebow</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": [...]
+12,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963449, ""created_on"": ""2022-08-13T06:18:01.927576+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""ebeobnoerioaeiow"", ""markup"": ""markdown"", ""html"": ""<p>ebeobnoerioaeiow</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
+13,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963450, ""created_on"": ""2022-08-13T06:18:11.146999+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""uttrberoiea"", ""markup"": ""markdown"", ""html"": ""<p>uttrberoiea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": " [...]
+14,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963451, ""created_on"": ""2022-08-13T06:18:19.376928+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""obfbdfonwea"", ""markup"": ""markdown"", ""html"": ""<p>obfbdfonwea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": " [...]
+15,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963452, ""created_on"": ""2022-08-13T06:18:26.505035+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""bjeorjboewaonwea"", ""markup"": ""markdown"", ""html"": ""<p>bjeorjboewaonwea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
+16,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963453, ""created_on"": ""2022-08-13T06:18:33.109491+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""brtoeorbnqqe"", ""markup"": ""markdown"", ""html"": ""<p>brtoeorbnqqe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": [...]
+17,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963454, ""created_on"": ""2022-08-13T06:18:39.542749+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""mmboeroi"", ""markup"": ""markdown"", ""html"": ""<p>mmboeroi</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https [...]
+18,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963455, ""created_on"": ""2022-08-13T06:18:46.158468+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""lqooaqjoe"", ""markup"": ""markdown"", ""html"": ""<p>lqooaqjoe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""htt [...]
+19,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963456, ""created_on"": ""2022-08-13T06:18:52.858029+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""vajoieiowaeovw"", ""markup"": ""markdown"", ""html"": ""<p>vajoieiowaeovw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
+20,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963457, ""created_on"": ""2022-08-13T06:18:58.979282+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""ieiojiwea"", ""markup"": ""markdown"", ""html"": ""<p>ieiojiwea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""htt [...]
+21,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963458, ""created_on"": ""2022-08-13T06:19:05.666753+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""beraowheuovhaewa"", ""markup"": ""markdown"", ""html"": ""<p>beraowheuovhaewa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
+22,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963459, ""created_on"": ""2022-08-13T06:19:15.956726+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""oeorjeorb"", ""markup"": ""markdown"", ""html"": ""<p>oeorjeorb</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""htt [...]
+23,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963460, ""created_on"": ""2022-08-13T06:19:23.294249+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""iocowevoanewww"", ""markup"": ""markdown"", ""html"": ""<p>iocowevoanewww</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
+24,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963461, ""created_on"": ""2022-08-13T06:19:31.457341+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""cowoehvowevaobnebiejaiweaw"", ""markup"": ""markdown"", ""html"": ""<p>cowoehvowevaobnebiejaiweaw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7 [...]
+25,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963462, ""created_on"": ""2022-08-13T06:19:38.433512+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""vbleklawi"", ""markup"": ""markdown"", ""html"": ""<p>vbleklawi</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""htt [...]
+26,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963463, ""created_on"": ""2022-08-13T06:19:45.160531+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""qowjoveawev"", ""markup"": ""markdown"", ""html"": ""<p>qowjoveawev</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": " [...]
+27,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963464, ""created_on"": ""2022-08-13T06:19:54.365641+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""ciojboieria"", ""markup"": ""markdown"", ""html"": ""<p>ciojboieria</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": " [...]
+28,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue_comment"", ""id"": 63963465, ""created_on"": ""2022-08-13T06:20:01.329125+00:00"", ""updated_on"": null, ""content"": {""type"": ""rendered"", ""raw"": ""cwiwhoervowwwqe"", ""markup"": ""markdown"", ""html"": ""<p>cwiwhoervowwwqe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""h [...]
diff --git a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issues.csv b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issues.csv
index c8cf4d143..25117f541 100644
--- a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issues.csv
+++ b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_issues.csv
@@ -1,31 +1,31 @@
 id,params,data,url,input,created_at
-31,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 27, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-32,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 30, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-33,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 29, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-34,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 28, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-35,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 26, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-36,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 25, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-37,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 6, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-38,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 24, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-39,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 23, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-40,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 22, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-41,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 21, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-42,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 20, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-43,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 19, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-44,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 18, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-45,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 17, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-46,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 16, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-47,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 15, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-48,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 14, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-49,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 13, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-50,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 12, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-51,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 11, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-52,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 10, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e7 [...]
-53,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 9, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-54,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 8, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-55,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 7, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-56,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 5, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-57,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 4, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-58,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 3, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-59,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 2, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
-60,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""issue"", ""id"": 1, ""repository"": {""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e76 [...]
+31,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 27, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+32,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 30, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+33,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 29, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+34,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 28, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+35,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 26, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+36,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 25, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+37,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 6, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+38,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 24, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+39,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 23, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+40,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 22, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+41,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 21, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+42,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 20, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+43,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 19, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+44,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 18, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+45,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 17, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+46,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 16, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+47,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 15, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+48,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 14, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+49,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 13, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+50,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 12, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+51,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 11, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+52,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 10, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f79 [...]
+53,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 9, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+54,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 8, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+55,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 7, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+56,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 5, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+57,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 4, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+58,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 3, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+59,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 2, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
+60,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""issue"", ""id"": 1, ""repository"": {""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}}, ""name"": ""ants"", ""uuid"": ""{e762bff4-f795 [...]
diff --git a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv
index 97156b371..115a5fb6c 100644
--- a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv
+++ b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pipelines.csv
@@ -1,12 +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 [...]
+41,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+42,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+43,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+44,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+45,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+46,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+47,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+48,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+49,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
+50,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""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%7D?ts=default [...]
 
diff --git a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pull_requests.csv b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pull_requests.csv
index 45ad9fe3f..db25491da 100644
--- a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pull_requests.csv
+++ b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pull_requests.csv
@@ -1,58 +1,58 @@
 id,params,data,url,input,created_at
-115,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 40, ""title"": ""Test016"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-116,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 4, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 54, ""title"": ""Bitbuckettest19"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-117,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 4, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 42, ""title"": ""Test019"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-118,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 3, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 31, ""title"": ""Test020"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-119,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 1, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 41, ""title"": ""Test018"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-120,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 44, ""title"": ""Test017"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-121,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 38, ""title"": ""Test029"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-122,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 51, ""title"": ""Bitbuckettest15"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-123,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 57, ""title"": ""Bitbuckettest22"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-124,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 56, ""title"": ""Bitbuckettest25"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-125,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 53, ""title"": ""Bitbuckettest17"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-126,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 55, ""title"": ""Bitbuckettest13"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-127,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 52, ""title"": ""Bitbuckettest16"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-128,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 50, ""title"": ""Bitbuckettest12"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* fea [...]
-129,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 49, ""title"": ""Test08"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""t [...]
-130,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 48, ""title"": ""Test07"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": [...]
-131,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 47, ""title"": ""Test06"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href" [...]
-132,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 29, ""title"": ""Test03"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-42 [...]
-133,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 46, ""title"": ""Test04"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users [...]
-134,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 45, ""title"": ""feat: add test01"", ""description"": """", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": "" [...]
-135,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 43, ""title"": ""Test015"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-136,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 39, ""title"": ""Test030"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-137,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 37, ""title"": ""Test028"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-138,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 36, ""title"": ""Test027"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-139,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 35, ""title"": ""Test025"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-140,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 34, ""title"": ""Test024"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-141,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 33, ""title"": ""Test023"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-142,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 32, ""title"": ""Test022"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add t [...]
-143,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 30, ""title"": ""Test05"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bit [...]
-144,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 24, ""title"": ""Test17"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-145,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 23, ""title"": ""Test20"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-146,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 22, ""title"": ""Test19"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-147,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 21, ""title"": ""Test18"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-148,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 20, ""title"": ""Test16"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-149,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 19, ""title"": ""Test15"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-150,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 18, ""title"": ""Test14"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-151,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 17, ""title"": ""Test13"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-152,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 28, ""title"": ""mergev1"", ""description"": ""merge all v1"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""c0e12b61e44b"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants/commit/c0e12b61e44b""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants/commits/c0e12 [...]
-153,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 27, ""title"": ""4"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add bra [...]
-154,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 26, ""title"": ""2"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add bra [...]
-155,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 25, ""title"": ""1"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add bra [...]
-156,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 15, ""title"": ""Test011"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {"" [...]
-157,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 14, ""title"": ""Test10"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-158,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 13, ""title"": ""Test8"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add [...]
-159,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 12, ""title"": ""Test7"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add [...]
-160,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 11, ""title"": ""Test6"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add [...]
-161,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 10, ""title"": ""Test007"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""h [...]
-162,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 9, ""title"": ""Test006"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.o [...]
-163,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 8, ""title"": ""Test005"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/p [...]
-164,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 7, ""title"": ""Test004"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants/commit/ [...]
-165,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 6, ""title"": ""Test003"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants/commit/983edbf06740""}, ""ht [...]
-166,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 5, ""title"": ""Test002"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants/commit/983edbf06740""}, ""html"": {""href"": ""ht [...]
-167,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 4, ""title"": ""Test001"", ""description"": ""* feat: add git.sh\n* feat: add test001\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants/commit/983edbf06740""}, ""html"": {""href"": ""https://bitbucket.o [...]
-168,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 3, ""title"": ""Test9"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add  [...]
-169,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 2, ""title"": ""feat: add git.sh"", ""description"": ""help to git commit faster"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants/commit/983edbf06740""}, ""html"": {""href"": ""https://bitbucket.org/panjf20 [...]
-170,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 16, ""title"": ""Test12"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: ad [...]
-171,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 1, ""title"": ""feat: add pr test"", ""description"": ""this is a pr test for Bitbucket plugin"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0 [...]
\ No newline at end of file
+115,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 40, ""title"": ""Test016"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+116,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 4, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 54, ""title"": ""Bitbuckettest19"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+117,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 4, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 42, ""title"": ""Test019"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+118,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 3, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 31, ""title"": ""Test020"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+119,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 1, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 41, ""title"": ""Test018"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+120,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 44, ""title"": ""Test017"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+121,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 38, ""title"": ""Test029"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+122,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 51, ""title"": ""Bitbuckettest15"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+123,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 57, ""title"": ""Bitbuckettest22"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+124,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 56, ""title"": ""Bitbuckettest25"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+125,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 53, ""title"": ""Bitbuckettest17"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+126,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 55, ""title"": ""Bitbuckettest13"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+127,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 52, ""title"": ""Bitbuckettest16"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+128,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 50, ""title"": ""Bitbuckettest12"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add tes [...]
+129,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 49, ""title"": ""Test08"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"",  [...]
+130,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 48, ""title"": ""Test07"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"" [...]
+131,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 47, ""title"": ""Test06"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https [...]
+132,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 29, ""title"": ""Test03"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21 [...]
+133,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 46, ""title"": ""Test04"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f [...]
+134,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 45, ""title"": ""feat: add test01"", ""description"": """", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""https://se [...]
+135,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 43, ""title"": ""Test015"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n\n [...]
+136,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 39, ""title"": ""Test030"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+137,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 37, ""title"": ""Test028"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+138,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 36, ""title"": ""Test027"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+139,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 35, ""title"": ""Test025"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+140,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 34, ""title"": ""Test024"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+141,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 33, ""title"": ""Test023"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+142,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 32, ""title"": ""Test022"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n* feat: add test06\n* feat: add test07\n* feat: add test08\n* feat: add test09\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add test014\n* feat: add test015\n*  [...]
+143,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 30, ""title"": ""Test05"", ""description"": ""* feat: add test01\n* feat: add test02\n* feat: add test03\n* feat: add test04\n* feat: add test05\n\n‌"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org [...]
+144,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 24, ""title"": ""Test17"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+145,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 23, ""title"": ""Test20"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+146,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 22, ""title"": ""Test19"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+147,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 21, ""title"": ""Test18"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+148,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 20, ""title"": ""Test16"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+149,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 19, ""title"": ""Test15"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+150,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 18, ""title"": ""Test14"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+151,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 17, ""title"": ""Test13"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+152,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 28, ""title"": ""mergev1"", ""description"": ""merge all v1"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""c0e12b61e44b"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/commit/c0e12b61e44b""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp/commits/c0e12b61e44b""} [...]
+153,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 27, ""title"": ""4"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchname\n* [...]
+154,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 26, ""title"": ""2"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchname\n* [...]
+155,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 25, ""title"": ""1"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchname\n* [...]
+156,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 15, ""title"": ""Test011"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": "" [...]
+157,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 14, ""title"": ""Test10"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+158,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 13, ""title"": ""Test8"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchnam [...]
+159,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 12, ""title"": ""Test7"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchnam [...]
+160,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 11, ""title"": ""Test6"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchnam [...]
+161,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 10, ""title"": ""Test007"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api [...]
+162,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 9, ""title"": ""Test006"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/rep [...]
+163,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 8, ""title"": ""Test005"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyh [...]
+164,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 7, ""title"": ""Test004"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/commit/983edbf067 [...]
+165,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 6, ""title"": ""Test003"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/commit/983edbf06740""}, ""html"": {""h [...]
+166,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 5, ""title"": ""Test002"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/commit/983edbf06740""}, ""html"": {""href"": ""https://bitb [...]
+167,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 4, ""title"": ""Test001"", ""description"": ""* feat: add git.sh\n* feat: add test001\n\n‌"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/commit/983edbf06740""}, ""html"": {""href"": ""https://bitbucket.org/likyh/l [...]
+168,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 3, ""title"": ""Test9"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchname [...]
+169,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 2, ""title"": ""feat: add git.sh"", ""description"": ""help to git commit faster"", ""state"": ""MERGED"", ""merge_commit"": {""type"": ""commit"", ""hash"": ""983edbf06740"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/commit/983edbf06740""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp/co [...]
+170,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 0, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 16, ""title"": ""Test12"", ""description"": ""* feat: add git.sh\n* feat: add test001\n* feat: test002\n* feat: add test003\n* feat: add test004\n* feat: add test005\n* feat: add test006\n* feat: add test007\n* feat: add test008\n* feat: add test009\n* feat: add test010\n* feat: add test011\n* feat: add test012\n* feat: add test013\n* feat: add branchna [...]
+171,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""comment_count"": 2, ""task_count"": 0, ""type"": ""pullrequest"", ""id"": 1, ""title"": ""feat: add pr test"", ""description"": ""this is a pr test for Bitbucket plugin"", ""state"": ""OPEN"", ""merge_commit"": null, ""close_source_branch"": false, ""closed_by"": null, ""author"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D"" [...]
\ No newline at end of file
diff --git a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pullrequest_comments.csv b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pullrequest_comments.csv
index c88e22af5..7f89ce966 100644
--- a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pullrequest_comments.csv
+++ b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_pullrequest_comments.csv
@@ -1,19 +1,19 @@
 id,params,data,url,input,created_at
-1,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120278, ""created_on"": ""2022-08-12T15:40:01.755214+00:00"", ""updated_on"": ""2022-08-12T15:40:01.755276+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""baroboerhba"", ""markup"": ""markdown"", ""html"": ""<p>baroboerhba</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": [...]
-2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120296, ""created_on"": ""2022-08-12T15:40:07.757075+00:00"", ""updated_on"": ""2022-08-12T15:40:07.757120+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""bweiwbeanb"", ""markup"": ""markdown"", ""html"": ""<p>bweiwbeanb</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": { [...]
-3,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120311, ""created_on"": ""2022-08-12T15:40:12.401207+00:00"", ""updated_on"": ""2022-08-12T15:40:12.401282+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""jbirobsore"", ""markup"": ""markdown"", ""html"": ""<p>jbirobsore</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": { [...]
-4,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119960, ""created_on"": ""2022-08-12T15:38:26.895025+00:00"", ""updated_on"": ""2022-08-12T15:38:26.895092+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""regrioteothro"", ""markup"": ""markdown"", ""html"": ""<p>regrioteothro</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avata [...]
-5,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119980, ""created_on"": ""2022-08-12T15:38:31.110923+00:00"", ""updated_on"": ""2022-08-12T15:38:31.110990+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""goeotbioebhotn"", ""markup"": ""markdown"", ""html"": ""<p>goeotbioebhotn</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""ava [...]
-6,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120140, ""created_on"": ""2022-08-12T15:39:27.438543+00:00"", ""updated_on"": ""2022-08-12T15:39:27.438603+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""evbnoera"", ""markup"": ""markdown"", ""html"": ""<p>evbnoera</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hr [...]
-7,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120196, ""created_on"": ""2022-08-12T15:39:41.750361+00:00"", ""updated_on"": ""2022-08-12T15:39:41.750414+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""vobreia"", ""markup"": ""markdown"", ""html"": ""<p>vobreia</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href [...]
-8,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119842, ""created_on"": ""2022-08-12T15:37:51.356670+00:00"", ""updated_on"": ""2022-08-12T15:37:51.356716+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feafoegoaweoir"", ""markup"": ""markdown"", ""html"": ""<p>feafoegoaweoir</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""ava [...]
-9,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119856, ""created_on"": ""2022-08-12T15:37:56.276961+00:00"", ""updated_on"": ""2022-08-12T15:37:56.277012+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""boroehruheorefe"", ""markup"": ""markdown"", ""html"": ""<p>boroehruheorefe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""a [...]
-10,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120091, ""created_on"": ""2022-08-12T15:39:11.612223+00:00"", ""updated_on"": ""2022-08-12T15:39:11.612271+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feawrjoiw"", ""markup"": ""markdown"", ""html"": ""<p>feawrjoiw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
-11,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323120101, ""created_on"": ""2022-08-12T15:39:15.632407+00:00"", ""updated_on"": ""2022-08-12T15:39:15.632461+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""bbeirobeoiahoa"", ""markup"": ""markdown"", ""html"": ""<p>bbeirobeoiahoa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""av [...]
-12,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119784, ""created_on"": ""2022-08-12T15:37:36.308815+00:00"", ""updated_on"": ""2022-08-12T15:37:36.308866+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""fejoafefewa"", ""markup"": ""markdown"", ""html"": ""<p>fejoafefewa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"" [...]
-13,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119790, ""created_on"": ""2022-08-12T15:37:40.769308+00:00"", ""updated_on"": ""2022-08-12T15:37:40.769367+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feafeogioehrw"", ""markup"": ""markdown"", ""html"": ""<p>feafeogioehrw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avat [...]
-14,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119643, ""created_on"": ""2022-08-12T15:36:49.030226+00:00"", ""updated_on"": ""2022-08-12T15:36:49.030284+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""this is test014"", ""markup"": ""markdown"", ""html"": ""<p>this is test014</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, "" [...]
-15,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119662, ""created_on"": ""2022-08-12T15:36:53.393567+00:00"", ""updated_on"": ""2022-08-12T15:36:53.393609+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feafjae"", ""markup"": ""markdown"", ""html"": ""<p>feafjae</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hre [...]
-16,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119897, ""created_on"": ""2022-08-12T15:38:11.179587+00:00"", ""updated_on"": ""2022-08-12T15:38:11.179637+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feajfweofjwvgrea"", ""markup"": ""markdown"", ""html"": ""<p>feajfweofjwvgrea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""},  [...]
-17,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119730, ""created_on"": ""2022-08-12T15:37:16.879523+00:00"", ""updated_on"": ""2022-08-12T15:37:16.879578+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""fejoiafjoej\n\n‌"", ""markup"": ""markdown"", ""html"": ""<p>fejoiafjoej</p>\n<p>‌</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D [...]
-18,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""id"": 323119743, ""created_on"": ""2022-08-12T15:37:21.558078+00:00"", ""updated_on"": ""2022-08-12T15:37:21.558120+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""fjeafojeoirjgoar"", ""markup"": ""markdown"", ""html"": ""<p>fjeafojeoirjgoar</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""},  [...]
+1,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120278, ""created_on"": ""2022-08-12T15:40:01.755214+00:00"", ""updated_on"": ""2022-08-12T15:40:01.755276+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""baroboerhba"", ""markup"": ""markdown"", ""html"": ""<p>baroboerhba</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"" [...]
+2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120296, ""created_on"": ""2022-08-12T15:40:07.757075+00:00"", ""updated_on"": ""2022-08-12T15:40:07.757120+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""bweiwbeanb"", ""markup"": ""markdown"", ""html"": ""<p>bweiwbeanb</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"":  [...]
+3,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120311, ""created_on"": ""2022-08-12T15:40:12.401207+00:00"", ""updated_on"": ""2022-08-12T15:40:12.401282+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""jbirobsore"", ""markup"": ""markdown"", ""html"": ""<p>jbirobsore</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"":  [...]
+4,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119960, ""created_on"": ""2022-08-12T15:38:26.895025+00:00"", ""updated_on"": ""2022-08-12T15:38:26.895092+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""regrioteothro"", ""markup"": ""markdown"", ""html"": ""<p>regrioteothro</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""hr [...]
+5,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119980, ""created_on"": ""2022-08-12T15:38:31.110923+00:00"", ""updated_on"": ""2022-08-12T15:38:31.110990+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""goeotbioebhotn"", ""markup"": ""markdown"", ""html"": ""<p>goeotbioebhotn</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {"" [...]
+6,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120140, ""created_on"": ""2022-08-12T15:39:27.438543+00:00"", ""updated_on"": ""2022-08-12T15:39:27.438603+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""evbnoera"", ""markup"": ""markdown"", ""html"": ""<p>evbnoera</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""ht [...]
+7,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120196, ""created_on"": ""2022-08-12T15:39:41.750361+00:00"", ""updated_on"": ""2022-08-12T15:39:41.750414+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""vobreia"", ""markup"": ""markdown"", ""html"": ""<p>vobreia</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""http [...]
+8,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119842, ""created_on"": ""2022-08-12T15:37:51.356670+00:00"", ""updated_on"": ""2022-08-12T15:37:51.356716+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feafoegoaweoir"", ""markup"": ""markdown"", ""html"": ""<p>feafoegoaweoir</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {"" [...]
+9,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119856, ""created_on"": ""2022-08-12T15:37:56.276961+00:00"", ""updated_on"": ""2022-08-12T15:37:56.277012+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""boroehruheorefe"", ""markup"": ""markdown"", ""html"": ""<p>boroehruheorefe</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": { [...]
+10,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120091, ""created_on"": ""2022-08-12T15:39:11.612223+00:00"", ""updated_on"": ""2022-08-12T15:39:11.612271+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feawrjoiw"", ""markup"": ""markdown"", ""html"": ""<p>feawrjoiw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": " [...]
+11,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323120101, ""created_on"": ""2022-08-12T15:39:15.632407+00:00"", ""updated_on"": ""2022-08-12T15:39:15.632461+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""bbeirobeoiahoa"", ""markup"": ""markdown"", ""html"": ""<p>bbeirobeoiahoa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {" [...]
+12,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119784, ""created_on"": ""2022-08-12T15:37:36.308815+00:00"", ""updated_on"": ""2022-08-12T15:37:36.308866+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""fejoafefewa"", ""markup"": ""markdown"", ""html"": ""<p>fejoafefewa</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href" [...]
+13,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119790, ""created_on"": ""2022-08-12T15:37:40.769308+00:00"", ""updated_on"": ""2022-08-12T15:37:40.769367+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feafeogioehrw"", ""markup"": ""markdown"", ""html"": ""<p>feafeogioehrw</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""h [...]
+14,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119643, ""created_on"": ""2022-08-12T15:36:49.030226+00:00"", ""updated_on"": ""2022-08-12T15:36:49.030284+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""this is test014"", ""markup"": ""markdown"", ""html"": ""<p>this is test014</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"":  [...]
+15,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119662, ""created_on"": ""2022-08-12T15:36:53.393567+00:00"", ""updated_on"": ""2022-08-12T15:36:53.393609+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feafjae"", ""markup"": ""markdown"", ""html"": ""<p>feafjae</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"": {""href"": ""htt [...]
+16,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119897, ""created_on"": ""2022-08-12T15:38:11.179587+00:00"", ""updated_on"": ""2022-08-12T15:38:11.179637+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""feajfweofjwvgrea"", ""markup"": ""markdown"", ""html"": ""<p>feajfweofjwvgrea</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"" [...]
+17,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119730, ""created_on"": ""2022-08-12T15:37:16.879523+00:00"", ""updated_on"": ""2022-08-12T15:37:16.879578+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""fejoiafjoej\n\n‌"", ""markup"": ""markdown"", ""html"": ""<p>fejoiafjoej</p>\n<p>‌</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""ava [...]
+18,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""id"": 323119743, ""created_on"": ""2022-08-12T15:37:21.558078+00:00"", ""updated_on"": ""2022-08-12T15:37:21.558120+00:00"", ""content"": {""type"": ""rendered"", ""raw"": ""fjeafojeoirjgoar"", ""markup"": ""markdown"", ""html"": ""<p>fjeafojeoirjgoar</p>""}, ""user"": {""display_name"": ""teoiaoe"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/users/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D""}, ""avatar"" [...]
diff --git a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_repositories.csv b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_repositories.csv
index 9ff1bbbf1..f618ce3f0 100644
--- a/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_repositories.csv
+++ b/backend/plugins/bitbucket/e2e/raw_tables/_raw_bitbucket_api_repositories.csv
@@ -1,3 +1,3 @@
 id,params,data,url,input,created_at
-2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}","{""type"": ""repository"", ""full_name"": ""panjf2000/ants"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ants""}, ""html"": {""href"": ""https://bitbucket.org/panjf2000/ants""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}, ""pullrequests"": {""href"": ""https://api.bitbucket.org/2.0/repositories/panjf2000/ant [...]
+2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}","{""type"": ""repository"", ""full_name"": ""likyh/likyhphp"", ""links"": {""self"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp""}, ""html"": {""href"": ""https://bitbucket.org/likyh/likyhphp""}, ""avatar"": {""href"": ""https://bytebucket.org/ravatar/%7Be762bff4-f795-49c6-b9f9-6ea334b6d964%7D?ts=default""}, ""pullrequests"": {""href"": ""https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/pullrequ [...]
 
diff --git a/backend/plugins/bitbucket/e2e/repo_test.go b/backend/plugins/bitbucket/e2e/repo_test.go
index ebe444d44..5edfd478c 100644
--- a/backend/plugins/bitbucket/e2e/repo_test.go
+++ b/backend/plugins/bitbucket/e2e/repo_test.go
@@ -18,12 +18,16 @@ limitations under the License.
 package e2e
 
 import (
+	"encoding/json"
+	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/apache/incubator-devlake/core/models/domainlayer/code"
 	"github.com/apache/incubator-devlake/core/models/domainlayer/ticket"
 	"github.com/apache/incubator-devlake/helpers/e2ehelper"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper"
 	"github.com/apache/incubator-devlake/plugins/bitbucket/impl"
 	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
 	"github.com/apache/incubator-devlake/plugins/bitbucket/tasks"
+	"github.com/stretchr/testify/assert"
 	"testing"
 )
 
@@ -31,36 +35,43 @@ func TestRepoDataFlow(t *testing.T) {
 	var plugin impl.Bitbucket
 	dataflowTester := e2ehelper.NewDataFlowTester(t, "bitbucket", plugin)
 
-	bitbucketRepository := &models.BitbucketRepo{
-		BitbucketId: "panjf2000/ants",
-	}
 	taskData := &tasks.BitbucketTaskData{
 		Options: &tasks.BitbucketOptions{
-			ConnectionId:        1,
-			Owner:               "panjf2000",
-			Repo:                "ants",
-			TransformationRules: models.TransformationRules{},
+			ConnectionId: 1,
+			FullName:     "likyh/likyhphp",
 		},
-		Repo: bitbucketRepository,
 	}
 
 	// import raw data table
-	dataflowTester.ImportCsvIntoRawTable("./raw_tables/_raw_bitbucket_api_repositories.csv", "_raw_bitbucket_api_repositories")
+	csvIter := pluginhelper.NewCsvFileIterator("./raw_tables/_raw_bitbucket_api_repositories.csv")
+	defer csvIter.Close()
+	apiRepo := &tasks.BitbucketApiRepo{}
+	// load rows and insert into target table
+	for csvIter.HasNext() {
+		toInsertValues := csvIter.Fetch()
+		data := json.RawMessage(toInsertValues[`data`].(string))
+		err := errors.Convert(json.Unmarshal(data, apiRepo))
+		assert.Nil(t, err)
+		break
+	}
 
 	// verify extraction
 	dataflowTester.FlushTabler(&models.BitbucketRepo{})
-	dataflowTester.Subtask(tasks.ExtractApiRepoMeta, taskData)
+	scope := tasks.ConvertApiRepoToScope(apiRepo, 1)
+	err := dataflowTester.Dal.CreateIfNotExist(scope)
+	assert.Nil(t, err)
 	dataflowTester.VerifyTable(
 		models.BitbucketRepo{},
-		"./snapshot_tables/_tool_bitbucket_repos.csv",
+		"./snapshot_tables/_tool_bitbucket_repos1.csv",
 		e2ehelper.ColumnWithRawData(
 			"connection_id",
 			"bitbucket_id",
 			"name",
 			"html_url",
 			"description",
-			"owner_id",
+			"owner",
 			"language",
+			"clone_url",
 		),
 	)
 
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts.csv
index 6665d105b..8b836696c 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts.csv
@@ -1,2 +1,2 @@
 connection_id,account_id,user_name,account_status,display_name,avatar_url,html_url,uuid,has2_fa_enabled,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,,0,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,171,
+1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,{64135f6a-3978-4297-99b1-21827e0faf0b},0,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,171,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_comments.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_comments.csv
index 72cc74084..58fc956a2 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_comments.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_comments.csv
@@ -1,2 +1,2 @@
 connection_id,account_id,user_name,account_status,display_name,avatar_url,html_url,uuid,has2_fa_enabled,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,,0,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,18,
+1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,{64135f6a-3978-4297-99b1-21827e0faf0b},0,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,18,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_issue.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_issue.csv
index 78ff1746e..75da54a0e 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_issue.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_issue.csv
@@ -1,2 +1,2 @@
 connection_id,account_id,user_name,account_status,display_name,avatar_url,html_url,uuid,has2_fa_enabled,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,,0,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,60,
+1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,{64135f6a-3978-4297-99b1-21827e0faf0b},0,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,60,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_pr.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_pr.csv
index 6665d105b..8b836696c 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_pr.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_accounts_in_pr.csv
@@ -1,2 +1,2 @@
 connection_id,account_id,user_name,account_status,display_name,avatar_url,html_url,uuid,has2_fa_enabled,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,,0,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,171,
+1,62abf394192edb006fa0e8cf,,,teoiaoe,https://secure.gravatar.com/avatar/d37e5c64cd007c5c20654af064c3f410?d=https%3A%2F%2Favatar-management--avatars.us-west-2.prod.public.atl-paas.net%2Finitials%2FT-0.png,https://bitbucket.org/%7B64135f6a-3978-4297-99b1-21827e0faf0b%7D/,{64135f6a-3978-4297-99b1-21827e0faf0b},0,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,171,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issue_comments.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issue_comments.csv
index c75ba8a8a..c51d4da0d 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issue_comments.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issue_comments.csv
@@ -1,29 +1,29 @@
 connection_id,bitbucket_id,issue_id,author_name,author_id,type,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,63860152,1,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,1,
-1,63860153,1,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,2,
-1,63860241,1,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,3,
-1,63961973,6,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,4,
-1,63961995,28,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,5,
-1,63963442,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,6,
-1,63963443,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,7,
-1,63963444,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,8,
-1,63963445,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,9,
-1,63963446,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,10,
-1,63963447,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,11,
-1,63963449,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,12,
-1,63963450,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,13,
-1,63963451,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,14,
-1,63963452,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,15,
-1,63963453,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,16,
-1,63963454,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,17,
-1,63963455,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,18,
-1,63963456,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,19,
-1,63963457,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,20,
-1,63963458,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,21,
-1,63963459,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,22,
-1,63963460,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,23,
-1,63963461,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,24,
-1,63963462,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,25,
-1,63963463,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,26,
-1,63963464,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,27,
-1,63963465,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,28,
+1,63860152,1,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,1,
+1,63860153,1,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,2,
+1,63860241,1,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,3,
+1,63961973,6,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,4,
+1,63961995,28,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,5,
+1,63963442,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,6,
+1,63963443,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,7,
+1,63963444,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,8,
+1,63963445,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,9,
+1,63963446,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,10,
+1,63963447,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,11,
+1,63963449,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,12,
+1,63963450,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,13,
+1,63963451,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,14,
+1,63963452,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,15,
+1,63963453,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,16,
+1,63963454,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,17,
+1,63963455,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,18,
+1,63963456,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,19,
+1,63963457,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,20,
+1,63963458,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,21,
+1,63963459,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,22,
+1,63963460,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,23,
+1,63963461,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,24,
+1,63963462,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,25,
+1,63963463,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,26,
+1,63963464,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,27,
+1,63963465,27,teoiaoe,62abf394192edb006fa0e8cf,issue_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,28,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issues.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issues.csv
index a9610f967..daa143f5f 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issues.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_issues.csv
@@ -1,35 +1,35 @@
-connection_id,bitbucket_id,repo_id,number,state,title,body,priority,type,author_id,author_name,assignee_id,assignee_name,milestone_id,lead_time_minutes,url,closed_at,severity,component,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,1,panjf2000/ants,1,TODO,issue test,bitbucket issues test for devants,major,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/1,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,60,
-1,2,panjf2000/ants,2,TODO,add bitbucket issue,feafejo,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/2,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,59,
-1,3,panjf2000/ants,3,TODO,bitbucket test,"efaegjeoaijefioaegrjoeior,af enfaoiee vioea.,,.wew",major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/3,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,58,
-1,4,panjf2000/ants,4,TODO,issue test001,Bitbucket issue test001,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/4,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,57,
-1,5,panjf2000/ants,5,TODO,issue test002,issue test002,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/5,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,56,
-1,6,panjf2000/ants,6,TODO,issue test003,issue test 003,major,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/6,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,37,
-1,7,panjf2000/ants,7,TODO,issue test004,issue test004,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/7,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,55,
-1,8,panjf2000/ants,8,TODO,issue test005,issue test005,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/8,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,54,
-1,9,panjf2000/ants,9,TODO,issue test006,issue test006,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/9,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,53,
-1,10,panjf2000/ants,10,TODO,issue test007,issue test007,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/10,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,52,
-1,11,panjf2000/ants,11,TODO,issue test008,issue test008,major,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/11,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,51,
-1,12,panjf2000/ants,12,TODO,issue test009,issue test009,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/12,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,50,
-1,13,panjf2000/ants,13,TODO,issue test010,issue test010,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/13,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,49,
-1,14,panjf2000/ants,14,TODO,issue test011,issue test011,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/14,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,48,
-1,15,panjf2000/ants,15,TODO,issue test012,issue test012,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/15,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,47,
-1,16,panjf2000/ants,16,TODO,issue test013,issue test013,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/16,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,46,
-1,17,panjf2000/ants,17,TODO,issue test014,issue test014,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/17,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,45,
-1,18,panjf2000/ants,18,TODO,issue test015,issue test015,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/18,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,44,
-1,19,panjf2000/ants,19,TODO,issue test016,issue test016,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/19,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,43,
-1,20,panjf2000/ants,20,TODO,issue test017,issue test017,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/20,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,42,
-1,21,panjf2000/ants,21,TODO,issue test018,issue test018,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/21,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,41,
-1,22,panjf2000/ants,22,TODO,issue test019,issue test019,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/22,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,40,
-1,23,panjf2000/ants,23,TODO,issue test020,issue test020,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/23,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,39,
-1,24,panjf2000/ants,24,TODO,issue test021,"issue test021 ijeiawgoeive/faveevaeviaevfejaofejfioejaiofe_veavejiovajgiorejoifjrogiorejieafajejaojoejvgioriovioraivjairobnrnoivaiorjbiorjiojaeiorjvioejroivjaoijeriojiaojioeefjafioejfiojeiofawefwefoiwefiwoiefweefwoefuwhufirfrw._  
+connection_id,bitbucket_id,repo_id,number,state,std_state,title,body,priority,type,author_id,author_name,assignee_id,assignee_name,milestone_id,lead_time_minutes,url,closed_at,severity,component,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
+1,1,likyh/likyhphp,1,new,TODO,issue test,bitbucket issues test for devants,major,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/1,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,60,
+1,2,likyh/likyhphp,2,new,TODO,add bitbucket issue,feafejo,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/2,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,59,
+1,3,likyh/likyhphp,3,new,TODO,bitbucket test,"efaegjeoaijefioaegrjoeior,af enfaoiee vioea.,,.wew",major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/3,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,58,
+1,4,likyh/likyhphp,4,new,TODO,issue test001,Bitbucket issue test001,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/4,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,57,
+1,5,likyh/likyhphp,5,new,TODO,issue test002,issue test002,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/5,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,56,
+1,6,likyh/likyhphp,6,new,TODO,issue test003,issue test 003,major,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/6,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,37,
+1,7,likyh/likyhphp,7,new,TODO,issue test004,issue test004,major,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/7,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,55,
+1,8,likyh/likyhphp,8,new,TODO,issue test005,issue test005,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/8,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,54,
+1,9,likyh/likyhphp,9,new,TODO,issue test006,issue test006,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/9,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,53,
+1,10,likyh/likyhphp,10,new,TODO,issue test007,issue test007,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/10,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,52,
+1,11,likyh/likyhphp,11,new,TODO,issue test008,issue test008,major,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/11,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,51,
+1,12,likyh/likyhphp,12,new,TODO,issue test009,issue test009,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/12,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,50,
+1,13,likyh/likyhphp,13,new,TODO,issue test010,issue test010,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/13,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,49,
+1,14,likyh/likyhphp,14,new,TODO,issue test011,issue test011,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/14,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,48,
+1,15,likyh/likyhphp,15,new,TODO,issue test012,issue test012,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/15,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,47,
+1,16,likyh/likyhphp,16,new,TODO,issue test013,issue test013,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/16,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,46,
+1,17,likyh/likyhphp,17,new,TODO,issue test014,issue test014,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/17,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,45,
+1,18,likyh/likyhphp,18,new,TODO,issue test015,issue test015,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/18,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,44,
+1,19,likyh/likyhphp,19,new,TODO,issue test016,issue test016,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/19,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,43,
+1,20,likyh/likyhphp,20,new,TODO,issue test017,issue test017,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/20,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,42,
+1,21,likyh/likyhphp,21,new,TODO,issue test018,issue test018,trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/21,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,41,
+1,22,likyh/likyhphp,22,new,TODO,issue test019,issue test019,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/22,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,40,
+1,23,likyh/likyhphp,23,new,TODO,issue test020,issue test020,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/23,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,39,
+1,24,likyh/likyhphp,24,new,TODO,issue test021,"issue test021 ijeiawgoeive/faveevaeviaevfejaofejfioejaiofe_veavejiovajgiorejoifjrogiorejieafajejaojoejvgioriovioraivjairobnrnoivaiorjbiorjiojaeiorjvioejroivjaoijeriojiaojioeefjafioejfiojeiofawefwefoiwefiwoiefweefwoefuwhufirfrw._
 
 ```
 ``eveaeaeae`jiovjeiojoa oi2j3oij23jovnw3
 ```
 
-‌",trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/24,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,38,
-1,25,panjf2000/ants,25,TODO,issue 023,"issue test023 ejoafehoafe
+‌",trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/24,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,38,
+1,25,likyh/likyhphp,25,new,TODO,issue 023,"issue test023 ejoafehoafe
 
 q
 
@@ -45,11 +45,11 @@ feaofe
 |  |  |  |
 |  |  |  |
 
-‌",critical,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/25,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,36,
-1,26,panjf2000/ants,26,TODO,issue test022,issue test022,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/26,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,35,
-1,27,panjf2000/ants,27,TODO,issue test024,"issue test024v  aejnoafoeiogoiae
+‌",critical,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/25,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,36,
+1,26,likyh/likyhphp,26,new,TODO,issue test022,issue test022,blocker,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/26,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,35,
+1,27,likyh/likyhphp,27,new,TODO,issue test024,"issue test024v  aejnoafoeiogoiae
 
-qwofjeoiwjf",trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/27,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,31,
-1,28,panjf2000/ants,28,TODO,issue test025,issue test025,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/28,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,34,
-1,29,panjf2000/ants,29,TODO,issue test026,issue test026,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/29,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,33,
-1,30,panjf2000/ants,30,TODO,issue test027,issue test027,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/30,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,32,
+qwofjeoiwjf",trivial,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/27,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,31,
+1,28,likyh/likyhphp,28,new,TODO,issue test025,issue test025,minor,issue,62abf394192edb006fa0e8cf,teoiaoe,,,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/28,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,34,
+1,29,likyh/likyhphp,29,new,TODO,issue test026,issue test026,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/29,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,33,
+1,30,likyh/likyhphp,30,new,TODO,issue test027,issue test027,critical,issue,62abf394192edb006fa0e8cf,teoiaoe,62abf394192edb006fa0e8cf,teoiaoe,0,0,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/30,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,32,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv
index 929dde8f9..84d8128b9 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pipelines.csv
@@ -1,11 +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,
+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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_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,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pipelines,44,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_request_comments.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_request_comments.csv
index f71495bef..1cc00f41b 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_request_comments.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_request_comments.csv
@@ -1,19 +1,19 @@
 connection_id,bitbucket_id,pull_request_id,author_name,author_id,type,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,323119643,54,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,14,
-1,323119662,54,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,15,
-1,323119730,57,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,17,
-1,323119743,57,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,18,
-1,323119784,51,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,12,
-1,323119790,51,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,13,
-1,323119842,42,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,8,
-1,323119856,42,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,9,
-1,323119897,54,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,16,
-1,323119960,38,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,4,
-1,323119980,38,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,5,
-1,323120091,44,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,10,
-1,323120101,44,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,11,
-1,323120140,40,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,6,
-1,323120196,41,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,7,
-1,323120278,31,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,1,
-1,323120296,31,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,2,
-1,323120311,31,,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,3,
+1,323119643,54,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,14,
+1,323119662,54,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,15,
+1,323119730,57,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,17,
+1,323119743,57,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,18,
+1,323119784,51,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,12,
+1,323119790,51,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,13,
+1,323119842,42,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,8,
+1,323119856,42,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,9,
+1,323119897,54,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,16,
+1,323119960,38,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,4,
+1,323119980,38,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,5,
+1,323120091,44,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,10,
+1,323120101,44,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,11,
+1,323120140,40,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,6,
+1,323120196,41,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,7,
+1,323120278,31,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,1,
+1,323120296,31,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,2,
+1,323120311,31,teoiaoe,62abf394192edb006fa0e8cf,pullrequest_comment,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,3,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_requests.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_requests.csv
index 0101589e9..a106b552a 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_requests.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_pull_requests.csv
@@ -1,58 +1,58 @@
 connection_id,bitbucket_id,repo_id,number,state,title,closed_at,comment_count,commits,merged_at,body,type,component,merge_commit_sha,head_ref,base_ref,base_commit_sha,head_commit_sha,url,author_name,author_id,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-1,1,panjf2000/ants,0,OPEN,feat: add pr test,,2,0,,,pullrequest,,,test,master,c0e12b61e44b,5e5bccfcc656,https://bitbucket.org/panjf2000/ants/pull-requests/1,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,171,
-1,2,panjf2000/ants,0,MERGED,feat: add git.sh,,0,0,,,pullrequest,,983edbf06740,vowehwo,master,0bd83caaf9c7,d4a6789a3751,https://bitbucket.org/panjf2000/ants/pull-requests/2,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,169,
-1,3,panjf2000/ants,0,MERGED,Test9,,0,0,,,pullrequest,,983edbf06740,test9,master,0bd83caaf9c7,6557208f4aa6,https://bitbucket.org/panjf2000/ants/pull-requests/3,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,168,
-1,4,panjf2000/ants,0,MERGED,Test001,,0,0,,,pullrequest,,983edbf06740,test001,master,0bd83caaf9c7,3bd3b34b2397,https://bitbucket.org/panjf2000/ants/pull-requests/4,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,167,
-1,5,panjf2000/ants,0,MERGED,Test002,,0,0,,,pullrequest,,983edbf06740,test002,master,0bd83caaf9c7,904df6924f1e,https://bitbucket.org/panjf2000/ants/pull-requests/5,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,166,
-1,6,panjf2000/ants,0,MERGED,Test003,,0,0,,,pullrequest,,983edbf06740,test003,master,0bd83caaf9c7,494b97e9d0b2,https://bitbucket.org/panjf2000/ants/pull-requests/6,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,165,
-1,7,panjf2000/ants,0,MERGED,Test004,,0,0,,,pullrequest,,983edbf06740,test004,master,0bd83caaf9c7,db921ce6537c,https://bitbucket.org/panjf2000/ants/pull-requests/7,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,164,
-1,8,panjf2000/ants,0,MERGED,Test005,,0,0,,,pullrequest,,983edbf06740,test005,master,0bd83caaf9c7,8a224cc0eaae,https://bitbucket.org/panjf2000/ants/pull-requests/8,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,163,
-1,9,panjf2000/ants,0,MERGED,Test006,,0,0,,,pullrequest,,983edbf06740,test006,master,0bd83caaf9c7,8a8f44b997a8,https://bitbucket.org/panjf2000/ants/pull-requests/9,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,162,
-1,10,panjf2000/ants,0,MERGED,Test007,,0,0,,,pullrequest,,983edbf06740,test007,master,0bd83caaf9c7,15a1fcb51f0e,https://bitbucket.org/panjf2000/ants/pull-requests/10,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,161,
-1,11,panjf2000/ants,0,MERGED,Test6,,0,0,,,pullrequest,,983edbf06740,test6,master,0bd83caaf9c7,bc9a3ddd47ea,https://bitbucket.org/panjf2000/ants/pull-requests/11,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,160,
-1,12,panjf2000/ants,0,MERGED,Test7,,0,0,,,pullrequest,,983edbf06740,test7,master,0bd83caaf9c7,dd2cd76d7098,https://bitbucket.org/panjf2000/ants/pull-requests/12,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,159,
-1,13,panjf2000/ants,0,MERGED,Test8,,0,0,,,pullrequest,,983edbf06740,test8,master,0bd83caaf9c7,94056215cf46,https://bitbucket.org/panjf2000/ants/pull-requests/13,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,158,
-1,14,panjf2000/ants,0,MERGED,Test10,,0,0,,,pullrequest,,983edbf06740,test10,master,0bd83caaf9c7,bc2cf25142e3,https://bitbucket.org/panjf2000/ants/pull-requests/14,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,157,
-1,15,panjf2000/ants,0,MERGED,Test011,,0,0,,,pullrequest,,983edbf06740,test011,master,0bd83caaf9c7,5db6b1a9e60c,https://bitbucket.org/panjf2000/ants/pull-requests/15,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,156,
-1,16,panjf2000/ants,0,MERGED,Test12,,0,0,,,pullrequest,,983edbf06740,test12,master,0bd83caaf9c7,d1321cb5e4fb,https://bitbucket.org/panjf2000/ants/pull-requests/16,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,170,
-1,17,panjf2000/ants,0,MERGED,Test13,,0,0,,,pullrequest,,c0e12b61e44b,test13,master,983edbf06740,3d6d96659c4d,https://bitbucket.org/panjf2000/ants/pull-requests/17,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,151,
-1,18,panjf2000/ants,0,MERGED,Test14,,0,0,,,pullrequest,,c0e12b61e44b,test14,master,983edbf06740,8d4896eef3ed,https://bitbucket.org/panjf2000/ants/pull-requests/18,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,150,
-1,19,panjf2000/ants,0,MERGED,Test15,,0,0,,,pullrequest,,c0e12b61e44b,test15,master,983edbf06740,daa1cfa7ec0c,https://bitbucket.org/panjf2000/ants/pull-requests/19,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,149,
-1,20,panjf2000/ants,0,MERGED,Test16,,0,0,,,pullrequest,,c0e12b61e44b,test16,master,983edbf06740,11f6c86580e2,https://bitbucket.org/panjf2000/ants/pull-requests/20,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,148,
-1,21,panjf2000/ants,0,MERGED,Test18,,0,0,,,pullrequest,,c0e12b61e44b,test18,master,983edbf06740,0b958bbfed04,https://bitbucket.org/panjf2000/ants/pull-requests/21,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,147,
-1,22,panjf2000/ants,0,MERGED,Test19,,0,0,,,pullrequest,,c0e12b61e44b,test19,master,983edbf06740,d1bb3677810e,https://bitbucket.org/panjf2000/ants/pull-requests/22,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,146,
-1,23,panjf2000/ants,0,MERGED,Test20,,0,0,,,pullrequest,,c0e12b61e44b,test20,master,983edbf06740,9d16a2368acd,https://bitbucket.org/panjf2000/ants/pull-requests/23,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,145,
-1,24,panjf2000/ants,0,MERGED,Test17,,0,0,,,pullrequest,,c0e12b61e44b,test17,master,983edbf06740,2bfd29a17665,https://bitbucket.org/panjf2000/ants/pull-requests/24,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,144,
-1,25,panjf2000/ants,0,MERGED,1,,0,0,,,pullrequest,,983edbf06740,1,master,0bd83caaf9c7,592612c47ff5,https://bitbucket.org/panjf2000/ants/pull-requests/25,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,155,
-1,26,panjf2000/ants,0,MERGED,2,,0,0,,,pullrequest,,983edbf06740,2,master,0bd83caaf9c7,05b97cf92b4b,https://bitbucket.org/panjf2000/ants/pull-requests/26,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,154,
-1,27,panjf2000/ants,0,MERGED,4,,0,0,,,pullrequest,,983edbf06740,4,master,0bd83caaf9c7,761fdb9244ef,https://bitbucket.org/panjf2000/ants/pull-requests/27,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,153,
-1,28,panjf2000/ants,0,MERGED,mergev1,,0,0,,,pullrequest,,c0e12b61e44b,mergev1,master,983edbf06740,581a89006076,https://bitbucket.org/panjf2000/ants/pull-requests/28,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,152,
-1,29,panjf2000/ants,0,OPEN,Test03,,0,0,,,pullrequest,,,test03,master,c0e12b61e44b,a3768d182d18,https://bitbucket.org/panjf2000/ants/pull-requests/29,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,132,
-1,30,panjf2000/ants,0,OPEN,Test05,,0,0,,,pullrequest,,,test05,master,c0e12b61e44b,91e7a1db9ae0,https://bitbucket.org/panjf2000/ants/pull-requests/30,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,143,
-1,31,panjf2000/ants,0,OPEN,Test020,,3,0,,,pullrequest,,,test020,master,c0e12b61e44b,0342bf5351f8,https://bitbucket.org/panjf2000/ants/pull-requests/31,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,118,
-1,32,panjf2000/ants,0,OPEN,Test022,,0,0,,,pullrequest,,,test022,master,c0e12b61e44b,d72c46a722d7,https://bitbucket.org/panjf2000/ants/pull-requests/32,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,142,
-1,33,panjf2000/ants,0,OPEN,Test023,,0,0,,,pullrequest,,,test023,master,c0e12b61e44b,0a8011c8c0ac,https://bitbucket.org/panjf2000/ants/pull-requests/33,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,141,
-1,34,panjf2000/ants,0,OPEN,Test024,,0,0,,,pullrequest,,,test024,master,c0e12b61e44b,bd7e81a581f1,https://bitbucket.org/panjf2000/ants/pull-requests/34,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,140,
-1,35,panjf2000/ants,0,OPEN,Test025,,0,0,,,pullrequest,,,test025,master,c0e12b61e44b,a15cb5876414,https://bitbucket.org/panjf2000/ants/pull-requests/35,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,139,
-1,36,panjf2000/ants,0,OPEN,Test027,,0,0,,,pullrequest,,,test027,master,c0e12b61e44b,200665f463ff,https://bitbucket.org/panjf2000/ants/pull-requests/36,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,138,
-1,37,panjf2000/ants,0,OPEN,Test028,,0,0,,,pullrequest,,,test028,master,c0e12b61e44b,8a9052ac97f4,https://bitbucket.org/panjf2000/ants/pull-requests/37,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,137,
-1,38,panjf2000/ants,0,OPEN,Test029,,2,0,,,pullrequest,,,test029,master,c0e12b61e44b,74bddc0541c6,https://bitbucket.org/panjf2000/ants/pull-requests/38,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,121,
-1,39,panjf2000/ants,0,OPEN,Test030,,0,0,,,pullrequest,,,test030,master,c0e12b61e44b,f332a9855de1,https://bitbucket.org/panjf2000/ants/pull-requests/39,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,136,
-1,40,panjf2000/ants,0,OPEN,Test016,,2,0,,,pullrequest,,,test016,master,c0e12b61e44b,a000a2a9fc11,https://bitbucket.org/panjf2000/ants/pull-requests/40,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,115,
-1,41,panjf2000/ants,0,OPEN,Test018,,1,0,,,pullrequest,,,test018,master,c0e12b61e44b,561e4e3ca4cc,https://bitbucket.org/panjf2000/ants/pull-requests/41,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,119,
-1,42,panjf2000/ants,0,OPEN,Test019,,4,0,,,pullrequest,,,test019,master,c0e12b61e44b,8597a99eebb3,https://bitbucket.org/panjf2000/ants/pull-requests/42,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,117,
-1,43,panjf2000/ants,0,OPEN,Test015,,0,0,,,pullrequest,,,test015,master,c0e12b61e44b,5af2c361263a,https://bitbucket.org/panjf2000/ants/pull-requests/43,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,135,
-1,44,panjf2000/ants,0,OPEN,Test017,,2,0,,,pullrequest,,,test017,master,c0e12b61e44b,ee87b0380472,https://bitbucket.org/panjf2000/ants/pull-requests/44,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,120,
-1,45,panjf2000/ants,0,OPEN,feat: add test01,,0,0,,,pullrequest,,,test01,master,c0e12b61e44b,63e6acbb60a9,https://bitbucket.org/panjf2000/ants/pull-requests/45,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,134,
-1,46,panjf2000/ants,0,OPEN,Test04,,0,0,,,pullrequest,,,test04,master,c0e12b61e44b,33419e166d75,https://bitbucket.org/panjf2000/ants/pull-requests/46,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,133,
-1,47,panjf2000/ants,0,OPEN,Test06,,0,0,,,pullrequest,,,test06,master,c0e12b61e44b,f8f9ef467e3a,https://bitbucket.org/panjf2000/ants/pull-requests/47,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,131,
-1,48,panjf2000/ants,0,OPEN,Test07,,0,0,,,pullrequest,,,test07,master,c0e12b61e44b,c28f34dde568,https://bitbucket.org/panjf2000/ants/pull-requests/48,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,130,
-1,49,panjf2000/ants,0,OPEN,Test08,,0,0,,,pullrequest,,,test08,master,c0e12b61e44b,6b59c8607581,https://bitbucket.org/panjf2000/ants/pull-requests/49,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,129,
-1,50,panjf2000/ants,0,OPEN,Bitbuckettest12,,0,0,,,pullrequest,,,bitbuckettest12,master,c0e12b61e44b,deff6927812f,https://bitbucket.org/panjf2000/ants/pull-requests/50,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,128,
-1,51,panjf2000/ants,0,OPEN,Bitbuckettest15,,2,0,,,pullrequest,,,bitbuckettest15,master,c0e12b61e44b,5c2c9af70da2,https://bitbucket.org/panjf2000/ants/pull-requests/51,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,122,
-1,52,panjf2000/ants,0,OPEN,Bitbuckettest16,,0,0,,,pullrequest,,,bitbuckettest16,master,c0e12b61e44b,7c91d40ffaa3,https://bitbucket.org/panjf2000/ants/pull-requests/52,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,127,
-1,53,panjf2000/ants,0,OPEN,Bitbuckettest17,,0,0,,,pullrequest,,,bitbuckettest17,master,c0e12b61e44b,765ab2594451,https://bitbucket.org/panjf2000/ants/pull-requests/53,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,125,
-1,54,panjf2000/ants,0,OPEN,Bitbuckettest19,,4,0,,,pullrequest,,,bitbuckettest19,master,c0e12b61e44b,2493b9212fd4,https://bitbucket.org/panjf2000/ants/pull-requests/54,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,116,
-1,55,panjf2000/ants,0,OPEN,Bitbuckettest13,,0,0,,,pullrequest,,,bitbuckettest13,master,c0e12b61e44b,9dd8c98115ca,https://bitbucket.org/panjf2000/ants/pull-requests/55,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,126,
-1,56,panjf2000/ants,0,OPEN,Bitbuckettest25,,0,0,,,pullrequest,,,bitbuckettest25,master,c0e12b61e44b,d69faf8c90c4,https://bitbucket.org/panjf2000/ants/pull-requests/56,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,124,
-1,57,panjf2000/ants,0,OPEN,Bitbuckettest22,,2,0,,,pullrequest,,,bitbuckettest22,master,c0e12b61e44b,2c23b3babf27,https://bitbucket.org/panjf2000/ants/pull-requests/57,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,123,
+1,1,likyh/likyhphp,1,OPEN,feat: add pr test,,2,0,,,pullrequest,,,test,master,c0e12b61e44b,5e5bccfcc656,https://bitbucket.org/likyh/likyhphp/pull-requests/1,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,171,
+1,2,likyh/likyhphp,2,MERGED,feat: add git.sh,,0,0,,,pullrequest,,983edbf06740,vowehwo,master,0bd83caaf9c7,d4a6789a3751,https://bitbucket.org/likyh/likyhphp/pull-requests/2,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,169,
+1,3,likyh/likyhphp,3,MERGED,Test9,,0,0,,,pullrequest,,983edbf06740,test9,master,0bd83caaf9c7,6557208f4aa6,https://bitbucket.org/likyh/likyhphp/pull-requests/3,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,168,
+1,4,likyh/likyhphp,4,MERGED,Test001,,0,0,,,pullrequest,,983edbf06740,test001,master,0bd83caaf9c7,3bd3b34b2397,https://bitbucket.org/likyh/likyhphp/pull-requests/4,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,167,
+1,5,likyh/likyhphp,5,MERGED,Test002,,0,0,,,pullrequest,,983edbf06740,test002,master,0bd83caaf9c7,904df6924f1e,https://bitbucket.org/likyh/likyhphp/pull-requests/5,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,166,
+1,6,likyh/likyhphp,6,MERGED,Test003,,0,0,,,pullrequest,,983edbf06740,test003,master,0bd83caaf9c7,494b97e9d0b2,https://bitbucket.org/likyh/likyhphp/pull-requests/6,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,165,
+1,7,likyh/likyhphp,7,MERGED,Test004,,0,0,,,pullrequest,,983edbf06740,test004,master,0bd83caaf9c7,db921ce6537c,https://bitbucket.org/likyh/likyhphp/pull-requests/7,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,164,
+1,8,likyh/likyhphp,8,MERGED,Test005,,0,0,,,pullrequest,,983edbf06740,test005,master,0bd83caaf9c7,8a224cc0eaae,https://bitbucket.org/likyh/likyhphp/pull-requests/8,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,163,
+1,9,likyh/likyhphp,9,MERGED,Test006,,0,0,,,pullrequest,,983edbf06740,test006,master,0bd83caaf9c7,8a8f44b997a8,https://bitbucket.org/likyh/likyhphp/pull-requests/9,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,162,
+1,10,likyh/likyhphp,10,MERGED,Test007,,0,0,,,pullrequest,,983edbf06740,test007,master,0bd83caaf9c7,15a1fcb51f0e,https://bitbucket.org/likyh/likyhphp/pull-requests/10,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,161,
+1,11,likyh/likyhphp,11,MERGED,Test6,,0,0,,,pullrequest,,983edbf06740,test6,master,0bd83caaf9c7,bc9a3ddd47ea,https://bitbucket.org/likyh/likyhphp/pull-requests/11,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,160,
+1,12,likyh/likyhphp,12,MERGED,Test7,,0,0,,,pullrequest,,983edbf06740,test7,master,0bd83caaf9c7,dd2cd76d7098,https://bitbucket.org/likyh/likyhphp/pull-requests/12,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,159,
+1,13,likyh/likyhphp,13,MERGED,Test8,,0,0,,,pullrequest,,983edbf06740,test8,master,0bd83caaf9c7,94056215cf46,https://bitbucket.org/likyh/likyhphp/pull-requests/13,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,158,
+1,14,likyh/likyhphp,14,MERGED,Test10,,0,0,,,pullrequest,,983edbf06740,test10,master,0bd83caaf9c7,bc2cf25142e3,https://bitbucket.org/likyh/likyhphp/pull-requests/14,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,157,
+1,15,likyh/likyhphp,15,MERGED,Test011,,0,0,,,pullrequest,,983edbf06740,test011,master,0bd83caaf9c7,5db6b1a9e60c,https://bitbucket.org/likyh/likyhphp/pull-requests/15,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,156,
+1,16,likyh/likyhphp,16,MERGED,Test12,,0,0,,,pullrequest,,983edbf06740,test12,master,0bd83caaf9c7,d1321cb5e4fb,https://bitbucket.org/likyh/likyhphp/pull-requests/16,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,170,
+1,17,likyh/likyhphp,17,MERGED,Test13,,0,0,,,pullrequest,,c0e12b61e44b,test13,master,983edbf06740,3d6d96659c4d,https://bitbucket.org/likyh/likyhphp/pull-requests/17,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,151,
+1,18,likyh/likyhphp,18,MERGED,Test14,,0,0,,,pullrequest,,c0e12b61e44b,test14,master,983edbf06740,8d4896eef3ed,https://bitbucket.org/likyh/likyhphp/pull-requests/18,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,150,
+1,19,likyh/likyhphp,19,MERGED,Test15,,0,0,,,pullrequest,,c0e12b61e44b,test15,master,983edbf06740,daa1cfa7ec0c,https://bitbucket.org/likyh/likyhphp/pull-requests/19,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,149,
+1,20,likyh/likyhphp,20,MERGED,Test16,,0,0,,,pullrequest,,c0e12b61e44b,test16,master,983edbf06740,11f6c86580e2,https://bitbucket.org/likyh/likyhphp/pull-requests/20,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,148,
+1,21,likyh/likyhphp,21,MERGED,Test18,,0,0,,,pullrequest,,c0e12b61e44b,test18,master,983edbf06740,0b958bbfed04,https://bitbucket.org/likyh/likyhphp/pull-requests/21,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,147,
+1,22,likyh/likyhphp,22,MERGED,Test19,,0,0,,,pullrequest,,c0e12b61e44b,test19,master,983edbf06740,d1bb3677810e,https://bitbucket.org/likyh/likyhphp/pull-requests/22,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,146,
+1,23,likyh/likyhphp,23,MERGED,Test20,,0,0,,,pullrequest,,c0e12b61e44b,test20,master,983edbf06740,9d16a2368acd,https://bitbucket.org/likyh/likyhphp/pull-requests/23,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,145,
+1,24,likyh/likyhphp,24,MERGED,Test17,,0,0,,,pullrequest,,c0e12b61e44b,test17,master,983edbf06740,2bfd29a17665,https://bitbucket.org/likyh/likyhphp/pull-requests/24,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,144,
+1,25,likyh/likyhphp,25,MERGED,1,,0,0,,,pullrequest,,983edbf06740,1,master,0bd83caaf9c7,592612c47ff5,https://bitbucket.org/likyh/likyhphp/pull-requests/25,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,155,
+1,26,likyh/likyhphp,26,MERGED,2,,0,0,,,pullrequest,,983edbf06740,2,master,0bd83caaf9c7,05b97cf92b4b,https://bitbucket.org/likyh/likyhphp/pull-requests/26,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,154,
+1,27,likyh/likyhphp,27,MERGED,4,,0,0,,,pullrequest,,983edbf06740,4,master,0bd83caaf9c7,761fdb9244ef,https://bitbucket.org/likyh/likyhphp/pull-requests/27,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,153,
+1,28,likyh/likyhphp,28,MERGED,mergev1,,0,0,,,pullrequest,,c0e12b61e44b,mergev1,master,983edbf06740,581a89006076,https://bitbucket.org/likyh/likyhphp/pull-requests/28,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,152,
+1,29,likyh/likyhphp,29,OPEN,Test03,,0,0,,,pullrequest,,,test03,master,c0e12b61e44b,a3768d182d18,https://bitbucket.org/likyh/likyhphp/pull-requests/29,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,132,
+1,30,likyh/likyhphp,30,OPEN,Test05,,0,0,,,pullrequest,,,test05,master,c0e12b61e44b,91e7a1db9ae0,https://bitbucket.org/likyh/likyhphp/pull-requests/30,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,143,
+1,31,likyh/likyhphp,31,OPEN,Test020,,3,0,,,pullrequest,,,test020,master,c0e12b61e44b,0342bf5351f8,https://bitbucket.org/likyh/likyhphp/pull-requests/31,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,118,
+1,32,likyh/likyhphp,32,OPEN,Test022,,0,0,,,pullrequest,,,test022,master,c0e12b61e44b,d72c46a722d7,https://bitbucket.org/likyh/likyhphp/pull-requests/32,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,142,
+1,33,likyh/likyhphp,33,OPEN,Test023,,0,0,,,pullrequest,,,test023,master,c0e12b61e44b,0a8011c8c0ac,https://bitbucket.org/likyh/likyhphp/pull-requests/33,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,141,
+1,34,likyh/likyhphp,34,OPEN,Test024,,0,0,,,pullrequest,,,test024,master,c0e12b61e44b,bd7e81a581f1,https://bitbucket.org/likyh/likyhphp/pull-requests/34,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,140,
+1,35,likyh/likyhphp,35,OPEN,Test025,,0,0,,,pullrequest,,,test025,master,c0e12b61e44b,a15cb5876414,https://bitbucket.org/likyh/likyhphp/pull-requests/35,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,139,
+1,36,likyh/likyhphp,36,OPEN,Test027,,0,0,,,pullrequest,,,test027,master,c0e12b61e44b,200665f463ff,https://bitbucket.org/likyh/likyhphp/pull-requests/36,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,138,
+1,37,likyh/likyhphp,37,OPEN,Test028,,0,0,,,pullrequest,,,test028,master,c0e12b61e44b,8a9052ac97f4,https://bitbucket.org/likyh/likyhphp/pull-requests/37,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,137,
+1,38,likyh/likyhphp,38,OPEN,Test029,,2,0,,,pullrequest,,,test029,master,c0e12b61e44b,74bddc0541c6,https://bitbucket.org/likyh/likyhphp/pull-requests/38,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,121,
+1,39,likyh/likyhphp,39,OPEN,Test030,,0,0,,,pullrequest,,,test030,master,c0e12b61e44b,f332a9855de1,https://bitbucket.org/likyh/likyhphp/pull-requests/39,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,136,
+1,40,likyh/likyhphp,40,OPEN,Test016,,2,0,,,pullrequest,,,test016,master,c0e12b61e44b,a000a2a9fc11,https://bitbucket.org/likyh/likyhphp/pull-requests/40,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,115,
+1,41,likyh/likyhphp,41,OPEN,Test018,,1,0,,,pullrequest,,,test018,master,c0e12b61e44b,561e4e3ca4cc,https://bitbucket.org/likyh/likyhphp/pull-requests/41,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,119,
+1,42,likyh/likyhphp,42,OPEN,Test019,,4,0,,,pullrequest,,,test019,master,c0e12b61e44b,8597a99eebb3,https://bitbucket.org/likyh/likyhphp/pull-requests/42,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,117,
+1,43,likyh/likyhphp,43,OPEN,Test015,,0,0,,,pullrequest,,,test015,master,c0e12b61e44b,5af2c361263a,https://bitbucket.org/likyh/likyhphp/pull-requests/43,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,135,
+1,44,likyh/likyhphp,44,OPEN,Test017,,2,0,,,pullrequest,,,test017,master,c0e12b61e44b,ee87b0380472,https://bitbucket.org/likyh/likyhphp/pull-requests/44,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,120,
+1,45,likyh/likyhphp,45,OPEN,feat: add test01,,0,0,,,pullrequest,,,test01,master,c0e12b61e44b,63e6acbb60a9,https://bitbucket.org/likyh/likyhphp/pull-requests/45,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,134,
+1,46,likyh/likyhphp,46,OPEN,Test04,,0,0,,,pullrequest,,,test04,master,c0e12b61e44b,33419e166d75,https://bitbucket.org/likyh/likyhphp/pull-requests/46,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,133,
+1,47,likyh/likyhphp,47,OPEN,Test06,,0,0,,,pullrequest,,,test06,master,c0e12b61e44b,f8f9ef467e3a,https://bitbucket.org/likyh/likyhphp/pull-requests/47,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,131,
+1,48,likyh/likyhphp,48,OPEN,Test07,,0,0,,,pullrequest,,,test07,master,c0e12b61e44b,c28f34dde568,https://bitbucket.org/likyh/likyhphp/pull-requests/48,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,130,
+1,49,likyh/likyhphp,49,OPEN,Test08,,0,0,,,pullrequest,,,test08,master,c0e12b61e44b,6b59c8607581,https://bitbucket.org/likyh/likyhphp/pull-requests/49,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,129,
+1,50,likyh/likyhphp,50,OPEN,Bitbuckettest12,,0,0,,,pullrequest,,,bitbuckettest12,master,c0e12b61e44b,deff6927812f,https://bitbucket.org/likyh/likyhphp/pull-requests/50,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,128,
+1,51,likyh/likyhphp,51,OPEN,Bitbuckettest15,,2,0,,,pullrequest,,,bitbuckettest15,master,c0e12b61e44b,5c2c9af70da2,https://bitbucket.org/likyh/likyhphp/pull-requests/51,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,122,
+1,52,likyh/likyhphp,52,OPEN,Bitbuckettest16,,0,0,,,pullrequest,,,bitbuckettest16,master,c0e12b61e44b,7c91d40ffaa3,https://bitbucket.org/likyh/likyhphp/pull-requests/52,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,127,
+1,53,likyh/likyhphp,53,OPEN,Bitbuckettest17,,0,0,,,pullrequest,,,bitbuckettest17,master,c0e12b61e44b,765ab2594451,https://bitbucket.org/likyh/likyhphp/pull-requests/53,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,125,
+1,54,likyh/likyhphp,54,OPEN,Bitbuckettest19,,4,0,,,pullrequest,,,bitbuckettest19,master,c0e12b61e44b,2493b9212fd4,https://bitbucket.org/likyh/likyhphp/pull-requests/54,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,116,
+1,55,likyh/likyhphp,55,OPEN,Bitbuckettest13,,0,0,,,pullrequest,,,bitbuckettest13,master,c0e12b61e44b,9dd8c98115ca,https://bitbucket.org/likyh/likyhphp/pull-requests/55,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,126,
+1,56,likyh/likyhphp,56,OPEN,Bitbuckettest25,,0,0,,,pullrequest,,,bitbuckettest25,master,c0e12b61e44b,d69faf8c90c4,https://bitbucket.org/likyh/likyhphp/pull-requests/56,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,124,
+1,57,likyh/likyhphp,57,OPEN,Bitbuckettest22,,2,0,,,pullrequest,,,bitbuckettest22,master,c0e12b61e44b,2c23b3babf27,https://bitbucket.org/likyh/likyhphp/pull-requests/57,teoiaoe,62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,123,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv
index 50d7b9bb9..dd4eb7c53 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/_tool_bitbucket_repos.csv
+++ b/backend/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,
\ No newline at end of file
+connection_id,bitbucket_id,name,html_url,description,owner,language,clone_url,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
+1,likyh/likyhphp,likyhphp,https://bitbucket.org/likyh/likyhphp,,,,https://likyh@bitbucket.org/likyh/likyhphp.git,,,0,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/board_issues.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/board_issues.csv
index 88e7d8823..2d65162cc 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/board_issues.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/board_issues.csv
@@ -1,31 +1,31 @@
 board_id,issue_id,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:1,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,60,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:10,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,52,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:11,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,51,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:12,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,50,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:13,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,49,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:14,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,48,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:15,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,47,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:16,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,46,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:17,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,45,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:18,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,44,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:19,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,43,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,59,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:20,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,42,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:21,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,41,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:22,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,40,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:23,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,39,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:24,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,38,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:25,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,36,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:26,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,35,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:27,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,31,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:28,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,34,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:29,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,33,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:3,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,58,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:30,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,32,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:4,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,57,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:5,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,56,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:6,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,37,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:7,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,55,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:8,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,54,
-bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketIssue:1:9,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,53,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:1,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,60,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:10,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,52,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:11,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,51,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:12,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,50,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:13,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,49,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:14,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,48,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:15,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,47,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:16,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,46,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:17,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,45,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:18,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,44,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:19,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,43,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,59,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:20,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,42,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:21,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,41,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:22,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,40,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:23,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,39,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:24,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,38,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:25,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,36,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:26,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,35,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,31,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:28,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,34,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:29,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,33,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:3,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,58,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:30,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,32,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:4,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,57,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:5,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,56,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:6,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,37,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:7,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,55,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:8,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,54,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketIssue:1:likyh/likyhphp:9,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,53,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/boards.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/boards.csv
index 72d623f44..cf77c6767 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/boards.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/boards.csv
@@ -1,2 +1,2 @@
 id,name,description,url,created_date,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketRepo:1:panjf2000/ants,panjf2000/ants,,https://bitbucket.org/panjf2000/ants/issues,2022-06-17T03:27:18.865+00:00,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_repositories,2,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,likyhphp,,https://bitbucket.org/likyh/likyhphp/issues,2022-06-17T03:27:18.865+00:00,,,0,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/issue_comments.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/issue_comments.csv
index eb12c307e..56d8aac6f 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/issue_comments.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/issue_comments.csv
@@ -1,29 +1,29 @@
 id,issue_id,body,account_id,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketIssue:1:63860152,bitbucket:BitbucketIssue:1:1,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,1,
-bitbucket:BitbucketIssue:1:63860153,bitbucket:BitbucketIssue:1:1,this is a comment test for issue,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,2,
-bitbucket:BitbucketIssue:1:63860241,bitbucket:BitbucketIssue:1:1,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,3,
-bitbucket:BitbucketIssue:1:63961973,bitbucket:BitbucketIssue:1:6,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,4,
-bitbucket:BitbucketIssue:1:63961995,bitbucket:BitbucketIssue:1:28,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,5,
-bitbucket:BitbucketIssue:1:63963442,bitbucket:BitbucketIssue:1:27,gvoeaoivae,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,6,
-bitbucket:BitbucketIssue:1:63963443,bitbucket:BitbucketIssue:1:27,vboeaorboaneo,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,7,
-bitbucket:BitbucketIssue:1:63963444,bitbucket:BitbucketIssue:1:27,beoiroierbieiaoboere,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,8,
-bitbucket:BitbucketIssue:1:63963445,bitbucket:BitbucketIssue:1:27,boetboenavnin,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,9,
-bitbucket:BitbucketIssue:1:63963446,bitbucket:BitbucketIssue:1:27,bbenoirbnavbeboa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,10,
-bitbucket:BitbucketIssue:1:63963447,bitbucket:BitbucketIssue:1:27,lboehoraebow,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,11,
-bitbucket:BitbucketIssue:1:63963449,bitbucket:BitbucketIssue:1:27,ebeobnoerioaeiow,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,12,
-bitbucket:BitbucketIssue:1:63963450,bitbucket:BitbucketIssue:1:27,uttrberoiea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,13,
-bitbucket:BitbucketIssue:1:63963451,bitbucket:BitbucketIssue:1:27,obfbdfonwea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,14,
-bitbucket:BitbucketIssue:1:63963452,bitbucket:BitbucketIssue:1:27,bjeorjboewaonwea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,15,
-bitbucket:BitbucketIssue:1:63963453,bitbucket:BitbucketIssue:1:27,brtoeorbnqqe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,16,
-bitbucket:BitbucketIssue:1:63963454,bitbucket:BitbucketIssue:1:27,mmboeroi,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,17,
-bitbucket:BitbucketIssue:1:63963455,bitbucket:BitbucketIssue:1:27,lqooaqjoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,18,
-bitbucket:BitbucketIssue:1:63963456,bitbucket:BitbucketIssue:1:27,vajoieiowaeovw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,19,
-bitbucket:BitbucketIssue:1:63963457,bitbucket:BitbucketIssue:1:27,ieiojiwea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,20,
-bitbucket:BitbucketIssue:1:63963458,bitbucket:BitbucketIssue:1:27,beraowheuovhaewa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,21,
-bitbucket:BitbucketIssue:1:63963459,bitbucket:BitbucketIssue:1:27,oeorjeorb,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,22,
-bitbucket:BitbucketIssue:1:63963460,bitbucket:BitbucketIssue:1:27,iocowevoanewww,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,23,
-bitbucket:BitbucketIssue:1:63963461,bitbucket:BitbucketIssue:1:27,cowoehvowevaobnebiejaiweaw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,24,
-bitbucket:BitbucketIssue:1:63963462,bitbucket:BitbucketIssue:1:27,vbleklawi,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,25,
-bitbucket:BitbucketIssue:1:63963463,bitbucket:BitbucketIssue:1:27,qowjoveawev,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,26,
-bitbucket:BitbucketIssue:1:63963464,bitbucket:BitbucketIssue:1:27,ciojboieria,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,27,
-bitbucket:BitbucketIssue:1:63963465,bitbucket:BitbucketIssue:1:27,cwiwhoervowwwqe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issue_comments,28,
+bitbucket:BitbucketIssueComment:1:63860152,bitbucket:BitbucketIssue:1:likyh/likyhphp:1,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,1,
+bitbucket:BitbucketIssueComment:1:63860153,bitbucket:BitbucketIssue:1:likyh/likyhphp:1,this is a comment test for issue,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,2,
+bitbucket:BitbucketIssueComment:1:63860241,bitbucket:BitbucketIssue:1:likyh/likyhphp:1,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,3,
+bitbucket:BitbucketIssueComment:1:63961973,bitbucket:BitbucketIssue:1:likyh/likyhphp:6,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,4,
+bitbucket:BitbucketIssueComment:1:63961995,bitbucket:BitbucketIssue:1:likyh/likyhphp:28,,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,5,
+bitbucket:BitbucketIssueComment:1:63963442,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,gvoeaoivae,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,6,
+bitbucket:BitbucketIssueComment:1:63963443,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,vboeaorboaneo,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,7,
+bitbucket:BitbucketIssueComment:1:63963444,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,beoiroierbieiaoboere,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,8,
+bitbucket:BitbucketIssueComment:1:63963445,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,boetboenavnin,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,9,
+bitbucket:BitbucketIssueComment:1:63963446,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,bbenoirbnavbeboa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,10,
+bitbucket:BitbucketIssueComment:1:63963447,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,lboehoraebow,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,11,
+bitbucket:BitbucketIssueComment:1:63963449,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,ebeobnoerioaeiow,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,12,
+bitbucket:BitbucketIssueComment:1:63963450,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,uttrberoiea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,13,
+bitbucket:BitbucketIssueComment:1:63963451,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,obfbdfonwea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,14,
+bitbucket:BitbucketIssueComment:1:63963452,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,bjeorjboewaonwea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,15,
+bitbucket:BitbucketIssueComment:1:63963453,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,brtoeorbnqqe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,16,
+bitbucket:BitbucketIssueComment:1:63963454,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,mmboeroi,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,17,
+bitbucket:BitbucketIssueComment:1:63963455,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,lqooaqjoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,18,
+bitbucket:BitbucketIssueComment:1:63963456,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,vajoieiowaeovw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,19,
+bitbucket:BitbucketIssueComment:1:63963457,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,ieiojiwea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,20,
+bitbucket:BitbucketIssueComment:1:63963458,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,beraowheuovhaewa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,21,
+bitbucket:BitbucketIssueComment:1:63963459,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,oeorjeorb,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,22,
+bitbucket:BitbucketIssueComment:1:63963460,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,iocowevoanewww,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,23,
+bitbucket:BitbucketIssueComment:1:63963461,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,cowoehvowevaobnebiejaiweaw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,24,
+bitbucket:BitbucketIssueComment:1:63963462,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,vbleklawi,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,25,
+bitbucket:BitbucketIssueComment:1:63963463,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,qowjoveawev,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,26,
+bitbucket:BitbucketIssueComment:1:63963464,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,ciojboieria,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,27,
+bitbucket:BitbucketIssueComment:1:63963465,bitbucket:BitbucketIssue:1:likyh/likyhphp:27,cwiwhoervowwwqe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issue_comments,28,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/issues.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/issues.csv
index 80730dec1..4ab51932a 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/issues.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/issues.csv
@@ -1,28 +1,28 @@
 id,url,icon_url,issue_key,title,description,epic_key,type,status,original_status,story_point,resolution_date,lead_time_minutes,parent_issue_id,priority,original_estimate_minutes,time_spent_minutes,time_remaining_minutes,creator_id,creator_name,assignee_id,assignee_name,severity,component,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketIssue:1:1,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/1,,1,issue test,bitbucket issues test for devants,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,60,
-bitbucket:BitbucketIssue:1:10,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/10,,10,issue test007,issue test007,,issue,TODO,TODO,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,52,
-bitbucket:BitbucketIssue:1:11,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/11,,11,issue test008,issue test008,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,51,
-bitbucket:BitbucketIssue:1:12,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/12,,12,issue test009,issue test009,,issue,TODO,TODO,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,50,
-bitbucket:BitbucketIssue:1:13,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/13,,13,issue test010,issue test010,,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,49,
-bitbucket:BitbucketIssue:1:14,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/14,,14,issue test011,issue test011,,issue,TODO,TODO,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,48,
-bitbucket:BitbucketIssue:1:15,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/15,,15,issue test012,issue test012,,issue,TODO,TODO,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,47,
-bitbucket:BitbucketIssue:1:16,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/16,,16,issue test013,issue test013,,issue,TODO,TODO,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,46,
-bitbucket:BitbucketIssue:1:17,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/17,,17,issue test014,issue test014,,issue,TODO,TODO,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,45,
-bitbucket:BitbucketIssue:1:18,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/18,,18,issue test015,issue test015,,issue,TODO,TODO,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,44,
-bitbucket:BitbucketIssue:1:19,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/19,,19,issue test016,issue test016,,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,43,
-bitbucket:BitbucketIssue:1:2,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/2,,2,add bitbucket issue,feafejo,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,59,
-bitbucket:BitbucketIssue:1:20,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/20,,20,issue test017,issue test017,,issue,TODO,TODO,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,42,
-bitbucket:BitbucketIssue:1:21,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/21,,21,issue test018,issue test018,,issue,TODO,TODO,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,41,
-bitbucket:BitbucketIssue:1:22,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/22,,22,issue test019,issue test019,,issue,TODO,TODO,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,40,
-bitbucket:BitbucketIssue:1:23,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/23,,23,issue test020,issue test020,,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,39,
-bitbucket:BitbucketIssue:1:24,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/24,,24,issue test021,"issue test021 ijeiawgoeive/faveevaeviaevfejaofejfioejaiofe_veavejiovajgiorejoifjrogiorejieafajejaojoejvgioriovioraivjairobnrnoivaiorjbiorjiojaeiorjvioejroivjaoijeriojiaojioeefjafioejfiojeiofawefwefoiwefiwoiefweefwoefuwhufirfrw._  
+bitbucket:BitbucketIssue:1:likyh/likyhphp:1,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/1,,1,issue test,bitbucket issues test for devants,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,60,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:10,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/10,,10,issue test007,issue test007,,issue,TODO,new,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,52,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:11,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/11,,11,issue test008,issue test008,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,51,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:12,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/12,,12,issue test009,issue test009,,issue,TODO,new,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,50,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:13,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/13,,13,issue test010,issue test010,,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,49,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:14,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/14,,14,issue test011,issue test011,,issue,TODO,new,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,48,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:15,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/15,,15,issue test012,issue test012,,issue,TODO,new,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,47,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:16,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/16,,16,issue test013,issue test013,,issue,TODO,new,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,46,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:17,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/17,,17,issue test014,issue test014,,issue,TODO,new,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,45,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:18,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/18,,18,issue test015,issue test015,,issue,TODO,new,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,44,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:19,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/19,,19,issue test016,issue test016,,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,43,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:2,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/2,,2,add bitbucket issue,feafejo,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,59,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:20,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/20,,20,issue test017,issue test017,,issue,TODO,new,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,42,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:21,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/21,,21,issue test018,issue test018,,issue,TODO,new,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,41,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:22,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/22,,22,issue test019,issue test019,,issue,TODO,new,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,40,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:23,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/23,,23,issue test020,issue test020,,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,39,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:24,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/24,,24,issue test021,"issue test021 ijeiawgoeive/faveevaeviaevfejaofejfioejaiofe_veavejiovajgiorejoifjrogiorejieafajejaojoejvgioriovioraivjairobnrnoivaiorjbiorjiojaeiorjvioejroivjaoijeriojiaojioeefjafioejfiojeiofawefwefoiwefiwoiefweefwoefuwhufirfrw._
 
 ```
 ``eveaeaeae`jiovjeiojoa oi2j3oij23jovnw3
 ```
 
-‌",,issue,TODO,TODO,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,38,
-bitbucket:BitbucketIssue:1:25,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/25,,25,issue 023,"issue test023 ejoafehoafe
+‌",,issue,TODO,new,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,38,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:25,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/25,,25,issue 023,"issue test023 ejoafehoafe
 
 q
 
@@ -38,18 +38,18 @@ feaofe
 |  |  |  |
 |  |  |  |
 
-‌",,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,36,
-bitbucket:BitbucketIssue:1:26,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/26,,26,issue test022,issue test022,,issue,TODO,TODO,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,35,
-bitbucket:BitbucketIssue:1:27,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/27,,27,issue test024,"issue test024v  aejnoafoeiogoiae
+‌",,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,36,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:26,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/26,,26,issue test022,issue test022,,issue,TODO,new,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,35,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:27,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/27,,27,issue test024,"issue test024v  aejnoafoeiogoiae
 
-qwofjeoiwjf",,issue,TODO,TODO,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,31,
-bitbucket:BitbucketIssue:1:28,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/28,,28,issue test025,issue test025,,issue,TODO,TODO,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,34,
-bitbucket:BitbucketIssue:1:29,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/29,,29,issue test026,issue test026,,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,33,
-bitbucket:BitbucketIssue:1:3,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/3,,3,bitbucket test,"efaegjeoaijefioaegrjoeior,af enfaoiee vioea.,,.wew",,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,58,
-bitbucket:BitbucketIssue:1:30,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/30,,30,issue test027,issue test027,,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,32,
-bitbucket:BitbucketIssue:1:4,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/4,,4,issue test001,Bitbucket issue test001,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,57,
-bitbucket:BitbucketIssue:1:5,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/5,,5,issue test002,issue test002,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,56,
-bitbucket:BitbucketIssue:1:6,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/6,,6,issue test003,issue test 003,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,37,
-bitbucket:BitbucketIssue:1:7,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/7,,7,issue test004,issue test004,,issue,TODO,TODO,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,55,
-bitbucket:BitbucketIssue:1:8,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/8,,8,issue test005,issue test005,,issue,TODO,TODO,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,54,
-bitbucket:BitbucketIssue:1:9,https://api.bitbucket.org/2.0/repositories/panjf2000/ants/issues/9,,9,issue test006,issue test006,,issue,TODO,TODO,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_issues,53,
+qwofjeoiwjf",,issue,TODO,new,0,,0,,trivial,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,31,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:28,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/28,,28,issue test025,issue test025,,issue,TODO,new,0,,0,,minor,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,34,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:29,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/29,,29,issue test026,issue test026,,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,33,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:3,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/3,,3,bitbucket test,"efaegjeoaijefioaegrjoeior,af enfaoiee vioea.,,.wew",,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,58,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:30,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/30,,30,issue test027,issue test027,,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,32,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:4,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/4,,4,issue test001,Bitbucket issue test001,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,57,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:5,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/5,,5,issue test002,issue test002,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,56,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:6,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/6,,6,issue test003,issue test 003,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,37,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:7,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/7,,7,issue test004,issue test004,,issue,TODO,new,0,,0,,major,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,55,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:8,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/8,,8,issue test005,issue test005,,issue,TODO,new,0,,0,,critical,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,54,
+bitbucket:BitbucketIssue:1:likyh/likyhphp:9,https://api.bitbucket.org/2.0/repositories/likyh/likyhphp/issues/9,,9,issue test006,issue test006,,issue,TODO,new,0,,0,,blocker,0,0,0,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,teoiaoe,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_issues,53,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/pull_request_comments.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/pull_request_comments.csv
index 189616fd9..9ddc97b5f 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/pull_request_comments.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/pull_request_comments.csv
@@ -1,21 +1,21 @@
 id,pull_request_id,body,account_id,commit_sha,position,type,review_id,status,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketPrComment:1:323119643,bitbucket:BitbucketPullRequest:1:54,this is test014,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,14,
-bitbucket:BitbucketPrComment:1:323119662,bitbucket:BitbucketPullRequest:1:54,feafjae,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,15,
-bitbucket:BitbucketPrComment:1:323119730,bitbucket:BitbucketPullRequest:1:57,"fejoiafjoej
+bitbucket:BitbucketPrComment:1:323119643,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:54,this is test014,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,14,
+bitbucket:BitbucketPrComment:1:323119662,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:54,feafjae,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,15,
+bitbucket:BitbucketPrComment:1:323119730,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:57,"fejoiafjoej
 
-‌",bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,17,
-bitbucket:BitbucketPrComment:1:323119743,bitbucket:BitbucketPullRequest:1:57,fjeafojeoirjgoar,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,18,
-bitbucket:BitbucketPrComment:1:323119784,bitbucket:BitbucketPullRequest:1:51,fejoafefewa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,12,
-bitbucket:BitbucketPrComment:1:323119790,bitbucket:BitbucketPullRequest:1:51,feafeogioehrw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,13,
-bitbucket:BitbucketPrComment:1:323119842,bitbucket:BitbucketPullRequest:1:42,feafoegoaweoir,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,8,
-bitbucket:BitbucketPrComment:1:323119856,bitbucket:BitbucketPullRequest:1:42,boroehruheorefe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,9,
-bitbucket:BitbucketPrComment:1:323119897,bitbucket:BitbucketPullRequest:1:54,feajfweofjwvgrea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,16,
-bitbucket:BitbucketPrComment:1:323119960,bitbucket:BitbucketPullRequest:1:38,regrioteothro,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,4,
-bitbucket:BitbucketPrComment:1:323119980,bitbucket:BitbucketPullRequest:1:38,goeotbioebhotn,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,5,
-bitbucket:BitbucketPrComment:1:323120091,bitbucket:BitbucketPullRequest:1:44,feawrjoiw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,10,
-bitbucket:BitbucketPrComment:1:323120101,bitbucket:BitbucketPullRequest:1:44,bbeirobeoiahoa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,11,
-bitbucket:BitbucketPrComment:1:323120140,bitbucket:BitbucketPullRequest:1:40,evbnoera,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,6,
-bitbucket:BitbucketPrComment:1:323120196,bitbucket:BitbucketPullRequest:1:41,vobreia,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,7,
-bitbucket:BitbucketPrComment:1:323120278,bitbucket:BitbucketPullRequest:1:31,baroboerhba,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,1,
-bitbucket:BitbucketPrComment:1:323120296,bitbucket:BitbucketPullRequest:1:31,bweiwbeanb,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,2,
-bitbucket:BitbucketPrComment:1:323120311,bitbucket:BitbucketPullRequest:1:31,jbirobsore,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_request_comments,3,
+‌",bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,17,
+bitbucket:BitbucketPrComment:1:323119743,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:57,fjeafojeoirjgoar,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,18,
+bitbucket:BitbucketPrComment:1:323119784,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:51,fejoafefewa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,12,
+bitbucket:BitbucketPrComment:1:323119790,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:51,feafeogioehrw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,13,
+bitbucket:BitbucketPrComment:1:323119842,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:42,feafoegoaweoir,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,8,
+bitbucket:BitbucketPrComment:1:323119856,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:42,boroehruheorefe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,9,
+bitbucket:BitbucketPrComment:1:323119897,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:54,feajfweofjwvgrea,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,16,
+bitbucket:BitbucketPrComment:1:323119960,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:38,regrioteothro,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,4,
+bitbucket:BitbucketPrComment:1:323119980,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:38,goeotbioebhotn,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,5,
+bitbucket:BitbucketPrComment:1:323120091,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:44,feawrjoiw,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,10,
+bitbucket:BitbucketPrComment:1:323120101,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:44,bbeirobeoiahoa,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,11,
+bitbucket:BitbucketPrComment:1:323120140,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:40,evbnoera,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,6,
+bitbucket:BitbucketPrComment:1:323120196,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:41,vobreia,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,7,
+bitbucket:BitbucketPrComment:1:323120278,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:31,baroboerhba,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,1,
+bitbucket:BitbucketPrComment:1:323120296,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:31,bweiwbeanb,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,2,
+bitbucket:BitbucketPrComment:1:323120311,bitbucket:BitbucketPullRequest:1:likyh/likyhphp:31,jbirobsore,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,pullrequest_comment,,,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_request_comments,3,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/pull_requests.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/pull_requests.csv
index 8052c1ff0..d3162fb98 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/pull_requests.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/pull_requests.csv
@@ -1,6 +1,6 @@
 id,base_repo_id,head_repo_id,status,title,description,url,author_name,author_id,parent_pr_id,pull_request_key,merged_date,closed_date,type,component,merge_commit_sha,head_ref,base_ref,base_commit_sha,head_commit_sha,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketPullRequest:1:1,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,feat: add pr test,this is a pr test for Bitbucket plugin,https://bitbucket.org/panjf2000/ants/pull-requests/1,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test,master,c0e12b61e44b,5e5bccfcc656,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,171,
-bitbucket:BitbucketPullRequest:1:10,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test007,"* feat: add git.sh
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:1,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,feat: add pr test,this is a pr test for Bitbucket plugin,https://bitbucket.org/likyh/likyhphp/pull-requests/1,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,1,,,pullrequest,,,test,master,c0e12b61e44b,5e5bccfcc656,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,171,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:10,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test007,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -9,8 +9,8 @@ bitbucket:BitbucketPullRequest:1:10,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test006
 * feat: add test007
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/10,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test007,master,0bd83caaf9c7,15a1fcb51f0e,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,161,
-bitbucket:BitbucketPullRequest:1:11,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test6,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/10,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,10,,,pullrequest,,983edbf06740,test007,master,0bd83caaf9c7,15a1fcb51f0e,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,161,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:11,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test6,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -32,8 +32,8 @@ bitbucket:BitbucketPullRequest:1:11,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add 5
 * feat: add test6
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/11,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test6,master,0bd83caaf9c7,bc9a3ddd47ea,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,160,
-bitbucket:BitbucketPullRequest:1:12,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test7,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/11,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,11,,,pullrequest,,983edbf06740,test6,master,0bd83caaf9c7,bc9a3ddd47ea,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,160,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:12,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test7,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -56,8 +56,8 @@ bitbucket:BitbucketPullRequest:1:12,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test6
 * feat: add test7
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/12,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test7,master,0bd83caaf9c7,dd2cd76d7098,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,159,
-bitbucket:BitbucketPullRequest:1:13,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test8,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/12,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,12,,,pullrequest,,983edbf06740,test7,master,0bd83caaf9c7,dd2cd76d7098,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,159,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:13,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test8,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -81,8 +81,8 @@ bitbucket:BitbucketPullRequest:1:13,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test7
 * feat: add test8
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/13,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test8,master,0bd83caaf9c7,94056215cf46,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,158,
-bitbucket:BitbucketPullRequest:1:14,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test10,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/13,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,13,,,pullrequest,,983edbf06740,test8,master,0bd83caaf9c7,94056215cf46,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,158,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:14,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test10,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -108,8 +108,8 @@ bitbucket:BitbucketPullRequest:1:14,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test9
 * feat: add test10
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/14,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test10,master,0bd83caaf9c7,bc2cf25142e3,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,157,
-bitbucket:BitbucketPullRequest:1:15,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test011,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/14,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,14,,,pullrequest,,983edbf06740,test10,master,0bd83caaf9c7,bc2cf25142e3,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,157,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:15,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test011,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -122,8 +122,8 @@ bitbucket:BitbucketPullRequest:1:15,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test010
 * feat: add test011
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/15,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test011,master,0bd83caaf9c7,5db6b1a9e60c,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,156,
-bitbucket:BitbucketPullRequest:1:16,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test12,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/15,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,15,,,pullrequest,,983edbf06740,test011,master,0bd83caaf9c7,5db6b1a9e60c,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,156,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:16,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test12,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -151,8 +151,8 @@ bitbucket:BitbucketPullRequest:1:16,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test11
 * feat: add test12
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/16,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test12,master,0bd83caaf9c7,d1321cb5e4fb,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,170,
-bitbucket:BitbucketPullRequest:1:17,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test13,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/16,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,16,,,pullrequest,,983edbf06740,test12,master,0bd83caaf9c7,d1321cb5e4fb,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,170,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:17,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test13,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -181,8 +181,8 @@ bitbucket:BitbucketPullRequest:1:17,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test12
 * feat: add test13
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/17,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test13,master,983edbf06740,3d6d96659c4d,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,151,
-bitbucket:BitbucketPullRequest:1:18,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test14,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/17,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,17,,,pullrequest,,c0e12b61e44b,test13,master,983edbf06740,3d6d96659c4d,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,151,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:18,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test14,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -212,8 +212,8 @@ bitbucket:BitbucketPullRequest:1:18,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test13
 * feat: add test14
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/18,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test14,master,983edbf06740,8d4896eef3ed,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,150,
-bitbucket:BitbucketPullRequest:1:19,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test15,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/18,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,18,,,pullrequest,,c0e12b61e44b,test14,master,983edbf06740,8d4896eef3ed,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,150,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:19,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test15,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -244,9 +244,9 @@ bitbucket:BitbucketPullRequest:1:19,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test14
 * feat: add test15
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/19,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test15,master,983edbf06740,daa1cfa7ec0c,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,149,
-bitbucket:BitbucketPullRequest:1:2,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,feat: add git.sh,help to git commit faster,https://bitbucket.org/panjf2000/ants/pull-requests/2,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,vowehwo,master,0bd83caaf9c7,d4a6789a3751,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,169,
-bitbucket:BitbucketPullRequest:1:20,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test16,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/19,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,19,,,pullrequest,,c0e12b61e44b,test15,master,983edbf06740,daa1cfa7ec0c,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,149,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:2,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,feat: add git.sh,help to git commit faster,https://bitbucket.org/likyh/likyhphp/pull-requests/2,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,2,,,pullrequest,,983edbf06740,vowehwo,master,0bd83caaf9c7,d4a6789a3751,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,169,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:20,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test16,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -278,8 +278,8 @@ bitbucket:BitbucketPullRequest:1:20,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test15
 * feat: add test16
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/20,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test16,master,983edbf06740,11f6c86580e2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,148,
-bitbucket:BitbucketPullRequest:1:21,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test18,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/20,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,20,,,pullrequest,,c0e12b61e44b,test16,master,983edbf06740,11f6c86580e2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,148,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:21,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test18,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -313,8 +313,8 @@ bitbucket:BitbucketPullRequest:1:21,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test17
 * feat: add test18
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/21,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test18,master,983edbf06740,0b958bbfed04,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,147,
-bitbucket:BitbucketPullRequest:1:22,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test19,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/21,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,21,,,pullrequest,,c0e12b61e44b,test18,master,983edbf06740,0b958bbfed04,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,147,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:22,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test19,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -349,8 +349,8 @@ bitbucket:BitbucketPullRequest:1:22,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test18
 * feat: add test19
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/22,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test19,master,983edbf06740,d1bb3677810e,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,146,
-bitbucket:BitbucketPullRequest:1:23,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test20,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/22,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,22,,,pullrequest,,c0e12b61e44b,test19,master,983edbf06740,d1bb3677810e,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,146,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:23,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test20,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -386,8 +386,8 @@ bitbucket:BitbucketPullRequest:1:23,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test19
 * feat: add test20
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/23,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test20,master,983edbf06740,9d16a2368acd,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,145,
-bitbucket:BitbucketPullRequest:1:24,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test17,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/23,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,23,,,pullrequest,,c0e12b61e44b,test20,master,983edbf06740,9d16a2368acd,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,145,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:24,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test17,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -420,8 +420,8 @@ bitbucket:BitbucketPullRequest:1:24,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test16
 * feat: add test17
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/24,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,test17,master,983edbf06740,2bfd29a17665,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,144,
-bitbucket:BitbucketPullRequest:1:25,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,1,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/24,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,24,,,pullrequest,,c0e12b61e44b,test17,master,983edbf06740,2bfd29a17665,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,144,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:25,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,1,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -438,8 +438,8 @@ bitbucket:BitbucketPullRequest:1:25,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add branchname
 * feat: add 1
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/25,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,1,master,0bd83caaf9c7,592612c47ff5,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,155,
-bitbucket:BitbucketPullRequest:1:26,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,2,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/25,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,25,,,pullrequest,,983edbf06740,1,master,0bd83caaf9c7,592612c47ff5,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,155,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:26,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,2,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -457,8 +457,8 @@ bitbucket:BitbucketPullRequest:1:26,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add 1
 * feat: add 2
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/26,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,2,master,0bd83caaf9c7,05b97cf92b4b,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,154,
-bitbucket:BitbucketPullRequest:1:27,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,4,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/26,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,26,,,pullrequest,,983edbf06740,2,master,0bd83caaf9c7,05b97cf92b4b,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,154,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:27,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,4,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -478,14 +478,14 @@ bitbucket:BitbucketPullRequest:1:27,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add 3
 * feat: add 4
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/27,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,4,master,0bd83caaf9c7,761fdb9244ef,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,153,
-bitbucket:BitbucketPullRequest:1:28,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,mergev1,merge all v1,https://bitbucket.org/panjf2000/ants/pull-requests/28,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,c0e12b61e44b,mergev1,master,983edbf06740,581a89006076,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,152,
-bitbucket:BitbucketPullRequest:1:29,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test03,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/27,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,27,,,pullrequest,,983edbf06740,4,master,0bd83caaf9c7,761fdb9244ef,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,153,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:28,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,mergev1,merge all v1,https://bitbucket.org/likyh/likyhphp/pull-requests/28,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,28,,,pullrequest,,c0e12b61e44b,mergev1,master,983edbf06740,581a89006076,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,152,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:29,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test03,"* feat: add test01
 * feat: add test02
 * feat: add test03
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/29,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test03,master,c0e12b61e44b,a3768d182d18,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,132,
-bitbucket:BitbucketPullRequest:1:3,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test9,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/29,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,29,,,pullrequest,,,test03,master,c0e12b61e44b,a3768d182d18,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,132,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:3,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test9,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -510,15 +510,15 @@ bitbucket:BitbucketPullRequest:1:3,bitbucket:BitbucketRepo:1:panjf2000/ants,bitb
 * feat: add test8
 * feat: add test9
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/3,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test9,master,0bd83caaf9c7,6557208f4aa6,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,168,
-bitbucket:BitbucketPullRequest:1:30,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test05,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/3,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,3,,,pullrequest,,983edbf06740,test9,master,0bd83caaf9c7,6557208f4aa6,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,168,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:30,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test05,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
 * feat: add test05
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/30,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test05,master,c0e12b61e44b,91e7a1db9ae0,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,143,
-bitbucket:BitbucketPullRequest:1:31,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test020,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/30,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,30,,,pullrequest,,,test05,master,c0e12b61e44b,91e7a1db9ae0,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,143,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:31,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test020,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -539,8 +539,8 @@ bitbucket:BitbucketPullRequest:1:31,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test019
 * feat: add test020
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/31,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test020,master,c0e12b61e44b,0342bf5351f8,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,118,
-bitbucket:BitbucketPullRequest:1:32,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test022,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/31,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,31,,,pullrequest,,,test020,master,c0e12b61e44b,0342bf5351f8,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,118,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:32,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test022,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -563,8 +563,8 @@ bitbucket:BitbucketPullRequest:1:32,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test021
 * feat: add test022
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/32,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test022,master,c0e12b61e44b,d72c46a722d7,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,142,
-bitbucket:BitbucketPullRequest:1:33,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test023,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/32,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,32,,,pullrequest,,,test022,master,c0e12b61e44b,d72c46a722d7,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,142,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:33,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test023,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -588,8 +588,8 @@ bitbucket:BitbucketPullRequest:1:33,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test022
 * feat: add test023
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/33,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test023,master,c0e12b61e44b,0a8011c8c0ac,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,141,
-bitbucket:BitbucketPullRequest:1:34,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test024,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/33,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,33,,,pullrequest,,,test023,master,c0e12b61e44b,0a8011c8c0ac,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,141,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:34,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test024,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -614,8 +614,8 @@ bitbucket:BitbucketPullRequest:1:34,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test023
 * feat: add test024
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/34,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test024,master,c0e12b61e44b,bd7e81a581f1,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,140,
-bitbucket:BitbucketPullRequest:1:35,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test025,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/34,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,34,,,pullrequest,,,test024,master,c0e12b61e44b,bd7e81a581f1,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,140,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:35,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test025,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -641,8 +641,8 @@ bitbucket:BitbucketPullRequest:1:35,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test024
 * feat: add test025
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/35,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test025,master,c0e12b61e44b,a15cb5876414,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,139,
-bitbucket:BitbucketPullRequest:1:36,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test027,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/35,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,35,,,pullrequest,,,test025,master,c0e12b61e44b,a15cb5876414,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,139,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:36,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test027,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -670,8 +670,8 @@ bitbucket:BitbucketPullRequest:1:36,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test026
 * feat: add test027
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/36,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test027,master,c0e12b61e44b,200665f463ff,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,138,
-bitbucket:BitbucketPullRequest:1:37,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test028,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/36,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,36,,,pullrequest,,,test027,master,c0e12b61e44b,200665f463ff,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,138,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:37,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test028,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -700,8 +700,8 @@ bitbucket:BitbucketPullRequest:1:37,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test027
 * feat: add test028
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/37,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test028,master,c0e12b61e44b,8a9052ac97f4,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,137,
-bitbucket:BitbucketPullRequest:1:38,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test029,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/37,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,37,,,pullrequest,,,test028,master,c0e12b61e44b,8a9052ac97f4,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,137,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:38,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test029,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -731,8 +731,8 @@ bitbucket:BitbucketPullRequest:1:38,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test028
 * feat: add test029
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/38,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test029,master,c0e12b61e44b,74bddc0541c6,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,121,
-bitbucket:BitbucketPullRequest:1:39,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test030,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/38,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,38,,,pullrequest,,,test029,master,c0e12b61e44b,74bddc0541c6,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,121,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:39,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test030,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -763,12 +763,12 @@ bitbucket:BitbucketPullRequest:1:39,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test029
 * feat: add test030
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/39,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test030,master,c0e12b61e44b,f332a9855de1,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,136,
-bitbucket:BitbucketPullRequest:1:4,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test001,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/39,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,39,,,pullrequest,,,test030,master,c0e12b61e44b,f332a9855de1,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,136,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:4,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test001,"* feat: add git.sh
 * feat: add test001
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/4,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test001,master,0bd83caaf9c7,3bd3b34b2397,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,167,
-bitbucket:BitbucketPullRequest:1:40,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test016,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/4,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,4,,,pullrequest,,983edbf06740,test001,master,0bd83caaf9c7,3bd3b34b2397,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,167,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:40,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test016,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -785,8 +785,8 @@ bitbucket:BitbucketPullRequest:1:40,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test015
 * feat: add test016
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/40,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test016,master,c0e12b61e44b,a000a2a9fc11,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,115,
-bitbucket:BitbucketPullRequest:1:41,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test018,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/40,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,40,,,pullrequest,,,test016,master,c0e12b61e44b,a000a2a9fc11,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,115,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:41,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test018,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -805,8 +805,8 @@ bitbucket:BitbucketPullRequest:1:41,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test017
 * feat: add test018
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/41,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test018,master,c0e12b61e44b,561e4e3ca4cc,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,119,
-bitbucket:BitbucketPullRequest:1:42,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test019,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/41,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,41,,,pullrequest,,,test018,master,c0e12b61e44b,561e4e3ca4cc,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,119,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:42,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test019,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -826,8 +826,8 @@ bitbucket:BitbucketPullRequest:1:42,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test018
 * feat: add test019
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/42,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test019,master,c0e12b61e44b,8597a99eebb3,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,117,
-bitbucket:BitbucketPullRequest:1:43,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test015,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/42,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,42,,,pullrequest,,,test019,master,c0e12b61e44b,8597a99eebb3,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,117,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:43,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test015,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -843,8 +843,8 @@ bitbucket:BitbucketPullRequest:1:43,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test014
 * feat: add test015
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/43,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test015,master,c0e12b61e44b,5af2c361263a,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,135,
-bitbucket:BitbucketPullRequest:1:44,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test017,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/43,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,43,,,pullrequest,,,test015,master,c0e12b61e44b,5af2c361263a,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,135,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:44,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test017,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -862,23 +862,23 @@ bitbucket:BitbucketPullRequest:1:44,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test016
 * feat: add test017
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/44,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test017,master,c0e12b61e44b,ee87b0380472,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,120,
-bitbucket:BitbucketPullRequest:1:45,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,feat: add test01,,https://bitbucket.org/panjf2000/ants/pull-requests/45,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test01,master,c0e12b61e44b,63e6acbb60a9,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,134,
-bitbucket:BitbucketPullRequest:1:46,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test04,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/44,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,44,,,pullrequest,,,test017,master,c0e12b61e44b,ee87b0380472,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,120,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:45,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,feat: add test01,,https://bitbucket.org/likyh/likyhphp/pull-requests/45,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,45,,,pullrequest,,,test01,master,c0e12b61e44b,63e6acbb60a9,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,134,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:46,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test04,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/46,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test04,master,c0e12b61e44b,33419e166d75,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,133,
-bitbucket:BitbucketPullRequest:1:47,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test06,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/46,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,46,,,pullrequest,,,test04,master,c0e12b61e44b,33419e166d75,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,133,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:47,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test06,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
 * feat: add test05
 * feat: add test06
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/47,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test06,master,c0e12b61e44b,f8f9ef467e3a,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,131,
-bitbucket:BitbucketPullRequest:1:48,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test07,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/47,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,47,,,pullrequest,,,test06,master,c0e12b61e44b,f8f9ef467e3a,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,131,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:48,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test07,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -886,8 +886,8 @@ bitbucket:BitbucketPullRequest:1:48,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test06
 * feat: add test07
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/48,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test07,master,c0e12b61e44b,c28f34dde568,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,130,
-bitbucket:BitbucketPullRequest:1:49,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Test08,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/48,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,48,,,pullrequest,,,test07,master,c0e12b61e44b,c28f34dde568,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,130,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:49,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Test08,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -896,13 +896,13 @@ bitbucket:BitbucketPullRequest:1:49,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add test07
 * feat: add test08
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/49,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,test08,master,c0e12b61e44b,6b59c8607581,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,129,
-bitbucket:BitbucketPullRequest:1:5,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test002,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/49,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,49,,,pullrequest,,,test08,master,c0e12b61e44b,6b59c8607581,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,129,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:5,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test002,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/5,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test002,master,0bd83caaf9c7,904df6924f1e,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,166,
-bitbucket:BitbucketPullRequest:1:50,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest12,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/5,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,5,,,pullrequest,,983edbf06740,test002,master,0bd83caaf9c7,904df6924f1e,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,166,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:50,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest12,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -936,8 +936,8 @@ bitbucket:BitbucketPullRequest:1:50,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbucket
 * feat: add bitbuckettest12
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/50,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest12,master,c0e12b61e44b,deff6927812f,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,128,
-bitbucket:BitbucketPullRequest:1:51,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest15,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/50,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,50,,,pullrequest,,,bitbuckettest12,master,c0e12b61e44b,deff6927812f,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,128,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:51,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest15,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -974,8 +974,8 @@ bitbucket:BitbucketPullRequest:1:51,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest14
 * feat: add bitbuckettest15
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/51,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest15,master,c0e12b61e44b,5c2c9af70da2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,122,
-bitbucket:BitbucketPullRequest:1:52,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest16,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/51,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,51,,,pullrequest,,,bitbuckettest15,master,c0e12b61e44b,5c2c9af70da2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,122,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:52,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest16,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -1013,8 +1013,8 @@ bitbucket:BitbucketPullRequest:1:52,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest15
 * feat: add bitbuckettest16
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/52,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest16,master,c0e12b61e44b,7c91d40ffaa3,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,127,
-bitbucket:BitbucketPullRequest:1:53,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest17,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/52,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,52,,,pullrequest,,,bitbuckettest16,master,c0e12b61e44b,7c91d40ffaa3,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,127,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:53,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest17,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -1053,8 +1053,8 @@ bitbucket:BitbucketPullRequest:1:53,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest16
 * feat: add bitbuckettest17
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/53,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest17,master,c0e12b61e44b,765ab2594451,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,125,
-bitbucket:BitbucketPullRequest:1:54,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest19,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/53,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,53,,,pullrequest,,,bitbuckettest17,master,c0e12b61e44b,765ab2594451,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,125,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:54,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest19,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -1095,8 +1095,8 @@ bitbucket:BitbucketPullRequest:1:54,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest18
 * feat: add bitbuckettest19
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/54,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest19,master,c0e12b61e44b,2493b9212fd4,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,116,
-bitbucket:BitbucketPullRequest:1:55,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest13,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/54,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,54,,,pullrequest,,,bitbuckettest19,master,c0e12b61e44b,2493b9212fd4,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,116,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:55,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest13,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -1131,8 +1131,8 @@ bitbucket:BitbucketPullRequest:1:55,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest12
 * feat: add bitbuckettest13
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/55,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest13,master,c0e12b61e44b,9dd8c98115ca,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,126,
-bitbucket:BitbucketPullRequest:1:56,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest25,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/55,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,55,,,pullrequest,,,bitbuckettest13,master,c0e12b61e44b,9dd8c98115ca,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,126,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:56,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest25,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -1179,8 +1179,8 @@ bitbucket:BitbucketPullRequest:1:56,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest24
 * feat: add bitbuckettest25
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/56,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest25,master,c0e12b61e44b,d69faf8c90c4,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,124,
-bitbucket:BitbucketPullRequest:1:57,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,OPEN,Bitbuckettest22,"* feat: add test01
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/56,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,56,,,pullrequest,,,bitbuckettest25,master,c0e12b61e44b,d69faf8c90c4,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,124,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:57,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,OPEN,Bitbuckettest22,"* feat: add test01
 * feat: add test02
 * feat: add test03
 * feat: add test04
@@ -1224,29 +1224,29 @@ bitbucket:BitbucketPullRequest:1:57,bitbucket:BitbucketRepo:1:panjf2000/ants,bit
 * feat: add bitbuckettest21
 * feat: add bitbuckettest22
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/57,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,,bitbuckettest22,master,c0e12b61e44b,2c23b3babf27,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,123,
-bitbucket:BitbucketPullRequest:1:6,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test003,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/57,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,57,,,pullrequest,,,bitbuckettest22,master,c0e12b61e44b,2c23b3babf27,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,123,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:6,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test003,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/6,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test003,master,0bd83caaf9c7,494b97e9d0b2,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,165,
-bitbucket:BitbucketPullRequest:1:7,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test004,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/6,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,6,,,pullrequest,,983edbf06740,test003,master,0bd83caaf9c7,494b97e9d0b2,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,165,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:7,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test004,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
 * feat: add test004
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/7,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test004,master,0bd83caaf9c7,db921ce6537c,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,164,
-bitbucket:BitbucketPullRequest:1:8,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test005,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/7,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,7,,,pullrequest,,983edbf06740,test004,master,0bd83caaf9c7,db921ce6537c,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,164,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:8,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test005,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
 * feat: add test004
 * feat: add test005
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/8,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test005,master,0bd83caaf9c7,8a224cc0eaae,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,163,
-bitbucket:BitbucketPullRequest:1:9,bitbucket:BitbucketRepo:1:panjf2000/ants,bitbucket:BitbucketRepo:1:panjf2000/ants,MERGED,Test006,"* feat: add git.sh
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/8,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,8,,,pullrequest,,983edbf06740,test005,master,0bd83caaf9c7,8a224cc0eaae,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,163,
+bitbucket:BitbucketPullRequest:1:likyh/likyhphp:9,bitbucket:BitbucketRepo:1:likyh/likyhphp,bitbucket:BitbucketRepo:1:likyh/likyhphp,MERGED,Test006,"* feat: add git.sh
 * feat: add test001
 * feat: test002
 * feat: add test003
@@ -1254,4 +1254,4 @@ bitbucket:BitbucketPullRequest:1:9,bitbucket:BitbucketRepo:1:panjf2000/ants,bitb
 * feat: add test005
 * feat: add test006
 
-‌",https://bitbucket.org/panjf2000/ants/pull-requests/9,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,0,,,pullrequest,,983edbf06740,test006,master,0bd83caaf9c7,8a8f44b997a8,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_pull_requests,162,
+‌",https://bitbucket.org/likyh/likyhphp/pull-requests/9,teoiaoe,bitbucket:BitbucketAccount:1:62abf394192edb006fa0e8cf,,9,,,pullrequest,,983edbf06740,test006,master,0bd83caaf9c7,8a8f44b997a8,"{""ConnectionId"":1,""FullName"":""likyh/likyhphp""}",_raw_bitbucket_api_pull_requests,162,
diff --git a/backend/plugins/bitbucket/e2e/snapshot_tables/repos.csv b/backend/plugins/bitbucket/e2e/snapshot_tables/repos.csv
index dc866278d..2cea80e69 100644
--- a/backend/plugins/bitbucket/e2e/snapshot_tables/repos.csv
+++ b/backend/plugins/bitbucket/e2e/snapshot_tables/repos.csv
@@ -1,2 +1,2 @@
 id,name,url,description,owner_id,language,forked_from,deleted,_raw_data_params,_raw_data_table,_raw_data_id,_raw_data_remark
-bitbucket:BitbucketRepo:1:panjf2000/ants,panjf2000/ants,https://bitbucket.org/panjf2000/ants,,,,,0,"{""ConnectionId"":1,""Owner"":""panjf2000"",""Repo"":""ants""}",_raw_bitbucket_api_repositories,2,
+bitbucket:BitbucketRepo:1:likyh/likyhphp,likyhphp,https://bitbucket.org/likyh/likyhphp,,,,,0,,,0,
diff --git a/backend/plugins/bitbucket/impl/impl.go b/backend/plugins/bitbucket/impl/impl.go
index c92d3adb9..12f5c051f 100644
--- a/backend/plugins/bitbucket/impl/impl.go
+++ b/backend/plugins/bitbucket/impl/impl.go
@@ -19,6 +19,8 @@ package impl
 
 import (
 	"fmt"
+	"time"
+
 	"github.com/apache/incubator-devlake/core/context"
 	"github.com/apache/incubator-devlake/core/dal"
 	"github.com/apache/incubator-devlake/core/errors"
@@ -36,11 +38,23 @@ var _ plugin.PluginTask = (*Bitbucket)(nil)
 var _ plugin.PluginApi = (*Bitbucket)(nil)
 var _ plugin.PluginModel = (*Bitbucket)(nil)
 var _ plugin.PluginMigration = (*Bitbucket)(nil)
-var _ plugin.PluginBlueprintV100 = (*Bitbucket)(nil)
 var _ plugin.CloseablePluginTask = (*Bitbucket)(nil)
+var _ plugin.PluginSource = (*Bitbucket)(nil)
 
 type Bitbucket string
 
+func (p Bitbucket) Connection() interface{} {
+	return &models.BitbucketConnection{}
+}
+
+func (p Bitbucket) Scope() interface{} {
+	return &models.BitbucketRepo{}
+}
+
+func (p Bitbucket) TransformationRule() interface{} {
+	return &models.BitbucketTransformationRule{}
+}
+
 func (p Bitbucket) Init(basicRes context.BasicRes) errors.Error {
 	api.Init(basicRes)
 	return nil
@@ -67,9 +81,6 @@ func (p Bitbucket) Description() string {
 
 func (p Bitbucket) SubTaskMetas() []plugin.SubTaskMeta {
 	return []plugin.SubTaskMeta{
-		tasks.CollectApiRepoMeta,
-		tasks.ExtractApiRepoMeta,
-
 		tasks.CollectApiPullRequestsMeta,
 		tasks.ExtractApiPullRequestsMeta,
 
@@ -79,6 +90,9 @@ func (p Bitbucket) SubTaskMetas() []plugin.SubTaskMeta {
 		tasks.CollectApiPrCommitsMeta,
 		tasks.ExtractApiPrCommitsMeta,
 
+		tasks.CollectApiCommitsMeta,
+		tasks.ExtractApiCommitsMeta,
+
 		tasks.CollectApiIssuesMeta,
 		tasks.ExtractApiIssuesMeta,
 
@@ -96,6 +110,7 @@ func (p Bitbucket) SubTaskMetas() []plugin.SubTaskMeta {
 		tasks.ConvertPullRequestsMeta,
 		tasks.ConvertPrCommentsMeta,
 		tasks.ConvertPrCommitsMeta,
+		tasks.ConvertCommitsMeta,
 		tasks.ConvertIssuesMeta,
 		tasks.ConvertIssueCommentsMeta,
 		tasks.ConvertPipelineMeta,
@@ -104,6 +119,8 @@ func (p Bitbucket) SubTaskMetas() []plugin.SubTaskMeta {
 }
 
 func (p Bitbucket) PrepareTaskData(taskCtx plugin.TaskContext, options map[string]interface{}) (interface{}, errors.Error) {
+	logger := taskCtx.GetLogger()
+	logger.Debug("%v", options)
 	op, err := tasks.DecodeAndValidateTaskOptions(options)
 	if err != nil {
 		return nil, err
@@ -122,11 +139,28 @@ func (p Bitbucket) PrepareTaskData(taskCtx plugin.TaskContext, options map[strin
 	if err != nil {
 		return nil, errors.Default.Wrap(err, "unable to get bitbucket API client instance")
 	}
+	err = EnrichOptions(taskCtx, op, apiClient.ApiClient)
+	if err != nil {
+		return nil, err
+	}
 
-	return &tasks.BitbucketTaskData{
+	var createdDateAfter time.Time
+	if op.CreatedDateAfter != "" {
+		createdDateAfter, err = errors.Convert01(time.Parse(time.RFC3339, op.CreatedDateAfter))
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "invalid value for `createdDateAfter`")
+		}
+	}
+	taskData := &tasks.BitbucketTaskData{
 		Options:   op,
 		ApiClient: apiClient,
-	}, nil
+	}
+	if !createdDateAfter.IsZero() {
+		taskData.CreatedDateAfter = &createdDateAfter
+		logger.Debug("collect data updated createdDateAfter %s", createdDateAfter)
+	}
+
+	return taskData, nil
 }
 
 func (p Bitbucket) RootPkgPath() string {
@@ -137,8 +171,8 @@ func (p Bitbucket) MigrationScripts() []plugin.MigrationScript {
 	return migrationscripts.All()
 }
 
-func (p Bitbucket) MakePipelinePlan(connectionId uint64, scope []*plugin.BlueprintScopeV100) (plugin.PipelinePlan, errors.Error) {
-	return api.MakePipelinePlan(p.SubTaskMetas(), connectionId, scope)
+func (p Bitbucket) MakeDataSourcePipelinePlanV200(connectionId uint64, scopes []*plugin.BlueprintScopeV200, syncPolicy plugin.BlueprintSyncPolicy) (pp plugin.PipelinePlan, sc []plugin.Scope, err errors.Error) {
+	return api.MakeDataSourcePipelinePlanV200(p.SubTaskMetas(), connectionId, scopes, &syncPolicy)
 }
 
 func (p Bitbucket) ApiResources() map[string]map[string]plugin.ApiResourceHandler {
@@ -155,6 +189,28 @@ func (p Bitbucket) ApiResources() map[string]map[string]plugin.ApiResourceHandle
 			"DELETE": api.DeleteConnection,
 			"GET":    api.GetConnection,
 		},
+		"connections/:connectionId/scopes/*repoId": {
+			"GET":   api.GetScope,
+			"PATCH": api.UpdateScope,
+		},
+		"connections/:connectionId/remote-scopes": {
+			"GET": api.RemoteScopes,
+		},
+		"connections/:connectionId/search-remote-scopes": {
+			"GET": api.SearchRemoteScopes,
+		},
+		"connections/:connectionId/scopes": {
+			"GET": api.GetScopeList,
+			"PUT": api.PutScope,
+		},
+		"transformation_rules": {
+			"POST": api.CreateTransformationRule,
+			"GET":  api.GetTransformationRuleList,
+		},
+		"transformation_rules/:id": {
+			"PATCH": api.UpdateTransformationRule,
+			"GET":   api.GetTransformationRule,
+		},
 	}
 }
 
@@ -166,3 +222,54 @@ func (p Bitbucket) Close(taskCtx plugin.TaskContext) errors.Error {
 	data.ApiClient.Release()
 	return nil
 }
+
+func EnrichOptions(taskCtx plugin.TaskContext,
+	op *tasks.BitbucketOptions,
+	apiClient *helper.ApiClient) errors.Error {
+	var repo models.BitbucketRepo
+	// validate the op and set name=owner/repo if this is from advanced mode or bpV100
+	err := tasks.ValidateTaskOptions(op)
+	if err != nil {
+		return err
+	}
+	logger := taskCtx.GetLogger()
+	// for advanced mode or others which we only have name, for bp v200, we have githubId
+	err = taskCtx.GetDal().First(&repo, dal.Where(
+		"connection_id = ? AND bitbucket_id = ?",
+		op.ConnectionId, op.FullName))
+	if err == nil {
+		if op.TransformationRuleId == 0 {
+			op.TransformationRuleId = repo.TransformationRuleId
+		}
+	} else {
+		if taskCtx.GetDal().IsErrorNotFound(err) && op.FullName != "" {
+			var repo *tasks.BitbucketApiRepo
+			repo, err = tasks.GetApiRepo(op, apiClient)
+			if err != nil {
+				return err
+			}
+			logger.Debug(fmt.Sprintf("Current repo: %s", repo.FullName))
+			scope := tasks.ConvertApiRepoToScope(repo, op.ConnectionId)
+			err = taskCtx.GetDal().CreateIfNotExist(scope)
+			if err != nil {
+				return err
+			}
+		} else {
+			return errors.Default.Wrap(err, fmt.Sprintf("fail to find repo %s", op.FullName))
+		}
+	}
+	// Set GithubTransformationRule if it's nil, this has lower priority
+	if op.BitbucketTransformationRule == nil && op.TransformationRuleId != 0 {
+		var transformationRule models.BitbucketTransformationRule
+		db := taskCtx.GetDal()
+		err = db.First(&transformationRule, dal.Where("id = ?", repo.TransformationRuleId))
+		if err != nil && !db.IsErrorNotFound(err) {
+			return errors.BadInput.Wrap(err, "fail to get transformationRule")
+		}
+		op.BitbucketTransformationRule = &transformationRule
+	}
+	if op.BitbucketTransformationRule == nil && op.TransformationRuleId == 0 {
+		op.BitbucketTransformationRule = new(models.BitbucketTransformationRule)
+	}
+	return err
+}
diff --git a/backend/plugins/bitbucket/models/connection.go b/backend/plugins/bitbucket/models/connection.go
index 9ddb865e0..fb440aea2 100644
--- a/backend/plugins/bitbucket/models/connection.go
+++ b/backend/plugins/bitbucket/models/connection.go
@@ -21,13 +21,6 @@ import (
 	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
 )
 
-type TransformationRules struct {
-	IssueStatusTODO       []string `mapstructure:"issueStatusTodo" json:"issueStatusTodo"`
-	IssueStatusINPROGRESS []string `mapstructure:"issueStatusInProgress" json:"issueStatusInProgress"`
-	IssueStatusDONE       []string `mapstructure:"issueStatusDone" json:"issueStatusDone"`
-	IssueStatusOTHER      []string `mapstructure:"issueStatusOther" json:"issueStatusOther"`
-}
-
 // BitbucketConn holds the essential information to connect to the Bitbucket API
 type BitbucketConn struct {
 	helper.RestConnection `mapstructure:",squash"`
diff --git a/backend/plugins/bitbucket/models/issue.go b/backend/plugins/bitbucket/models/issue.go
index 8de5a6997..4032d3505 100644
--- a/backend/plugins/bitbucket/models/issue.go
+++ b/backend/plugins/bitbucket/models/issue.go
@@ -24,10 +24,11 @@ import (
 
 type BitbucketIssue struct {
 	ConnectionId       uint64 `gorm:"primaryKey"`
+	RepoId             string `gorm:"primaryKey;type:varchar(255)"`
 	BitbucketId        int    `gorm:"primaryKey"`
-	RepoId             string `gorm:"index;type:varchar(255)"`
 	Number             int    `gorm:"index;comment:Used in API requests ex. api/issues/<THIS_NUMBER>"`
 	State              string `gorm:"type:varchar(255)"`
+	StdState           string `gorm:"type:varchar(255)"`
 	Title              string `gorm:"type:varchar(255)"`
 	Body               string
 	Priority           string `gorm:"type:varchar(255)"`
diff --git a/backend/plugins/bitbucket/models/migrationscripts/20230206_add_scope_and_transformation.go b/backend/plugins/bitbucket/models/migrationscripts/20230206_add_scope_and_transformation.go
new file mode 100644
index 000000000..3a94a9b3d
--- /dev/null
+++ b/backend/plugins/bitbucket/models/migrationscripts/20230206_add_scope_and_transformation.go
@@ -0,0 +1,205 @@
+/*
+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/core/context"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/models/migrationscripts/archived"
+	"github.com/apache/incubator-devlake/helpers/migrationhelper"
+	archivedModel "github.com/apache/incubator-devlake/plugins/bitbucket/models/migrationscripts/archived"
+	"time"
+)
+
+type BitbucketRepo20230206 struct {
+	TransformationRuleId uint64 `json:"transformationRuleId,omitempty" mapstructure:"transformationRuleId,omitempty"`
+	CloneUrl             string `json:"cloneUrl" gorm:"type:varchar(255)" mapstructure:"cloneUrl,omitempty"`
+	Owner                string `json:"owner" mapstructure:"owner,omitempty"`
+}
+
+func (BitbucketRepo20230206) TableName() string {
+	return "_tool_bitbucket_repos"
+}
+
+type BitbucketIssue20230206 struct {
+	ConnectionId       uint64 `gorm:"primaryKey"`
+	RepoId             string `gorm:"primaryKey;type:varchar(255)"`
+	BitbucketId        int    `gorm:"primaryKey"`
+	Number             int    `gorm:"index;comment:Used in API requests ex. api/issues/<THIS_NUMBER>"`
+	State              string `gorm:"type:varchar(255)"`
+	StdState           string `gorm:"type:varchar(255)"`
+	Title              string `gorm:"type:varchar(255)"`
+	Body               string
+	Priority           string `gorm:"type:varchar(255)"`
+	Type               string `gorm:"type:varchar(100)"`
+	AuthorId           string `gorm:"type:varchar(255)"`
+	AuthorName         string `gorm:"type:varchar(255)"`
+	AssigneeId         string `gorm:"type:varchar(255)"`
+	AssigneeName       string `gorm:"type:varchar(255)"`
+	MilestoneId        int    `gorm:"index"`
+	LeadTimeMinutes    uint
+	Url                string `gorm:"type:varchar(255)"`
+	ClosedAt           *time.Time
+	BitbucketCreatedAt time.Time
+	BitbucketUpdatedAt time.Time `gorm:"index"`
+	Severity           string    `gorm:"type:varchar(255)"`
+	Component          string    `gorm:"type:varchar(255)"`
+	archived.NoPKModel
+}
+
+func (BitbucketIssue20230206) TableName() string {
+	return "_tool_bitbucket_issues"
+}
+
+type BitbucketPullRequest20230206 struct {
+	ConnectionId       uint64 `gorm:"primaryKey"`
+	RepoId             string `gorm:"primaryKey;type:varchar(255)"`
+	BitbucketId        int    `gorm:"primaryKey"`
+	Number             int    `gorm:"index"` // This number is used in GET requests to the API associated to reviewers / comments / etc.
+	BaseRepoId         string
+	HeadRepoId         string
+	State              string `gorm:"type:varchar(255)"`
+	Title              string
+	Description        string
+	BitbucketCreatedAt time.Time
+	BitbucketUpdatedAt time.Time `gorm:"index"`
+	ClosedAt           *time.Time
+	CommentCount       int
+	Commits            int
+	MergedAt           *time.Time
+	Body               string
+	Type               string `gorm:"type:varchar(255)"`
+	Component          string `gorm:"type:varchar(255)"`
+	MergeCommitSha     string `gorm:"type:varchar(40)"`
+	HeadRef            string `gorm:"type:varchar(255)"`
+	BaseRef            string `gorm:"type:varchar(255)"`
+	BaseCommitSha      string `gorm:"type:varchar(255)"`
+	HeadCommitSha      string `gorm:"type:varchar(255)"`
+	Url                string `gorm:"type:varchar(255)"`
+	AuthorName         string `gorm:"type:varchar(255)"`
+	AuthorId           string `gorm:"type:varchar(255)"`
+	archived.NoPKModel
+}
+
+func (BitbucketPullRequest20230206) TableName() string {
+	return "_tool_bitbucket_pull_requests"
+}
+
+type addScope20230206 struct{}
+
+func (script *addScope20230206) Up(basicRes context.BasicRes) errors.Error {
+	db := basicRes.GetDal()
+	err := db.RenameColumn("_tool_bitbucket_repos", "owner_id", "owner")
+	if err != nil {
+		return err
+	}
+
+	// add `RepoId` as primary key
+	err = migrationhelper.TransformTable(
+		basicRes,
+		script,
+		"_tool_bitbucket_issues",
+		func(s *archivedModel.BitbucketIssue) (*BitbucketIssue20230206, errors.Error) {
+			dst := &BitbucketIssue20230206{
+				ConnectionId:       s.ConnectionId,
+				RepoId:             s.RepoId,
+				BitbucketId:        s.BitbucketId,
+				Number:             s.Number,
+				State:              ``,
+				StdState:           s.State,
+				Title:              s.Title,
+				Body:               s.Body,
+				Priority:           s.Priority,
+				Type:               s.Type,
+				AuthorId:           s.AuthorId,
+				AuthorName:         s.AuthorName,
+				AssigneeId:         s.AssigneeId,
+				AssigneeName:       s.AssigneeName,
+				MilestoneId:        s.MilestoneId,
+				LeadTimeMinutes:    s.LeadTimeMinutes,
+				Url:                s.Url,
+				ClosedAt:           s.ClosedAt,
+				BitbucketCreatedAt: s.BitbucketCreatedAt,
+				BitbucketUpdatedAt: s.BitbucketUpdatedAt,
+				Severity:           s.Severity,
+				Component:          s.Component,
+				NoPKModel:          s.NoPKModel,
+			}
+			return dst, nil
+		},
+	)
+	if err != nil {
+		return err
+	}
+
+	// add `RepoId` as primary key
+	err = migrationhelper.TransformTable(
+		basicRes,
+		script,
+		"_tool_bitbucket_pull_requests",
+		func(s *archivedModel.BitbucketPullRequest) (*BitbucketPullRequest20230206, errors.Error) {
+			dst := &BitbucketPullRequest20230206{
+				ConnectionId:       s.ConnectionId,
+				RepoId:             s.RepoId,
+				BitbucketId:        s.BitbucketId,
+				Number:             s.Number,
+				BaseRepoId:         s.BaseRepoId,
+				HeadRepoId:         s.HeadRepoId,
+				State:              s.State,
+				Title:              s.Title,
+				Description:        s.Description,
+				BitbucketCreatedAt: s.BitbucketCreatedAt,
+				BitbucketUpdatedAt: s.BitbucketUpdatedAt,
+				ClosedAt:           s.ClosedAt,
+				CommentCount:       s.CommentCount,
+				Commits:            s.Commits,
+				MergedAt:           s.MergedAt,
+				Body:               s.Body,
+				Type:               s.Type,
+				Component:          s.Component,
+				MergeCommitSha:     s.MergeCommitSha,
+				HeadRef:            s.HeadRef,
+				BaseRef:            s.BaseRef,
+				BaseCommitSha:      s.BaseCommitSha,
+				HeadCommitSha:      s.HeadCommitSha,
+				Url:                s.Url,
+				AuthorName:         s.AuthorName,
+				AuthorId:           s.AuthorId,
+				NoPKModel:          s.NoPKModel,
+			}
+			return dst, nil
+		},
+	)
+	if err != nil {
+		return err
+	}
+
+	return migrationhelper.AutoMigrateTables(
+		basicRes,
+		&BitbucketRepo20230206{},
+		&archivedModel.BitbucketTransformationRule{},
+	)
+}
+
+func (*addScope20230206) Version() uint64 {
+	return 20230206000008
+}
+
+func (*addScope20230206) Name() string {
+	return "add scope and table _tool_bitbucket_transformation_rules"
+}
diff --git a/backend/plugins/bitbucket/models/migrationscripts/archived/transformation_rule.go b/backend/plugins/bitbucket/models/migrationscripts/archived/transformation_rule.go
new file mode 100644
index 000000000..2abd7743f
--- /dev/null
+++ b/backend/plugins/bitbucket/models/migrationscripts/archived/transformation_rule.go
@@ -0,0 +1,41 @@
+/*
+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
+
+import (
+	"github.com/apache/incubator-devlake/core/models/migrationscripts/archived"
+	"gorm.io/datatypes"
+)
+
+type BitbucketTransformationRule struct {
+	archived.Model
+	Name              string            `gorm:"type:varchar(255);index:idx_name_github,unique"`
+	DeploymentPattern string            `gorm:"type:varchar(255)"`
+	ProductionPattern string            `gorm:"type:varchar(255)"`
+	Refdiff           datatypes.JSONMap `format:"json"`
+
+	// a string array, split by `,`.
+	IssueStatusTodo       string `gorm:"type:varchar(255)"`
+	IssueStatusInProgress string `gorm:"type:varchar(255)"`
+	IssueStatusDone       string `gorm:"type:varchar(255)"`
+	IssueStatusOther      string `gorm:"type:varchar(255)"`
+}
+
+func (BitbucketTransformationRule) TableName() string {
+	return "_tool_bitbucket_transformation_rules"
+}
diff --git a/backend/plugins/bitbucket/models/migrationscripts/register.go b/backend/plugins/bitbucket/models/migrationscripts/register.go
index fac04e640..183d1b268 100644
--- a/backend/plugins/bitbucket/models/migrationscripts/register.go
+++ b/backend/plugins/bitbucket/models/migrationscripts/register.go
@@ -29,5 +29,6 @@ func All() []plugin.MigrationScript {
 		new(addPrCommits20221008),
 		new(addDeployment20221013),
 		new(addRepoIdAndCommitShaField20221014),
+		new(addScope20230206),
 	}
 }
diff --git a/backend/plugins/bitbucket/models/pr.go b/backend/plugins/bitbucket/models/pr.go
index e061c2e2b..b256b54b4 100644
--- a/backend/plugins/bitbucket/models/pr.go
+++ b/backend/plugins/bitbucket/models/pr.go
@@ -24,8 +24,8 @@ import (
 
 type BitbucketPullRequest struct {
 	ConnectionId       uint64 `gorm:"primaryKey"`
+	RepoId             string `gorm:"primaryKey;type:varchar(255)"`
 	BitbucketId        int    `gorm:"primaryKey"`
-	RepoId             string `gorm:"index;type:varchar(255)"`
 	Number             int    `gorm:"index"` // This number is used in GET requests to the API associated to reviewers / comments / etc.
 	BaseRepoId         string
 	HeadRepoId         string
diff --git a/backend/plugins/bitbucket/models/repo.go b/backend/plugins/bitbucket/models/repo.go
index 4a10d8f30..249651a7b 100644
--- a/backend/plugins/bitbucket/models/repo.go
+++ b/backend/plugins/bitbucket/models/repo.go
@@ -23,16 +23,18 @@ import (
 )
 
 type BitbucketRepo struct {
-	ConnectionId uint64 `gorm:"primaryKey"`
-	BitbucketId  string `gorm:"primaryKey;type:varchar(255)"`
-	Name         string `gorm:"type:varchar(255)"`
-	HTMLUrl      string `gorm:"type:varchar(255)"`
-	Description  string
-	OwnerId      string     `json:"ownerId"`
-	Language     string     `json:"language" gorm:"type:varchar(255)"`
-	CreatedDate  time.Time  `json:"createdDate"`
-	UpdatedDate  *time.Time `json:"updatedDate"`
-	common.NoPKModel
+	ConnectionId         uint64     `json:"connectionId" gorm:"primaryKey" mapstructure:"connectionId,omitempty"`
+	BitbucketId          string     `json:"bitbucketId" gorm:"primaryKey;type:varchar(255)" mapstructure:"bitbucketId"`
+	Name                 string     `json:"name" gorm:"type:varchar(255)" mapstructure:"name,omitempty"`
+	HTMLUrl              string     `json:"HTMLUrl" gorm:"type:varchar(255)" mapstructure:"HTMLUrl,omitempty"`
+	Description          string     `json:"description" mapstructure:"description,omitempty"`
+	TransformationRuleId uint64     `json:"transformationRuleId,omitempty" mapstructure:"transformationRuleId,omitempty"`
+	Owner                string     `json:"owner" mapstructure:"owner,omitempty"`
+	Language             string     `json:"language" gorm:"type:varchar(255)" mapstructure:"language,omitempty"`
+	CloneUrl             string     `json:"cloneUrl" gorm:"type:varchar(255)" mapstructure:"cloneUrl,omitempty"`
+	CreatedDate          *time.Time `json:"createdDate" mapstructure:"-"`
+	UpdatedDate          *time.Time `json:"updatedDate" mapstructure:"-"`
+	common.NoPKModel     `json:"-" mapstructure:"-"`
 }
 
 func (BitbucketRepo) TableName() string {
diff --git a/backend/plugins/bitbucket/models/transformation_rule.go b/backend/plugins/bitbucket/models/transformation_rule.go
new file mode 100644
index 000000000..26a622239
--- /dev/null
+++ b/backend/plugins/bitbucket/models/transformation_rule.go
@@ -0,0 +1,41 @@
+/*
+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 models
+
+import (
+	"github.com/apache/incubator-devlake/core/models/common"
+	"gorm.io/datatypes"
+)
+
+type BitbucketTransformationRule struct {
+	common.Model      `mapstructure:"-"`
+	Name              string            `mapstructure:"name" json:"name" gorm:"type:varchar(255);index:idx_name_github,unique" validate:"required"`
+	DeploymentPattern string            `mapstructure:"deploymentPattern,omitempty" json:"deploymentPattern" gorm:"type:varchar(255)"`
+	ProductionPattern string            `mapstructure:"productionPattern,omitempty" json:"productionPattern" gorm:"type:varchar(255)"`
+	Refdiff           datatypes.JSONMap `mapstructure:"refdiff,omitempty" json:"refdiff" swaggertype:"object" format:"json"`
+
+	// a string array, split by `,`.
+	IssueStatusTodo       string `mapstructure:"issueStatusTodo,omitempty" json:"issueStatusTodo" gorm:"type:varchar(255)"`
+	IssueStatusInProgress string `mapstructure:"issueStatusInProgress,omitempty" json:"issueStatusInProgress" gorm:"type:varchar(255)"`
+	IssueStatusDone       string `mapstructure:"issueStatusDone,omitempty" json:"issueStatusDone" gorm:"type:varchar(255)"`
+	IssueStatusOther      string `mapstructure:"issueStatusOther,omitempty" json:"issueStatusOther" gorm:"type:varchar(255)"`
+}
+
+func (BitbucketTransformationRule) TableName() string {
+	return "_tool_bitbucket_transformation_rules"
+}
diff --git a/backend/plugins/bitbucket/tasks/account_convertor.go b/backend/plugins/bitbucket/tasks/account_convertor.go
index f137d3ba8..716afc5e3 100644
--- a/backend/plugins/bitbucket/tasks/account_convertor.go
+++ b/backend/plugins/bitbucket/tasks/account_convertor.go
@@ -41,8 +41,8 @@ var ConvertAccountsMeta = plugin.SubTaskMeta{
 }
 
 func ConvertAccounts(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ACCOUNT_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
 
 	cursor, err := db.Cursor(dal.From(&bitbucketModels.BitbucketAccount{}))
 	if err != nil {
@@ -53,17 +53,9 @@ func ConvertAccounts(taskCtx plugin.SubTaskContext) errors.Error {
 	accountIdGen := didgen.NewDomainIdGenerator(&bitbucketModels.BitbucketAccount{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(bitbucketModels.BitbucketAccount{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_ACCOUNT_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(bitbucketModels.BitbucketAccount{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			bitbucketUser := inputRow.(*bitbucketModels.BitbucketAccount)
 			domainUser := &crossdomain.Account{
diff --git a/backend/plugins/bitbucket/tasks/account_extractor.go b/backend/plugins/bitbucket/tasks/account_extractor.go
index 0d25fafa5..49b85fbc2 100644
--- a/backend/plugins/bitbucket/tasks/account_extractor.go
+++ b/backend/plugins/bitbucket/tasks/account_extractor.go
@@ -24,18 +24,19 @@ import (
 )
 
 type BitbucketAccountResponse struct {
+	Uuid          string    `json:"uuid"`
 	UserName      string    `json:"username"`
 	DisplayName   string    `json:"display_name"`
 	AccountId     string    `json:"account_id"`
 	AccountStatus string    `json:"account_status"`
 	CreateOn      time.Time `json:"create_on"`
 	Links         struct {
-		Self       struct{ Href string } `json:"self"`
-		Html       struct{ Href string } `json:"html"`
-		Avatar     struct{ Href string } `json:"avatar"`
-		Followers  struct{ Href string } `json:"followers"`
-		Following  struct{ Href string } `json:"following"`
-		Repository struct{ Href string } `json:"repository"`
+		//Self       struct{ Href string } `json:"self"`
+		Html   struct{ Href string } `json:"html"`
+		Avatar struct{ Href string } `json:"avatar"`
+		//Followers  struct{ Href string } `json:"followers"`
+		//Following  struct{ Href string } `json:"following"`
+		//Repository struct{ Href string } `json:"repository"`
 	}
 }
 
@@ -45,6 +46,7 @@ func convertAccount(res *BitbucketAccountResponse, connId uint64) (*models.Bitbu
 		UserName:      res.UserName,
 		DisplayName:   res.DisplayName,
 		AccountId:     res.AccountId,
+		Uuid:          res.Uuid,
 		AccountStatus: res.AccountStatus,
 		AvatarUrl:     res.Links.Avatar.Href,
 		HtmlUrl:       res.Links.Html.Href,
diff --git a/backend/plugins/bitbucket/tasks/api_common.go b/backend/plugins/bitbucket/tasks/api_common.go
index 4e456ac46..9774bd38d 100644
--- a/backend/plugins/bitbucket/tasks/api_common.go
+++ b/backend/plugins/bitbucket/tasks/api_common.go
@@ -32,8 +32,7 @@ import (
 
 type BitbucketApiParams struct {
 	ConnectionId uint64
-	Owner        string
-	Repo         string
+	FullName     string
 }
 
 type BitbucketInput struct {
@@ -54,8 +53,7 @@ func CreateRawDataSubTaskArgs(taskCtx plugin.SubTaskContext, Table string) (*api
 		Ctx: taskCtx,
 		Params: BitbucketApiParams{
 			ConnectionId: data.Options.ConnectionId,
-			Owner:        data.Options.Owner,
-			Repo:         data.Options.Repo,
+			FullName:     data.Options.FullName,
 		},
 		Table: Table,
 	}
@@ -88,15 +86,41 @@ func GetQuery(reqData *api.RequestData) (url.Values, errors.Error) {
 	return query, nil
 }
 
-func GetQueryForNext(reqData *api.RequestData) (url.Values, errors.Error) {
-	query := url.Values{}
-	query.Set("state", "all")
-	query.Set("pagelen", fmt.Sprintf("%v", reqData.Pager.Size))
+// GetQueryCreatedAndUpdated is a common GeyQuery for timeFilter and incremental
+func GetQueryCreatedAndUpdated(fields string, collectorWithState *api.ApiCollectorStateManager) func(reqData *api.RequestData) (url.Values, errors.Error) {
+	return func(reqData *api.RequestData) (url.Values, errors.Error) {
+		query, err := GetQuery(reqData)
+		if err != nil {
+			return nil, err
+		}
+		query.Set("fields", fields)
+		query.Set("sort", "created_on")
+		if collectorWithState.IsIncremental() && collectorWithState.CreatedDateAfter != nil {
+			latestSuccessStart := collectorWithState.LatestState.LatestSuccessStart.Format("2006-01-02")
+			createdDateAfter := collectorWithState.CreatedDateAfter.Format("2006-01-02")
+			query.Set("q", fmt.Sprintf("updated_on>=%s AND created_on>=%s", latestSuccessStart, createdDateAfter))
+		} else if collectorWithState.IsIncremental() {
+			latestSuccessStart := collectorWithState.LatestState.LatestSuccessStart.Format("2006-01-02")
+			query.Set("q", fmt.Sprintf("updated_on>=%s", latestSuccessStart))
+		} else if collectorWithState.CreatedDateAfter != nil {
+			createdDateAfter := collectorWithState.CreatedDateAfter.Format("2006-01-02")
+			query.Set("q", fmt.Sprintf("created_on>=%s", createdDateAfter))
+		}
+
+		return query, nil
+	}
+}
+
+func GetQueryFields(fields string) func(reqData *api.RequestData) (url.Values, errors.Error) {
+	return func(reqData *api.RequestData) (url.Values, errors.Error) {
+		query, err := GetQuery(reqData)
+		if err != nil {
+			return nil, err
+		}
+		query.Set("fields", fields)
 
-	if reqData.CustomData != nil {
-		query.Set("page", reqData.CustomData.(string))
+		return query, nil
 	}
-	return query, nil
 }
 
 func GetNextPageCustomData(_ *api.RequestData, prevPageResponse *http.Response) (interface{}, errors.Error) {
@@ -142,7 +166,7 @@ func GetRawMessageFromResponse(res *http.Response) ([]json.RawMessage, errors.Er
 	return rawMessages.Values, nil
 }
 
-func GetPullRequestsIterator(taskCtx plugin.SubTaskContext) (*api.DalCursorIterator, errors.Error) {
+func GetPullRequestsIterator(taskCtx plugin.SubTaskContext, collectorWithState *api.ApiCollectorStateManager) (*api.DalCursorIterator, errors.Error) {
 	db := taskCtx.GetDal()
 	data := taskCtx.GetData().(*BitbucketTaskData)
 	clauses := []dal.Clause{
@@ -150,9 +174,15 @@ func GetPullRequestsIterator(taskCtx plugin.SubTaskContext) (*api.DalCursorItera
 		dal.From("_tool_bitbucket_pull_requests bpr"),
 		dal.Where(
 			`bpr.repo_id = ? and bpr.connection_id = ?`,
-			fmt.Sprintf("%s/%s", data.Options.Owner, data.Options.Repo), data.Options.ConnectionId,
+			data.Options.FullName, data.Options.ConnectionId,
 		),
 	}
+	if collectorWithState.CreatedDateAfter != nil {
+		clauses = append(clauses, dal.Where("bitbucket_created_at > ?", *collectorWithState.CreatedDateAfter))
+	}
+	if collectorWithState.IsIncremental() {
+		clauses = append(clauses, dal.Where("bitbucket_updated_at > ?", *collectorWithState.LatestState.LatestSuccessStart))
+	}
 	// construct the input iterator
 	cursor, err := db.Cursor(clauses...)
 	if err != nil {
@@ -162,7 +192,7 @@ func GetPullRequestsIterator(taskCtx plugin.SubTaskContext) (*api.DalCursorItera
 	return api.NewDalCursorIterator(db, cursor, reflect.TypeOf(BitbucketInput{}))
 }
 
-func GetIssuesIterator(taskCtx plugin.SubTaskContext) (*api.DalCursorIterator, errors.Error) {
+func GetIssuesIterator(taskCtx plugin.SubTaskContext, collectorWithState *api.ApiCollectorStateManager) (*api.DalCursorIterator, errors.Error) {
 	db := taskCtx.GetDal()
 	data := taskCtx.GetData().(*BitbucketTaskData)
 	clauses := []dal.Clause{
@@ -170,9 +200,15 @@ func GetIssuesIterator(taskCtx plugin.SubTaskContext) (*api.DalCursorIterator, e
 		dal.From("_tool_bitbucket_issues bpr"),
 		dal.Where(
 			`bpr.repo_id = ? and bpr.connection_id = ?`,
-			fmt.Sprintf("%s/%s", data.Options.Owner, data.Options.Repo), data.Options.ConnectionId,
+			data.Options.FullName, data.Options.ConnectionId,
 		),
 	}
+	if collectorWithState.CreatedDateAfter != nil {
+		clauses = append(clauses, dal.Where("bitbucket_created_at > ?", *collectorWithState.CreatedDateAfter))
+	}
+	if collectorWithState.IsIncremental() {
+		clauses = append(clauses, dal.Where("bitbucket_updated_at > ?", *collectorWithState.LatestState.LatestSuccessStart))
+	}
 	// construct the input iterator
 	cursor, err := db.Cursor(clauses...)
 	if err != nil {
diff --git a/backend/plugins/bitbucket/tasks/commit_collector.go b/backend/plugins/bitbucket/tasks/commit_collector.go
index 63bb9e247..d7fd4f0ec 100644
--- a/backend/plugins/bitbucket/tasks/commit_collector.go
+++ b/backend/plugins/bitbucket/tasks/commit_collector.go
@@ -42,7 +42,7 @@ func CollectApiCommits(taskCtx plugin.SubTaskContext) errors.Error {
 		ApiClient:          data.ApiClient,
 		PageSize:           100,
 		Incremental:        false,
-		UrlTemplate:        "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/commits",
+		UrlTemplate:        "repositories/{{ .Params.FullName }}/commits",
 		Query:              GetQuery,
 		GetTotalPages:      GetTotalPagesFromResponse,
 		ResponseParser:     GetRawMessageFromResponse,
diff --git a/backend/plugins/bitbucket/tasks/commit_convertor.go b/backend/plugins/bitbucket/tasks/commit_convertor.go
index 489dadcd9..98e57a0ee 100644
--- a/backend/plugins/bitbucket/tasks/commit_convertor.go
+++ b/backend/plugins/bitbucket/tasks/commit_convertor.go
@@ -38,17 +38,17 @@ var ConvertCommitsMeta = plugin.SubTaskMeta{
 }
 
 func ConvertCommits(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	cursor, err := db.Cursor(
-		dal.From("_tool_bitbucket_commits gc"),
-		dal.Join(`left join _tool_bitbucket_repo_commits grc on (
-			grc.commit_sha = gc.sha
+		dal.From("_tool_bitbucket_commits c"),
+		dal.Join(`left join _tool_bitbucket_repo_commits rc on (
+			rc.commit_sha = c.sha
 		)`),
-		dal.Select("gc.*"),
-		dal.Where("grc.repo_id = ? AND grc.connection_id = ?", repoId, data.Options.ConnectionId),
+		dal.Select("c.*"),
+		dal.Where("rc.repo_id = ? AND rc.connection_id = ?", repoId, data.Options.ConnectionId),
 	)
 	if err != nil {
 		return err
@@ -59,17 +59,9 @@ func ConvertCommits(taskCtx plugin.SubTaskContext) errors.Error {
 	domainRepoId := repoDidGen.Generate(data.Options.ConnectionId, repoId)
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_COMMIT_TABLE,
-		},
-		InputRowType: reflect.TypeOf(models.BitbucketCommit{}),
-		Input:        cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		InputRowType:       reflect.TypeOf(models.BitbucketCommit{}),
+		Input:              cursor,
 
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			bitbucketCommit := inputRow.(*models.BitbucketCommit)
diff --git a/backend/plugins/bitbucket/tasks/commit_extractor.go b/backend/plugins/bitbucket/tasks/commit_extractor.go
index 0cf9d24e6..00d2d48fd 100644
--- a/backend/plugins/bitbucket/tasks/commit_extractor.go
+++ b/backend/plugins/bitbucket/tasks/commit_extractor.go
@@ -36,7 +36,7 @@ var ExtractApiCommitsMeta = plugin.SubTaskMeta{
 }
 
 type CommitsResponse struct {
-	Type string    `json:"type"`
+	//Type string    `json:"type"`
 	Sha  string    `json:"hash"`
 	Date time.Time `json:"date"`
 	//Author *models.BitbucketAccount
@@ -59,25 +59,10 @@ type CommitsResponse struct {
 }
 
 func ExtractApiCommits(taskCtx plugin.SubTaskContext) errors.Error {
-	data := taskCtx.GetData().(*BitbucketTaskData)
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_TABLE)
 
 	extractor, err := api.NewApiExtractor(api.ApiExtractorArgs{
-		RawDataSubTaskArgs: api.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 JiraCommits by Board
-			*/
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			/*
-				Table store raw data
-			*/
-			Table: RAW_COMMIT_TABLE,
-		},
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Extract: func(row *api.RawData) ([]interface{}, errors.Error) {
 			commit := &CommitsResponse{}
 			err := errors.Convert(json.Unmarshal(row.Data, commit))
@@ -105,7 +90,7 @@ func ExtractApiCommits(taskCtx plugin.SubTaskContext) errors.Error {
 
 			bitbucketRepoCommit := &models.BitbucketRepoCommit{
 				ConnectionId: data.Options.ConnectionId,
-				RepoId:       data.Repo.BitbucketId,
+				RepoId:       data.Options.FullName,
 				CommitSha:    commit.Sha,
 			}
 
diff --git a/backend/plugins/bitbucket/tasks/deployment_collector.go b/backend/plugins/bitbucket/tasks/deployment_collector.go
index eb548c023..4e7ed014f 100644
--- a/backend/plugins/bitbucket/tasks/deployment_collector.go
+++ b/backend/plugins/bitbucket/tasks/deployment_collector.go
@@ -41,12 +41,14 @@ func CollectApiDeployments(taskCtx plugin.SubTaskContext) errors.Error {
 		ApiClient:          data.ApiClient,
 		PageSize:           50,
 		Incremental:        false,
-		UrlTemplate:        "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/deployments/",
-		Query:              GetQuery,
-		ResponseParser:     GetRawMessageFromResponse,
-		GetTotalPages:      GetTotalPagesFromResponse,
+		UrlTemplate:        "repositories/{{ .Params.FullName }}/deployments/",
+		Query: GetQueryFields(`values.type,values.uuid,` +
+			`values.release.pipeline,values.release.key,values.release.name,values.release.url,values.release.created_on,` +
+			`values.release.commit.hash,values.release.commit.links.html,` +
+			`values.state.name,values.state.url,values.state.started_on,values.state.completed_on,values.last_update_time`),
+		ResponseParser: GetRawMessageFromResponse,
+		GetTotalPages:  GetTotalPagesFromResponse,
 	})
-
 	if err != nil {
 		return err
 	}
diff --git a/backend/plugins/bitbucket/tasks/deployment_convertor.go b/backend/plugins/bitbucket/tasks/deployment_convertor.go
index 1dff84c48..2b3c410b0 100644
--- a/backend/plugins/bitbucket/tasks/deployment_convertor.go
+++ b/backend/plugins/bitbucket/tasks/deployment_convertor.go
@@ -39,8 +39,8 @@ var ConvertDeploymentMeta = plugin.SubTaskMeta{
 }
 
 func ConvertDeployments(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_DEPLOYMENT_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
 
 	cursor, err := db.Cursor(dal.From(models.BitbucketDeployment{}))
 	if err != nil {
@@ -51,17 +51,9 @@ func ConvertDeployments(taskCtx plugin.SubTaskContext) errors.Error {
 	pipelineIdGen := didgen.NewDomainIdGenerator(&models.BitbucketDeployment{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(models.BitbucketDeployment{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_DEPLOYMENT_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(models.BitbucketDeployment{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			bitbucketDeployment := inputRow.(*models.BitbucketDeployment)
 
diff --git a/backend/plugins/bitbucket/tasks/deployment_extractor.go b/backend/plugins/bitbucket/tasks/deployment_extractor.go
index 008a19af3..1b5fa12ea 100644
--- a/backend/plugins/bitbucket/tasks/deployment_extractor.go
+++ b/backend/plugins/bitbucket/tasks/deployment_extractor.go
@@ -27,19 +27,18 @@ import (
 )
 
 type bitbucketApiDeploymentsResponse struct {
-	Type   string `json:"type"`
-	Number int    `json:"number"`
-	UUID   string `json:"uuid"`
-	Key    string `json:"key"`
-	Step   struct {
-		UUID string `json:"uuid"`
-	} `json:"step"`
-	Environment struct {
-		UUID string `json:"uuid"`
-	} `json:"environment"`
+	Type string `json:"type"`
+	UUID string `json:"uuid"`
+	//Key  string `json:"key"`
+	//Step struct {
+	//	UUID string `json:"uuid"`
+	//} `json:"step"`
+	//Environment struct {
+	//	UUID string `json:"uuid"`
+	//} `json:"environment"`
 	Release struct {
-		Type     string `json:"type"`
-		UUID     string `json:"uuid"`
+		//Type     string `json:"type"`
+		//UUID     string `json:"uuid"`
 		Pipeline struct {
 			UUID string `json:"uuid"`
 			Type string `json:"type"`
@@ -48,12 +47,9 @@ type bitbucketApiDeploymentsResponse struct {
 		Name   string `json:"name"`
 		URL    string `json:"url"`
 		Commit struct {
-			Type  string `json:"type"`
+			//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"`
@@ -62,18 +58,18 @@ type bitbucketApiDeploymentsResponse struct {
 		CreatedOn *time.Time `json:"created_on"`
 	} `json:"release"`
 	State struct {
-		Type   string `json:"type"`
-		Name   string `json:"name"`
-		URL    string `json:"url"`
-		Status struct {
-			Type string `json:"type"`
-			Name string `json:"name"`
-		} `json:"status"`
+		//Type   string `json:"type"`
+		Name string `json:"name"`
+		URL  string `json:"url"`
+		//Status struct {
+		//	Type string `json:"type"`
+		//	Name string `json:"name"`
+		//} `json:"status"`
 		StartedOn   *time.Time `json:"started_on"`
 		CompletedOn *time.Time `json:"completed_on"`
 	} `json:"state"`
 	LastUpdateTime *time.Time `json:"last_update_time"`
-	Version        int        `json:"version"`
+	//Version        int        `json:"version"`
 }
 
 var ExtractApiDeploymentsMeta = plugin.SubTaskMeta{
diff --git a/backend/plugins/bitbucket/tasks/issue_collector.go b/backend/plugins/bitbucket/tasks/issue_collector.go
index a7e8b1749..6294f5c04 100644
--- a/backend/plugins/bitbucket/tasks/issue_collector.go
+++ b/backend/plugins/bitbucket/tasks/issue_collector.go
@@ -35,22 +35,29 @@ var CollectApiIssuesMeta = plugin.SubTaskMeta{
 
 func CollectApiIssues(taskCtx plugin.SubTaskContext) errors.Error {
 	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_TABLE)
+	collectorWithState, err := helper.NewApiCollectorWithState(*rawDataSubTaskArgs, data.CreatedDateAfter)
+	if err != nil {
+		return err
+	}
 
-	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
-		RawDataSubTaskArgs: *rawDataSubTaskArgs,
-		ApiClient:          data.ApiClient,
-		PageSize:           100,
-		Incremental:        false,
-		UrlTemplate:        "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/issues",
-		Query:              GetQuery,
-		GetTotalPages:      GetTotalPagesFromResponse,
-		ResponseParser:     GetRawMessageFromResponse,
-		AfterResponse:      ignoreHTTPStatus404,
+	err = collectorWithState.InitCollector(helper.ApiCollectorArgs{
+		ApiClient:   data.ApiClient,
+		PageSize:    100,
+		Incremental: collectorWithState.IsIncremental(),
+		UrlTemplate: "repositories/{{ .Params.FullName }}/issues",
+		Query: GetQueryCreatedAndUpdated(
+			`values.type,values.id,values.links.self,`+
+				`values.title,values.content.raw,values.reporter,values.assignee,`+
+				`values.state,values.milestone.id,values.component,values.priority,values.created_on,values.updated_on`,
+			collectorWithState),
+		GetTotalPages:  GetTotalPagesFromResponse,
+		ResponseParser: GetRawMessageFromResponse,
+		// some repo have no issue tracker
+		AfterResponse: ignoreHTTPStatus404,
 	})
-
 	if err != nil {
 		return err
 	}
 
-	return collector.Execute()
+	return collectorWithState.Execute()
 }
diff --git a/backend/plugins/bitbucket/tasks/issue_comment_collector.go b/backend/plugins/bitbucket/tasks/issue_comment_collector.go
index 58aa00ae6..dc9645348 100644
--- a/backend/plugins/bitbucket/tasks/issue_comment_collector.go
+++ b/backend/plugins/bitbucket/tasks/issue_comment_collector.go
@@ -36,28 +36,31 @@ var CollectApiIssueCommentsMeta = plugin.SubTaskMeta{
 
 func CollectApiIssueComments(taskCtx plugin.SubTaskContext) errors.Error {
 	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_COMMENTS_TABLE)
+	collectorWithState, err := helper.NewApiCollectorWithState(*rawDataSubTaskArgs, data.CreatedDateAfter)
+	if err != nil {
+		return err
+	}
 
-	iterator, err := GetIssuesIterator(taskCtx)
+	iterator, err := GetIssuesIterator(taskCtx, collectorWithState)
 	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 }}/issues/{{ .Input.BitbucketId }}/comments",
-		Query:              GetQuery,
-		GetTotalPages:      GetTotalPagesFromResponse,
-		ResponseParser:     GetRawMessageFromResponse,
-		AfterResponse:      ignoreHTTPStatus404,
+	err = collectorWithState.InitCollector(helper.ApiCollectorArgs{
+		ApiClient:      data.ApiClient,
+		PageSize:       100,
+		Incremental:    collectorWithState.IsIncremental(),
+		Input:          iterator,
+		UrlTemplate:    "repositories/{{ .Params.FullName }}/issues/{{ .Input.BitbucketId }}/comments",
+		Query:          GetQueryFields(`values.type,values.id,values.created_on,values.updated_on,values.content,values.issue.id,values.user`),
+		GetTotalPages:  GetTotalPagesFromResponse,
+		ResponseParser: GetRawMessageFromResponse,
+		AfterResponse:  ignoreHTTPStatus404,
 	})
 	if err != nil {
 		return err
 	}
 
-	return collector.Execute()
+	return collectorWithState.Execute()
 }
diff --git a/backend/plugins/bitbucket/tasks/issue_comment_convertor.go b/backend/plugins/bitbucket/tasks/issue_comment_convertor.go
index 76347916c..420a9be8b 100644
--- a/backend/plugins/bitbucket/tasks/issue_comment_convertor.go
+++ b/backend/plugins/bitbucket/tasks/issue_comment_convertor.go
@@ -38,9 +38,9 @@ var ConvertIssueCommentsMeta = plugin.SubTaskMeta{
 }
 
 func ConvertIssueComments(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_COMMENTS_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	cursor, err := db.Cursor(
 		dal.From(&models.BitbucketIssueComment{}),
@@ -54,27 +54,20 @@ func ConvertIssueComments(taskCtx plugin.SubTaskContext) errors.Error {
 	defer cursor.Close()
 
 	issueIdGen := didgen.NewDomainIdGenerator(&models.BitbucketIssue{})
+	issueCommentIdGen := didgen.NewDomainIdGenerator(&models.BitbucketIssueComment{})
 	accountIdGen := didgen.NewDomainIdGenerator(&models.BitbucketAccount{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(models.BitbucketIssueComment{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_ISSUE_COMMENTS_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(models.BitbucketIssueComment{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			bitbucketIssueComment := inputRow.(*models.BitbucketIssueComment)
 			domainIssueComment := &ticket.IssueComment{
 				DomainEntity: domainlayer.DomainEntity{
-					Id: issueIdGen.Generate(data.Options.ConnectionId, bitbucketIssueComment.BitbucketId),
+					Id: issueCommentIdGen.Generate(data.Options.ConnectionId, bitbucketIssueComment.BitbucketId),
 				},
-				IssueId:     issueIdGen.Generate(data.Options.ConnectionId, bitbucketIssueComment.IssueId),
+				IssueId:     issueIdGen.Generate(data.Options.ConnectionId, data.Options.FullName, bitbucketIssueComment.IssueId),
 				AccountId:   accountIdGen.Generate(data.Options.ConnectionId, bitbucketIssueComment.AuthorId),
 				CreatedDate: bitbucketIssueComment.CreatedAt,
 				Body:        bitbucketIssueComment.Body,
diff --git a/backend/plugins/bitbucket/tasks/issue_comment_extractor.go b/backend/plugins/bitbucket/tasks/issue_comment_extractor.go
index 76c87affd..54403a597 100644
--- a/backend/plugins/bitbucket/tasks/issue_comment_extractor.go
+++ b/backend/plugins/bitbucket/tasks/issue_comment_extractor.go
@@ -37,24 +37,11 @@ type BitbucketIssueCommentsResponse struct {
 	} `json:"content"`
 	User  *BitbucketAccountResponse `json:"user"`
 	Issue struct {
-		Type       string            `json:"type"`
-		Id         int               `json:"id"`
-		Repository *BitbucketApiRepo `json:"repository"`
-		Links      struct {
-			Self struct {
-				Href string `json:"href"`
-			} `json:"self"`
-		} `json:"links"`
-		Title string `json:"title"`
+		//Type string `json:"type"`
+		Id int `json:"id"`
+		//Repository *BitbucketApiRepo `json:"repository"`
+		//Title string `json:"title"`
 	}
-	Links struct {
-		Self struct {
-			Href string `json:"href"`
-		} `json:"self"`
-		Html struct {
-			Href string `json:"href"`
-		} `json:"html"`
-	} `json:"links"`
 }
 
 var ExtractApiIssueCommentsMeta = plugin.SubTaskMeta{
diff --git a/backend/plugins/bitbucket/tasks/issue_convertor.go b/backend/plugins/bitbucket/tasks/issue_convertor.go
index dc3568786..4975c82ce 100644
--- a/backend/plugins/bitbucket/tasks/issue_convertor.go
+++ b/backend/plugins/bitbucket/tasks/issue_convertor.go
@@ -39,9 +39,9 @@ var ConvertIssuesMeta = plugin.SubTaskMeta{
 }
 
 func ConvertIssues(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_COMMENTS_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	issue := &models.BitbucketIssue{}
 	cursor, err := db.Cursor(
@@ -58,26 +58,19 @@ func ConvertIssues(taskCtx plugin.SubTaskContext) errors.Error {
 	accountIdGen := didgen.NewDomainIdGenerator(&models.BitbucketAccount{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_ISSUE_TABLE,
-		},
-		InputRowType: reflect.TypeOf(models.BitbucketIssue{}),
-		Input:        cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		InputRowType:       reflect.TypeOf(models.BitbucketIssue{}),
+		Input:              cursor,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			issue := inputRow.(*models.BitbucketIssue)
 			domainIssue := &ticket.Issue{
-				DomainEntity:    domainlayer.DomainEntity{Id: issueIdGen.Generate(data.Options.ConnectionId, issue.BitbucketId)},
+				DomainEntity:    domainlayer.DomainEntity{Id: issueIdGen.Generate(data.Options.ConnectionId, issue.RepoId, issue.BitbucketId)},
 				IssueKey:        strconv.Itoa(issue.Number),
 				Title:           issue.Title,
 				Description:     issue.Body,
 				Priority:        issue.Priority,
 				Type:            issue.Type,
+				Status:          issue.StdState,
 				OriginalStatus:  issue.State,
 				LeadTimeMinutes: int64(issue.LeadTimeMinutes),
 				Url:             issue.Url,
@@ -87,11 +80,6 @@ func ConvertIssues(taskCtx plugin.SubTaskContext) errors.Error {
 				Severity:        issue.Severity,
 				Component:       issue.Component,
 			}
-			if issue.State == "closed" {
-				domainIssue.Status = ticket.DONE
-			} else {
-				domainIssue.Status = ticket.TODO
-			}
 			if issue.AssigneeName != "" {
 				domainIssue.AssigneeName = issue.AssigneeName
 				domainIssue.AssigneeId = accountIdGen.Generate(data.Options.ConnectionId, issue.AssigneeId)
diff --git a/backend/plugins/bitbucket/tasks/issue_extractor.go b/backend/plugins/bitbucket/tasks/issue_extractor.go
index 3103d4247..2a0b7a090 100644
--- a/backend/plugins/bitbucket/tasks/issue_extractor.go
+++ b/backend/plugins/bitbucket/tasks/issue_extractor.go
@@ -24,37 +24,30 @@ import (
 	plugin "github.com/apache/incubator-devlake/core/plugin"
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
 	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"strings"
 	"time"
 )
 
 type IssuesResponse struct {
-	Type        string            `json:"type"`
-	BitbucketId int               `json:"id"`
-	Repository  *BitbucketApiRepo `json:"repository"`
+	Type        string `json:"type"`
+	BitbucketId int    `json:"id"`
 	Links       struct {
 		Self struct {
 			Href string `json:"href"`
 		} `json:"self"`
-		Html struct {
-			Href string `json:"href"`
-		} `json:"html"`
 	} `json:"links"`
 	Title   string `json:"title"`
 	Content struct {
-		Type string `json:"type"`
-		Raw  string `json:"raw"`
+		Raw string `json:"raw"`
 	} `json:"content"`
 	Reporter  *BitbucketAccountResponse `json:"reporter"`
 	Assignee  *BitbucketAccountResponse `json:"assignee"`
 	State     string                    `json:"state"`
-	Kind      string                    `json:"kind"`
 	Milestone *struct {
 		Id int `json:"id"`
 	} `json:"milestone"`
 	Component          string    `json:"component"`
 	Priority           string    `json:"priority"`
-	Votes              int       `json:"votes"`
-	Watches            int       `json:"watches"`
 	BitbucketCreatedAt time.Time `json:"created_on"`
 	BitbucketUpdatedAt time.Time `json:"updated_on"`
 }
@@ -68,29 +61,13 @@ var ExtractApiIssuesMeta = plugin.SubTaskMeta{
 }
 
 func ExtractApiIssues(taskCtx plugin.SubTaskContext) errors.Error {
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	config := data.Options.TransformationRules
-	issueStatusMap, err := newIssueStatusMap(config)
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_TABLE)
+	issueStatusMap, err := newIssueStatusMap(data.Options.BitbucketTransformationRule)
 	if err != nil {
-		return nil
+		return err
 	}
 	extractor, err := api.NewApiExtractor(api.ApiExtractorArgs{
-		RawDataSubTaskArgs: api.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_ISSUE_TABLE,
-		},
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Extract: func(row *api.RawData) ([]interface{}, errors.Error) {
 			body := &IssuesResponse{}
 			err := errors.Convert(json.Unmarshal(row.Data, body))
@@ -107,12 +84,11 @@ func ExtractApiIssues(taskCtx plugin.SubTaskContext) errors.Error {
 			}
 			results := make([]interface{}, 0, 2)
 
-			bitbucketIssue, err := convertBitbucketIssue(body, data.Options.ConnectionId, data.Repo.BitbucketId)
+			bitbucketIssue, err := convertBitbucketIssue(body, data.Options.ConnectionId, data.Options.FullName)
 			if err != nil {
 				return nil, err
 			}
 
-			results = append(results, bitbucketIssue)
 			if body.Assignee != nil {
 				bitbucketIssue.AssigneeId = body.Assignee.AccountId
 				bitbucketIssue.AssigneeName = body.Assignee.DisplayName
@@ -132,8 +108,9 @@ func ExtractApiIssues(taskCtx plugin.SubTaskContext) errors.Error {
 				results = append(results, relatedUser)
 			}
 			if status, ok := issueStatusMap[bitbucketIssue.State]; ok {
-				bitbucketIssue.State = status
+				bitbucketIssue.StdState = status
 			}
+			results = append(results, bitbucketIssue)
 			return results, nil
 		},
 	})
@@ -176,18 +153,21 @@ func convertBitbucketIssue(issue *IssuesResponse, connectionId uint64, repositor
 	return bitbucketIssue, nil
 }
 
-func newIssueStatusMap(config models.TransformationRules) (map[string]string, errors.Error) {
+func newIssueStatusMap(config *models.BitbucketTransformationRule) (map[string]string, errors.Error) {
 	issueStatusMap := make(map[string]string, 3)
-	for _, state := range config.IssueStatusTODO {
+	if config == nil {
+		return issueStatusMap, nil
+	}
+	for _, state := range strings.Split(config.IssueStatusTodo, `,`) {
 		issueStatusMap[state] = ticket.TODO
 	}
-	for _, state := range config.IssueStatusINPROGRESS {
+	for _, state := range strings.Split(config.IssueStatusInProgress, `,`) {
 		issueStatusMap[state] = ticket.IN_PROGRESS
 	}
-	for _, state := range config.IssueStatusDONE {
+	for _, state := range strings.Split(config.IssueStatusDone, `,`) {
 		issueStatusMap[state] = ticket.DONE
 	}
-	for _, state := range config.IssueStatusOTHER {
+	for _, state := range strings.Split(config.IssueStatusOther, `,`) {
 		issueStatusMap[state] = ticket.OTHER
 	}
 	return issueStatusMap, nil
diff --git a/backend/plugins/bitbucket/tasks/pipeline_collector.go b/backend/plugins/bitbucket/tasks/pipeline_collector.go
index d5b23c21f..d49559c87 100644
--- a/backend/plugins/bitbucket/tasks/pipeline_collector.go
+++ b/backend/plugins/bitbucket/tasks/pipeline_collector.go
@@ -35,21 +35,27 @@ var CollectApiPipelinesMeta = plugin.SubTaskMeta{
 
 func CollectApiPipelines(taskCtx plugin.SubTaskContext) errors.Error {
 	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PIPELINE_TABLE)
+	collectorWithState, err := helper.NewApiCollectorWithState(*rawDataSubTaskArgs, data.CreatedDateAfter)
+	if err != nil {
+		return err
+	}
 
-	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
-		RawDataSubTaskArgs: *rawDataSubTaskArgs,
-		ApiClient:          data.ApiClient,
-		PageSize:           50,
-		Incremental:        false,
-		UrlTemplate:        "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/pipelines/",
-		Query:              GetQuery,
-		ResponseParser:     GetRawMessageFromResponse,
-		GetTotalPages:      GetTotalPagesFromResponse,
+	err = collectorWithState.InitCollector(helper.ApiCollectorArgs{
+		ApiClient:   data.ApiClient,
+		PageSize:    50,
+		Incremental: collectorWithState.IsIncremental(),
+		UrlTemplate: "repositories/{{ .Params.FullName }}/pipelines/",
+		Query: GetQueryCreatedAndUpdated(
+			`values.uuid,values.type,values.state.name,values.state.result.name,values.state.result.type,values.state.stage.name,values.state.stage.type,`+
+				`values.target.ref_name,values.target.commit.hash,`+
+				`values.created_on,values.completed_on,values.duration_in_seconds,values.links.self`,
+			collectorWithState),
+		ResponseParser: GetRawMessageFromResponse,
+		GetTotalPages:  GetTotalPagesFromResponse,
 	})
-
 	if err != nil {
 		return err
 	}
 
-	return collector.Execute()
+	return collectorWithState.Execute()
 }
diff --git a/backend/plugins/bitbucket/tasks/pipeline_convertor.go b/backend/plugins/bitbucket/tasks/pipeline_convertor.go
index 2eb4454fe..83a293983 100644
--- a/backend/plugins/bitbucket/tasks/pipeline_convertor.go
+++ b/backend/plugins/bitbucket/tasks/pipeline_convertor.go
@@ -39,8 +39,8 @@ var ConvertPipelineMeta = plugin.SubTaskMeta{
 }
 
 func ConvertPipelines(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PIPELINE_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
 
 	cursor, err := db.Cursor(dal.From(models.BitbucketPipeline{}))
 	if err != nil {
@@ -51,17 +51,9 @@ func ConvertPipelines(taskCtx plugin.SubTaskContext) errors.Error {
 	pipelineIdGen := didgen.NewDomainIdGenerator(&models.BitbucketPipeline{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(models.BitbucketPipeline{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_PIPELINE_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(models.BitbucketPipeline{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			bitbucketPipeline := inputRow.(*models.BitbucketPipeline)
 
diff --git a/backend/plugins/bitbucket/tasks/pipeline_extractor.go b/backend/plugins/bitbucket/tasks/pipeline_extractor.go
index a572a716a..43f903b41 100644
--- a/backend/plugins/bitbucket/tasks/pipeline_extractor.go
+++ b/backend/plugins/bitbucket/tasks/pipeline_extractor.go
@@ -26,34 +26,26 @@ import (
 )
 
 type bitbucketApiCommit struct {
-	Type  string `json:"type"`
-	Hash  string `json:"hash"`
-	Links struct {
-		Self struct {
-			Href string `json:"href"`
-		}
-		Html struct {
-			Href string `json:"href"`
-		}
-	} `json:"links"`
+	//Type  string `json:"type"`
+	Hash string `json:"hash"`
 }
 
 type bitbucketApiPipelineTarget struct {
-	Type     string `json:"type"`
-	RefType  string `json:"ref_type"`
-	RefName  string `json:"ref_name"`
-	Selector struct {
-		Type string `json:"type"`
-	} `json:"selector"`
+	//Type     string `json:"type"`
+	//RefType  string `json:"ref_type"`
+	RefName string `json:"ref_name"`
+	//Selector struct {
+	//	Type string `json:"type"`
+	//} `json:"selector"`
 	Commit *bitbucketApiCommit `json:"commit"`
 }
 
 type BitbucketApiPipeline struct {
-	Uuid  string `json:"uuid"`
-	Type  string `json:"type"`
+	Uuid string `json:"uuid"`
+	//Type  string `json:"type"`
 	State struct {
-		Name   string `json:"name"`
-		Type   string `json:"type"`
+		Name string `json:"name"`
+		//	Type   string `json:"type"`
 		Result *struct {
 			Name string `json:"name"`
 			Type string `json:"type"`
@@ -63,29 +55,14 @@ type BitbucketApiPipeline struct {
 			Type string `json:"type"`
 		} `json:"stage"`
 	} `json:"state"`
-	BuildNumber int                         `json:"build_number"`
-	Creator     *BitbucketAccountResponse   `json:"creator"`
-	Repo        *BitbucketApiRepo           `json:"repository"`
-	Target      *bitbucketApiPipelineTarget `json:"target"`
-	Trigger     struct {
-		Name string `json:"name"`
-		Type string `json:"type"`
-	} `json:"trigger"`
-	CreatedOn         *api.Iso8601Time `json:"created_on"`
-	CompletedOn       *api.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"`
+	Target            *bitbucketApiPipelineTarget `json:"target"`
+	CreatedOn         *api.Iso8601Time            `json:"created_on"`
+	CompletedOn       *api.Iso8601Time            `json:"completed_on"`
+	DurationInSeconds uint64                      `json:"duration_in_seconds"`
 	Links             struct {
 		Self struct {
 			Href string `json:"href"`
 		} `json:"self"`
-		Steps struct {
-			Href string `json:"href"`
-		} `json:"steps"`
 	} `json:"links"`
 }
 
@@ -117,7 +94,7 @@ func ExtractApiPipelines(taskCtx plugin.SubTaskContext) errors.Error {
 				Status:              bitbucketApiPipeline.State.Name,
 				RefName:             bitbucketApiPipeline.Target.RefName,
 				CommitSha:           bitbucketApiPipeline.Target.Commit.Hash,
-				RepoId:              bitbucketApiPipeline.Repo.BitbucketId,
+				RepoId:              data.Options.FullName,
 				DurationInSeconds:   bitbucketApiPipeline.DurationInSeconds,
 				BitbucketCreatedOn:  api.Iso8601TimeToTime(bitbucketApiPipeline.CreatedOn),
 				BitbucketCompleteOn: api.Iso8601TimeToTime(bitbucketApiPipeline.CompletedOn),
diff --git a/backend/plugins/bitbucket/tasks/pr_collector.go b/backend/plugins/bitbucket/tasks/pr_collector.go
index 613f3bf0a..671f320db 100644
--- a/backend/plugins/bitbucket/tasks/pr_collector.go
+++ b/backend/plugins/bitbucket/tasks/pr_collector.go
@@ -38,21 +38,28 @@ var CollectApiPullRequestsMeta = plugin.SubTaskMeta{
 
 func CollectApiPullRequests(taskCtx plugin.SubTaskContext) errors.Error {
 	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_TABLE)
+	collectorWithState, err := helper.NewApiCollectorWithState(*rawDataSubTaskArgs, data.CreatedDateAfter)
+	if err != nil {
+		return err
+	}
 
-	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
-		RawDataSubTaskArgs: *rawDataSubTaskArgs,
-		ApiClient:          data.ApiClient,
-		PageSize:           50,
-		Incremental:        false,
-		UrlTemplate:        "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/pullrequests",
-		Query:              GetQuery,
-		GetTotalPages:      GetTotalPagesFromResponse,
-		ResponseParser:     GetRawMessageFromResponse,
+	err = collectorWithState.InitCollector(helper.ApiCollectorArgs{
+		ApiClient:   data.ApiClient,
+		PageSize:    50,
+		Incremental: collectorWithState.IsIncremental(),
+		UrlTemplate: "repositories/{{ .Params.FullName }}/pullrequests",
+		Query: GetQueryCreatedAndUpdated(
+			`values.id,values.comment_count,values.type,values.state,values.title,values.description,`+
+				`values.merge_commit.hash,values.merge_commit.date,values.links.html,values.author,values.created_on,values.updated_on,`+
+				`values.destination.branch.name,values.destination.commit.hash,values.destination.repository.full_name,`+
+				`values.source.branch.name,values.source.commit.hash,values.source.repository.full_name`,
+			collectorWithState),
+		GetTotalPages:  GetTotalPagesFromResponse,
+		ResponseParser: GetRawMessageFromResponse,
 	})
-
 	if err != nil {
 		return err
 	}
 
-	return collector.Execute()
+	return collectorWithState.Execute()
 }
diff --git a/backend/plugins/bitbucket/tasks/pr_comment_collector.go b/backend/plugins/bitbucket/tasks/pr_comment_collector.go
index 0dc7379e6..53b6f4ab6 100644
--- a/backend/plugins/bitbucket/tasks/pr_comment_collector.go
+++ b/backend/plugins/bitbucket/tasks/pr_comment_collector.go
@@ -36,27 +36,30 @@ var CollectApiPrCommentsMeta = plugin.SubTaskMeta{
 
 func CollectApiPullRequestsComments(taskCtx plugin.SubTaskContext) errors.Error {
 	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_COMMENTS_TABLE)
+	collectorWithState, err := helper.NewApiCollectorWithState(*rawDataSubTaskArgs, data.CreatedDateAfter)
+	if err != nil {
+		return err
+	}
 
-	iterator, err := GetPullRequestsIterator(taskCtx)
+	iterator, err := GetPullRequestsIterator(taskCtx, collectorWithState)
 	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 }}/comments",
-		Query:              GetQuery,
-		GetTotalPages:      GetTotalPagesFromResponse,
-		ResponseParser:     GetRawMessageFromResponse,
+	err = collectorWithState.InitCollector(helper.ApiCollectorArgs{
+		ApiClient:      data.ApiClient,
+		PageSize:       100,
+		Incremental:    collectorWithState.IsIncremental(),
+		Input:          iterator,
+		UrlTemplate:    "repositories/{{ .Params.FullName }}/pullrequests/{{ .Input.BitbucketId }}/comments",
+		Query:          GetQueryFields(`values.id,values.type,values.created_on,values.updated_on,values.content.raw,values.pullrequest.id,values.user`),
+		GetTotalPages:  GetTotalPagesFromResponse,
+		ResponseParser: GetRawMessageFromResponse,
 	})
 	if err != nil {
 		return err
 	}
 
-	return collector.Execute()
+	return collectorWithState.Execute()
 }
diff --git a/backend/plugins/bitbucket/tasks/pr_comment_convertor.go b/backend/plugins/bitbucket/tasks/pr_comment_convertor.go
index 901657e5b..66cf0a785 100644
--- a/backend/plugins/bitbucket/tasks/pr_comment_convertor.go
+++ b/backend/plugins/bitbucket/tasks/pr_comment_convertor.go
@@ -38,9 +38,9 @@ var ConvertPrCommentsMeta = plugin.SubTaskMeta{
 }
 
 func ConvertPullRequestComments(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_COMMENTS_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	cursor, err := db.Cursor(
 		dal.From(&models.BitbucketPrComment{}),
@@ -58,24 +58,16 @@ func ConvertPullRequestComments(taskCtx plugin.SubTaskContext) errors.Error {
 	accountIdGen := didgen.NewDomainIdGenerator(&models.BitbucketAccount{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(models.BitbucketPrComment{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_PULL_REQUEST_COMMENTS_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(models.BitbucketPrComment{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			bitbucketPullRequestComment := inputRow.(*models.BitbucketPrComment)
 			domainPrComment := &code.PullRequestComment{
 				DomainEntity: domainlayer.DomainEntity{
 					Id: domainIdGeneratorComment.Generate(data.Options.ConnectionId, bitbucketPullRequestComment.BitbucketId),
 				},
-				PullRequestId: prIdGen.Generate(data.Options.ConnectionId, bitbucketPullRequestComment.PullRequestId),
+				PullRequestId: prIdGen.Generate(data.Options.ConnectionId, data.Options.FullName, bitbucketPullRequestComment.PullRequestId),
 				AccountId:     accountIdGen.Generate(data.Options.ConnectionId, bitbucketPullRequestComment.AuthorId),
 				CreatedDate:   bitbucketPullRequestComment.CreatedAt,
 				Body:          bitbucketPullRequestComment.Body,
diff --git a/backend/plugins/bitbucket/tasks/pr_comment_extractor.go b/backend/plugins/bitbucket/tasks/pr_comment_extractor.go
index c1559f41f..84e99ad4a 100644
--- a/backend/plugins/bitbucket/tasks/pr_comment_extractor.go
+++ b/backend/plugins/bitbucket/tasks/pr_comment_extractor.go
@@ -39,34 +39,14 @@ type BitbucketPrCommentsResponse struct {
 	BitbucketId int        `json:"id"`
 	CreatedOn   time.Time  `json:"created_on"`
 	UpdatedOn   *time.Time `json:"updated_on"`
+	Type        string     `json:"type"`
 	Content     struct {
-		Type string `json:"type"`
-		Raw  string `json:"raw"`
+		Raw string `json:"raw"`
 	} `json:"content"`
-	User    *BitbucketAccountResponse `json:"user"`
-	Deleted bool                      `json:"deleted"`
-	Type    string                    `json:"type"`
-	Links   struct {
-		Self struct {
-			Href string
-		} `json:"self"`
-		Html struct {
-			Href string
-		} `json:"html"`
-	} `json:"links"`
 	PullRequest struct {
-		Type  string `json:"type"`
-		Id    int    `json:"id"`
-		Title string `json:"title"`
-		Links struct {
-			Self struct {
-				Href string `json:"href"`
-			} `json:"self"`
-			Html struct {
-				Href string `json:"href"`
-			} `json:"html"`
-		} `json:"links"`
-	}
+		Id int `json:"id"`
+	} `json:"pullrequest"`
+	User *BitbucketAccountResponse `json:"user"`
 }
 
 func ExtractApiPullRequestsComments(taskCtx plugin.SubTaskContext) errors.Error {
@@ -94,7 +74,7 @@ func ExtractApiPullRequestsComments(taskCtx plugin.SubTaskContext) errors.Error
 					return nil, err
 				}
 				toolprComment.AuthorId = bitbucketUser.AccountId
-				toolprComment.AuthorName = bitbucketUser.UserName
+				toolprComment.AuthorName = bitbucketUser.DisplayName
 				results = append(results, bitbucketUser)
 			}
 			results = append(results, toolprComment)
@@ -102,7 +82,6 @@ func ExtractApiPullRequestsComments(taskCtx plugin.SubTaskContext) errors.Error
 			return results, nil
 		},
 	})
-
 	if err != nil {
 		return err
 	}
diff --git a/backend/plugins/bitbucket/tasks/pr_commit_collector.go b/backend/plugins/bitbucket/tasks/pr_commit_collector.go
index a0303275c..91c33fe13 100644
--- a/backend/plugins/bitbucket/tasks/pr_commit_collector.go
+++ b/backend/plugins/bitbucket/tasks/pr_commit_collector.go
@@ -18,9 +18,11 @@ limitations under the License.
 package tasks
 
 import (
+	"fmt"
 	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/apache/incubator-devlake/core/plugin"
 	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"net/url"
 )
 
 const RAW_PULL_REQUEST_COMMITS_TABLE = "bitbucket_api_pull_request_commits"
@@ -35,27 +37,44 @@ var CollectApiPrCommitsMeta = plugin.SubTaskMeta{
 
 func CollectApiPullRequestCommits(taskCtx plugin.SubTaskContext) errors.Error {
 	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_COMMITS_TABLE)
+	collectorWithState, err := helper.NewApiCollectorWithState(*rawDataSubTaskArgs, data.CreatedDateAfter)
+	if err != nil {
+		return err
+	}
 
-	iterator, err := GetPullRequestsIterator(taskCtx)
+	iterator, err := GetPullRequestsIterator(taskCtx, collectorWithState)
 	if err != nil {
 		return err
 	}
 	defer iterator.Close()
 
-	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
-		RawDataSubTaskArgs:    *rawDataSubTaskArgs,
+	err = collectorWithState.InitCollector(helper.ApiCollectorArgs{
 		ApiClient:             data.ApiClient,
 		PageSize:              100,
-		Incremental:           false,
+		Incremental:           collectorWithState.IsIncremental(),
 		Input:                 iterator,
-		UrlTemplate:           "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}/pullrequests/{{ .Input.BitbucketId }}/commits",
+		UrlTemplate:           "repositories/{{ .Params.FullName }}/pullrequests/{{ .Input.BitbucketId }}/commits",
 		GetNextPageCustomData: GetNextPageCustomData,
-		Query:                 GetQueryForNext,
-		ResponseParser:        GetRawMessageFromResponse,
-	})
+		Query: func(reqData *helper.RequestData) (url.Values, errors.Error) {
+			query := url.Values{}
+			query.Set("state", "all")
+			query.Set("pagelen", fmt.Sprintf("%v", reqData.Pager.Size))
+			query.Set("sort", "created_on")
+			query.Set("fields", "values.hash,values.date,values.message,values.author.raw,values.author.user.username,values.author.user.account_id,values.links.self")
 
+			if reqData.CustomData != nil {
+				query.Set("page", reqData.CustomData.(string))
+			}
+			return query, nil
+		},
+		ResponseParser: GetRawMessageFromResponse,
+		// some pr have no commit
+		// such as: https://bitbucket.org/amdatulabs/amdatu-kubernetes-deployer/pull-requests/21
+		AfterResponse: ignoreHTTPStatus404,
+	})
 	if err != nil {
 		return err
 	}
-	return collector.Execute()
+
+	return collectorWithState.Execute()
 }
diff --git a/backend/plugins/bitbucket/tasks/pr_commit_convertor.go b/backend/plugins/bitbucket/tasks/pr_commit_convertor.go
index 644afe30c..cdf6d72c7 100644
--- a/backend/plugins/bitbucket/tasks/pr_commit_convertor.go
+++ b/backend/plugins/bitbucket/tasks/pr_commit_convertor.go
@@ -38,9 +38,9 @@ var ConvertPrCommitsMeta = plugin.SubTaskMeta{
 }
 
 func ConvertPullRequestCommits(taskCtx plugin.SubTaskContext) (err errors.Error) {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_COMMITS_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	pullIdGen := didgen.NewDomainIdGenerator(&bitbucketModels.BitbucketPullRequest{})
 
@@ -56,22 +56,14 @@ func ConvertPullRequestCommits(taskCtx plugin.SubTaskContext) (err errors.Error)
 	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,
-		},
+		InputRowType:       reflect.TypeOf(bitbucketModels.BitbucketPrCommit{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		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),
+				PullRequestId: pullIdGen.Generate(data.Options.ConnectionId, data.Options.FullName, bitbucketPullRequestCommit.PullRequestId),
 			}
 			return []interface{}{
 				domainPrCommit,
diff --git a/backend/plugins/bitbucket/tasks/pr_commit_extractor.go b/backend/plugins/bitbucket/tasks/pr_commit_extractor.go
index 0d65bbe4e..4abd76966 100644
--- a/backend/plugins/bitbucket/tasks/pr_commit_extractor.go
+++ b/backend/plugins/bitbucket/tasks/pr_commit_extractor.go
@@ -37,64 +37,27 @@ var ExtractApiPrCommitsMeta = plugin.SubTaskMeta{
 }
 
 type ApiPrCommitsResponse struct {
-	Type   string    `json:"type"`
+	//Type   string    `json:"type"`
 	Hash   string    `json:"hash"`
 	Date   time.Time `json:"date"`
 	Author struct {
-		Type string                   `json:"type"`
+		//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 {
+	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 plugin.SubTaskContext) errors.Error {
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_COMMITS_TABLE)
+	repoId := data.Options.FullName
 	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,
-		},
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Extract: func(row *helper.RawData) ([]interface{}, errors.Error) {
 			apiPullRequestCommit := &ApiPrCommitsResponse{}
 			if strings.HasPrefix(string(row.Data), "Not Found") {
@@ -149,7 +112,7 @@ func convertPullRequestCommit(prCommit *ApiPrCommitsResponse, connId uint64) (*m
 		Sha:           prCommit.Hash,
 		Message:       prCommit.Message,
 		AuthorId:      prCommit.Author.User.AccountId,
-		AuthorName:    prCommit.Author.User.UserName,
+		AuthorName:    prCommit.Author.User.DisplayName,
 		AuthorEmail:   prCommit.Author.Raw,
 		AuthoredDate:  prCommit.Date,
 		CommittedDate: prCommit.Date,
diff --git a/backend/plugins/bitbucket/tasks/pr_convertor.go b/backend/plugins/bitbucket/tasks/pr_convertor.go
index 0f8794608..0d0877708 100644
--- a/backend/plugins/bitbucket/tasks/pr_convertor.go
+++ b/backend/plugins/bitbucket/tasks/pr_convertor.go
@@ -39,9 +39,9 @@ var ConvertPullRequestsMeta = plugin.SubTaskMeta{
 }
 
 func ConvertPullRequests(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	cursor, err := db.Cursor(
 		dal.From(&models.BitbucketPullRequest{}),
@@ -57,31 +57,14 @@ func ConvertPullRequests(taskCtx plugin.SubTaskContext) errors.Error {
 	domainUserIdGen := didgen.NewDomainIdGenerator(&models.BitbucketAccount{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(models.BitbucketPullRequest{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_PULL_REQUEST_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(models.BitbucketPullRequest{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			pr := inputRow.(*models.BitbucketPullRequest)
-
-			// Getting the merge reference commit
-			mergeCommit := &models.BitbucketCommit{}
-			err = db.First(mergeCommit, dal.Where("LEFT(sha, 12) = ?", pr.MergeCommitSha))
-			if err == nil {
-				// Setting the PR merged datetime to the commit commited datetime
-				pr.MergedAt = &mergeCommit.CommittedDate
-			}
-
 			domainPr := &code.PullRequest{
 				DomainEntity: domainlayer.DomainEntity{
-					Id: prIdGen.Generate(data.Options.ConnectionId, pr.BitbucketId),
+					Id: prIdGen.Generate(data.Options.ConnectionId, pr.RepoId, pr.BitbucketId),
 				},
 				BaseRepoId:     repoIdGen.Generate(data.Options.ConnectionId, pr.BaseRepoId),
 				HeadRepoId:     repoIdGen.Generate(data.Options.ConnectionId, pr.HeadRepoId),
diff --git a/backend/plugins/bitbucket/tasks/pr_extractor.go b/backend/plugins/bitbucket/tasks/pr_extractor.go
index 8e54c8d63..8a8ab6c21 100644
--- a/backend/plugins/bitbucket/tasks/pr_extractor.go
+++ b/backend/plugins/bitbucket/tasks/pr_extractor.go
@@ -36,34 +36,24 @@ var ExtractApiPullRequestsMeta = plugin.SubTaskMeta{
 }
 
 type BitbucketApiPullRequest struct {
-	BitbucketId  int    `json:"id"`
-	CommentCount int    `json:"comment_count"`
-	TaskCount    int    `json:"task_count"`
-	Type         string `json:"type"`
-	State        string `json:"state"`
-	Title        string `json:"title"`
-	Description  string `json:"description"`
-	MergeCommit  *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"`
+	BitbucketId  int `json:"id"`
+	CommentCount int `json:"comment_count"`
+	//TaskCount    int    `json:"task_count"`
+	Type        string `json:"type"`
+	State       string `json:"state"`
+	Title       string `json:"title"`
+	Description string `json:"description"`
+	MergeCommit *struct {
+		Hash string `json:"hash"`
+		// date only return when fields defined
+		Date *api.Iso8601Time `json:"date"`
 	} `json:"merge_commit"`
 	Links *struct {
-		Self struct {
-			Href string `json:"href"`
-		} `json:"self"`
 		Html struct {
 			Href string `json:"href"`
 		} `json:"html"`
 	} `json:"links"`
-	ClosedBy           *BitbucketAccountResponse `json:"closed_by"`
+	//ClosedBy           *BitbucketAccountResponse `json:"closed_by"`
 	Author             *BitbucketAccountResponse `json:"author"`
 	BitbucketCreatedAt time.Time                 `json:"created_on"`
 	BitbucketUpdatedAt time.Time                 `json:"updated_on"`
@@ -72,7 +62,6 @@ type BitbucketApiPullRequest struct {
 			Name string `json:"name"`
 		} `json:"branch"`
 		Commit struct {
-			Type string `json:"type"`
 			Hash string `json:"hash"`
 		} `json:"commit"`
 		Repo *BitbucketApiRepo `json:"repository"`
@@ -82,35 +71,19 @@ type BitbucketApiPullRequest struct {
 			Name string `json:"name"`
 		} `json:"branch"`
 		Commit struct {
-			Type string `json:"type"`
 			Hash string `json:"hash"`
 		} `json:"commit"`
 		Repo *BitbucketApiRepo `json:"repository"`
 	} `json:"source"`
-	Reviewers    []BitbucketAccountResponse `json:"reviewers"`
-	Participants []BitbucketAccountResponse `json:"participants"`
+	//Reviewers    []BitbucketAccountResponse `json:"reviewers"`
+	//Participants []BitbucketAccountResponse `json:"participants"`
 }
 
 func ExtractApiPullRequests(taskCtx plugin.SubTaskContext) errors.Error {
-	data := taskCtx.GetData().(*BitbucketTaskData)
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PULL_REQUEST_TABLE)
 	var err errors.Error
 	extractor, err := api.NewApiExtractor(api.ApiExtractorArgs{
-		RawDataSubTaskArgs: api.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_TABLE,
-		},
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Extract: func(row *api.RawData) ([]interface{}, errors.Error) {
 			rawL := &BitbucketApiPullRequest{}
 			err := errors.Convert(json.Unmarshal(row.Data, rawL))
@@ -123,7 +96,7 @@ func ExtractApiPullRequests(taskCtx plugin.SubTaskContext) errors.Error {
 				return nil, nil
 			}
 
-			bitbucketPr, err := convertBitbucketPullRequest(rawL, data.Options.ConnectionId, data.Repo.BitbucketId)
+			bitbucketPr, err := convertBitbucketPullRequest(rawL, data.Options.ConnectionId, data.Options.FullName)
 			if err != nil {
 				return nil, err
 			}
@@ -138,6 +111,7 @@ func ExtractApiPullRequests(taskCtx plugin.SubTaskContext) errors.Error {
 			}
 			if rawL.MergeCommit != nil {
 				bitbucketPr.MergeCommitSha = rawL.MergeCommit.Hash
+				bitbucketPr.MergedAt = rawL.MergeCommit.Date.ToNullableTime()
 			}
 			results = append(results, bitbucketPr)
 
@@ -153,6 +127,7 @@ func convertBitbucketPullRequest(pull *BitbucketApiPullRequest, connId uint64, r
 	bitbucketPull := &models.BitbucketPullRequest{
 		ConnectionId:       connId,
 		BitbucketId:        pull.BitbucketId,
+		Number:             pull.BitbucketId,
 		RepoId:             repoId,
 		BaseRepoId:         pull.BaseRef.Repo.FullName,
 		HeadRepoId:         pull.HeadRef.Repo.FullName,
diff --git a/backend/plugins/bitbucket/tasks/repo_collector.go b/backend/plugins/bitbucket/tasks/repo_collector.go
deleted file mode 100644
index 3b82b6423..000000000
--- a/backend/plugins/bitbucket/tasks/repo_collector.go
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
-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"
-	"github.com/apache/incubator-devlake/core/errors"
-	plugin "github.com/apache/incubator-devlake/core/plugin"
-	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	"io"
-	"net/http"
-)
-
-const RAW_REPOSITORIES_TABLE = "bitbucket_api_repositories"
-
-var CollectApiRepoMeta = plugin.SubTaskMeta{
-	Name:        "collectApiRepo",
-	EntryPoint:  CollectApiRepositories,
-	Required:    true,
-	Description: "Collect repositories data from Bitbucket api",
-	DomainTypes: []string{plugin.DOMAIN_TYPE_CODE},
-}
-
-func CollectApiRepositories(taskCtx plugin.SubTaskContext) errors.Error {
-	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_REPOSITORIES_TABLE)
-
-	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
-		RawDataSubTaskArgs: *rawDataSubTaskArgs,
-		ApiClient:          data.ApiClient,
-
-		UrlTemplate: "repositories/{{ .Params.Owner }}/{{ .Params.Repo }}",
-		Query:       GetQuery,
-		ResponseParser: func(res *http.Response) ([]json.RawMessage, errors.Error) {
-			body, err := io.ReadAll(res.Body)
-			res.Body.Close()
-			if err != nil {
-				return nil, errors.Convert(err)
-			}
-			return []json.RawMessage{body}, nil
-		},
-	})
-
-	if err != nil {
-		return err
-	}
-
-	return collector.Execute()
-}
diff --git a/backend/plugins/bitbucket/tasks/repo_convertor.go b/backend/plugins/bitbucket/tasks/repo_convertor.go
index ddae4ce28..baf855317 100644
--- a/backend/plugins/bitbucket/tasks/repo_convertor.go
+++ b/backend/plugins/bitbucket/tasks/repo_convertor.go
@@ -18,6 +18,7 @@ limitations under the License.
 package tasks
 
 import (
+	"encoding/json"
 	"fmt"
 	"github.com/apache/incubator-devlake/core/dal"
 	"github.com/apache/incubator-devlake/core/errors"
@@ -27,10 +28,17 @@ import (
 	"github.com/apache/incubator-devlake/core/models/domainlayer/ticket"
 	plugin "github.com/apache/incubator-devlake/core/plugin"
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	aha "github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"io"
+	"net/http"
+	"path"
 	"reflect"
+	"time"
 )
 
+const RAW_REPOSITORIES_TABLE = "bitbucket_api_repositories"
+
 var ConvertRepoMeta = plugin.SubTaskMeta{
 	Name:             "convertRepo",
 	EntryPoint:       ConvertRepo,
@@ -39,10 +47,92 @@ var ConvertRepoMeta = plugin.SubTaskMeta{
 	DomainTypes:      []string{plugin.DOMAIN_TYPE_CODE},
 }
 
+type ApiRepoResponse BitbucketApiRepo
+
+type BitbucketApiRepo struct {
+	//Scm         string `json:"scm"`
+	//HasWiki     bool   `json:"has_wiki"`
+	//Uuid        string `json:"uuid"`
+	//Type        string `json:"type"`
+	//HasIssue    bool   `json:"has_issue"`
+	//ForkPolicy  string `json:"fork_policy"`
+	Name        string `json:"name"`
+	FullName    string `json:"full_name"`
+	Language    string `json:"language"`
+	Description string `json:"description"`
+	Owner       struct {
+		Username string `json:"username"`
+	} `json:"owner"`
+	CreatedAt *time.Time `json:"created_on"`
+	UpdatedAt *time.Time `json:"updated_on"`
+	Links     struct {
+		Clone []struct {
+			Href string `json:"href"`
+			Name string `json:"name"`
+		} `json:"clone"`
+		Html struct {
+			Href string `json:"href"`
+		} `json:"html"`
+	} `json:"links"`
+}
+
+func ConvertApiRepoToScope(repo *BitbucketApiRepo, connectionId uint64) *models.BitbucketRepo {
+	var scope models.BitbucketRepo
+	scope.ConnectionId = connectionId
+	scope.BitbucketId = repo.FullName
+	scope.CreatedDate = repo.CreatedAt
+	scope.UpdatedDate = repo.UpdatedAt
+	scope.Language = repo.Language
+	scope.Description = repo.Description
+	scope.Name = repo.Name
+	scope.Owner = repo.Owner.Username
+	scope.HTMLUrl = repo.Links.Html.Href
+
+	scope.CloneUrl = ""
+	for _, u := range repo.Links.Clone {
+		if u.Name == "https" {
+			scope.CloneUrl = u.Href
+		}
+	}
+	return &scope
+}
+
+func GetApiRepo(
+	op *BitbucketOptions,
+	apiClient aha.ApiClientAbstract,
+) (*BitbucketApiRepo, errors.Error) {
+	res, err := apiClient.Get(path.Join("repositories", op.FullName), nil, nil)
+	if err != nil {
+		return nil, err
+	}
+	defer res.Body.Close()
+	if res.StatusCode != http.StatusOK {
+		return nil, errors.Default.New(fmt.Sprintf(
+			"unexpected status code when requesting repo detail %d %s",
+			res.StatusCode, res.Request.URL.String(),
+		))
+	}
+	body, err := errors.Convert01(io.ReadAll(res.Body))
+	if err != nil {
+		return nil, err
+	}
+	apiRepo := new(BitbucketApiRepo)
+	err = errors.Convert(json.Unmarshal(body, apiRepo))
+	if err != nil {
+		return nil, err
+	}
+	for _, u := range apiRepo.Links.Clone {
+		if u.Name == "https" {
+			return apiRepo, nil
+		}
+	}
+	return nil, errors.Default.New("no clone url")
+}
+
 func ConvertRepo(taskCtx plugin.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_REPOSITORIES_TABLE)
 	db := taskCtx.GetDal()
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	repoId := data.Repo.BitbucketId
+	repoId := data.Options.FullName
 
 	cursor, err := db.Cursor(
 		dal.From(&models.BitbucketRepo{}),
@@ -56,17 +146,9 @@ func ConvertRepo(taskCtx plugin.SubTaskContext) errors.Error {
 	repoIdGen := didgen.NewDomainIdGenerator(&models.BitbucketRepo{})
 
 	converter, err := api.NewDataConverter(api.DataConverterArgs{
-		InputRowType: reflect.TypeOf(models.BitbucketRepo{}),
-		Input:        cursor,
-		RawDataSubTaskArgs: api.RawDataSubTaskArgs{
-			Ctx: taskCtx,
-			Params: BitbucketApiParams{
-				ConnectionId: data.Options.ConnectionId,
-				Owner:        data.Options.Owner,
-				Repo:         data.Options.Repo,
-			},
-			Table: RAW_REPOSITORIES_TABLE,
-		},
+		InputRowType:       reflect.TypeOf(models.BitbucketRepo{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
 		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
 			repository := inputRow.(*models.BitbucketRepo)
 			domainRepository := &code.Repo{
@@ -77,7 +159,7 @@ func ConvertRepo(taskCtx plugin.SubTaskContext) errors.Error {
 				Url:         repository.HTMLUrl,
 				Description: repository.Description,
 				Language:    repository.Language,
-				CreatedDate: &repository.CreatedDate,
+				CreatedDate: repository.CreatedDate,
 				UpdatedDate: repository.UpdatedDate,
 			}
 
@@ -88,7 +170,7 @@ func ConvertRepo(taskCtx plugin.SubTaskContext) errors.Error {
 				Name:        repository.Name,
 				Url:         fmt.Sprintf("%s/%s", repository.HTMLUrl, "issues"),
 				Description: repository.Description,
-				CreatedDate: &repository.CreatedDate,
+				CreatedDate: repository.CreatedDate,
 			}
 
 			return []interface{}{
diff --git a/backend/plugins/bitbucket/tasks/repo_extractor.go b/backend/plugins/bitbucket/tasks/repo_extractor.go
deleted file mode 100644
index 0fe366e2e..000000000
--- a/backend/plugins/bitbucket/tasks/repo_extractor.go
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
-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"
-	"fmt"
-	"github.com/apache/incubator-devlake/core/errors"
-	plugin "github.com/apache/incubator-devlake/core/plugin"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
-	"time"
-)
-
-var ExtractApiRepoMeta = plugin.SubTaskMeta{
-	Name:        "extractApiRepo",
-	EntryPoint:  ExtractApiRepositories,
-	Required:    true,
-	Description: "Extract raw Repositories data into tool layer table bitbucket_repos",
-	DomainTypes: []string{plugin.DOMAIN_TYPE_CODE},
-}
-
-type ApiRepoResponse BitbucketApiRepo
-
-type BitbucketApiRepo struct {
-	BitbucketId string
-	Scm         string                  `json:"scm"`
-	HasWiki     bool                    `json:"has_wiki"`
-	Uuid        string                  `json:"uuid"`
-	FullName    string                  `json:"full_name"`
-	Language    string                  `json:"language"`
-	Description string                  `json:"description"`
-	Type        string                  `json:"type"`
-	HasIssue    bool                    `json:"has_issue"`
-	ForkPolicy  string                  `json:"fork_policy"`
-	Owner       models.BitbucketAccount `json:"owner"`
-	CreatedAt   time.Time               `json:"created_on"`
-	UpdatedAt   *time.Time              `json:"updated_on"`
-	Links       struct {
-		Clone []struct {
-			Href string `json:"href"`
-			Name string `json:"name"`
-		} `json:"clone"`
-		Self struct {
-			Href string `json:"href"`
-		} `json:"self"`
-		Html struct {
-			Href string `json:"href"`
-		} `json:"html"`
-	} `json:"links"`
-}
-
-func ExtractApiRepositories(taskCtx plugin.SubTaskContext) errors.Error {
-	data := taskCtx.GetData().(*BitbucketTaskData)
-	extractor, err := api.NewApiExtractor(api.ApiExtractorArgs{
-		RawDataSubTaskArgs: api.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_REPOSITORIES_TABLE,
-		},
-		Extract: func(row *api.RawData) ([]interface{}, errors.Error) {
-			body := &ApiRepoResponse{}
-			err := errors.Convert(json.Unmarshal(row.Data, body))
-			if err != nil {
-				return nil, err
-			}
-			if body.FullName == "" {
-				return nil, errors.NotFound.New(fmt.Sprintf("repo %s/%s not found", data.Options.Owner, data.Options.Repo))
-			}
-			results := make([]interface{}, 0, 1)
-			bitbucketRepository := &models.BitbucketRepo{
-				ConnectionId: data.Options.ConnectionId,
-				BitbucketId:  body.FullName,
-				Name:         body.FullName,
-				HTMLUrl:      body.Links.Html.Href,
-				Description:  body.Description,
-				OwnerId:      body.Owner.AccountId,
-				Language:     body.Language,
-				CreatedDate:  body.CreatedAt,
-				UpdatedDate:  body.UpdatedAt,
-			}
-			data.Repo = bitbucketRepository
-
-			results = append(results, bitbucketRepository)
-
-			parentTaskContext := taskCtx.TaskContext()
-			if parentTaskContext != nil {
-				parentTaskContext.GetData().(*BitbucketTaskData).Repo = bitbucketRepository
-			}
-			return results, nil
-		},
-	})
-
-	if err != nil {
-		return err
-	}
-
-	return extractor.Execute()
-}
diff --git a/backend/plugins/bitbucket/tasks/task_data.go b/backend/plugins/bitbucket/tasks/task_data.go
index 4a8150878..4f447c899 100644
--- a/backend/plugins/bitbucket/tasks/task_data.go
+++ b/backend/plugins/bitbucket/tasks/task_data.go
@@ -26,37 +26,57 @@ import (
 )
 
 type BitbucketOptions struct {
-	ConnectionId               uint64   `json:"connectionId"`
-	Tasks                      []string `json:"tasks,omitempty"`
-	Since                      string
-	Owner                      string
-	Repo                       string
-	models.TransformationRules `mapstructure:"transformationRules" json:"transformationRules"`
+	ConnectionId                        uint64   `json:"connectionId" mapstructure:"connectionId,omitempty"`
+	Tasks                               []string `json:"tasks,omitempty" mapstructure:",omitempty"`
+	FullName                            string   `json:"fullName" mapstructure:"fullName"`
+	CreatedDateAfter                    string   `json:"createdDateAfter" mapstructure:"createdDateAfter,omitempty"`
+	TransformationRuleId                uint64   `json:"transformationRuleId" mapstructure:"transformationRuleId,omitempty"`
+	*models.BitbucketTransformationRule `mapstructure:"transformationRules,omitempty" json:"transformationRules"`
 }
 
 type BitbucketTaskData struct {
-	Options   *BitbucketOptions
-	ApiClient *api.ApiAsyncClient
-	Since     *time.Time
-	Repo      *models.BitbucketRepo
+	Options          *BitbucketOptions
+	ApiClient        *api.ApiAsyncClient
+	CreatedDateAfter *time.Time
 }
 
 func DecodeAndValidateTaskOptions(options map[string]interface{}) (*BitbucketOptions, errors.Error) {
+	op, err := DecodeTaskOptions(options)
+	if err != nil {
+		return nil, err
+	}
+	err = ValidateTaskOptions(op)
+	if err != nil {
+		return nil, err
+	}
+	return op, nil
+}
+
+func DecodeTaskOptions(options map[string]interface{}) (*BitbucketOptions, errors.Error) {
 	var op BitbucketOptions
 	err := api.Decode(options, &op, nil)
 	if err != nil {
-		return nil, errors.Default.New("could not decode options for Bitbucket execution")
-	}
-	if op.Owner == "" {
-		return nil, errors.Default.New("owner is required for Bitbucket execution")
+		return nil, err
 	}
-	if op.Repo == "" {
-		return nil, errors.Default.New("repo is required for Bitbucket execution")
+	return &op, nil
+}
+
+func EncodeTaskOptions(op *BitbucketOptions) (map[string]interface{}, errors.Error) {
+	var result map[string]interface{}
+	err := api.Decode(op, &result, nil)
+	if err != nil {
+		return nil, err
 	}
+	return result, nil
+}
 
+func ValidateTaskOptions(op *BitbucketOptions) errors.Error {
+	if op.FullName == "" {
+		return errors.BadInput.New("no enough info for Bitbucket execution")
+	}
 	// find the needed Bitbucket now
 	if op.ConnectionId == 0 {
-		return nil, errors.Default.New("Bitbucket connectionId is invalid")
+		return errors.BadInput.New("connectionId is invalid")
 	}
-	return &op, nil
+	return nil
 }
diff --git a/backend/plugins/gitlab/api/remote.go b/backend/plugins/gitlab/api/remote.go
index 4478158ba..eff56d13a 100644
--- a/backend/plugins/gitlab/api/remote.go
+++ b/backend/plugins/gitlab/api/remote.go
@@ -148,6 +148,9 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 
 		resBody := []GroupResponse{}
 		err = api.UnmarshalResponse(res, &resBody)
+		if err != nil {
+			return nil, err
+		}
 
 		// append group to output
 		for _, group := range resBody {
@@ -184,9 +187,6 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 		}
 
 		// check groups count
-		if err != nil {
-			return nil, err
-		}
 		if len(resBody) < pageData.PerPage {
 			pageData.Tag = TypeProject
 			pageData.Page = 1
@@ -212,6 +212,9 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 
 		resBody := []tasks.GitlabApiProject{}
 		err = api.UnmarshalResponse(res, &resBody)
+		if err != nil {
+			return nil, err
+		}
 
 		// append project to output
 		for _, project := range resBody {
@@ -230,9 +233,6 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 		}
 
 		// check project count
-		if err != nil {
-			return nil, err
-		}
 		if len(resBody) < pageData.PerPage {
 			pageData = nil
 		}
@@ -259,7 +259,7 @@ func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, er
 // @Tags plugins/gitlab
 // @Accept application/json
 // @Param connectionId path int false "connection ID"
-// @Param search query string false "group ID"
+// @Param search query string false "search"
 // @Param page query int false "page number"
 // @Param pageSize query int false "page size per page"
 // @Success 200  {object} SearchRemoteScopesOutput
diff --git a/config-ui/src/pages/pipeline/components/task/index.tsx b/config-ui/src/pages/pipeline/components/task/index.tsx
index d24f3ab2f..9237edc5f 100644
--- a/config-ui/src/pages/pipeline/components/task/index.tsx
+++ b/config-ui/src/pages/pipeline/components/task/index.tsx
@@ -55,7 +55,10 @@ export const PipelineTask = ({ task }: Props) => {
         name = `${name}:${options.projectName}`;
         break;
       case ['gitlab'].includes(config.plugin):
-        name = `${name}:projectId:${options.projectId}`;
+        name = `${name}:id:${options.projectId}`;
+        break;
+      case ['bitbucket'].includes(config.plugin):
+        name = `${name}:${options.fullName}`;
         break;
       case ['jira', 'jenkins'].includes(config.plugin):
         name = `${name}:${options.scopeId}`;
diff --git a/config-ui/src/pages/transformation/detail/index.tsx b/config-ui/src/pages/transformation/detail/index.tsx
index df181a3dd..47d7fb3d1 100644
--- a/config-ui/src/pages/transformation/detail/index.tsx
+++ b/config-ui/src/pages/transformation/detail/index.tsx
@@ -24,6 +24,7 @@ import { PageLoading, PageHeader, Card } from '@/components';
 import { GitHubTransformation } from '@/plugins/register/github';
 import { GitLabTransformation } from '@/plugins/register/gitlab';
 import { JenkinsTransformation } from '@/plugins/register/jenkins';
+import { BitbucketTransformation } from '@/plugins/register/bitbucket';
 
 import { useDetail } from './use-detail';
 import * as S from './styled';
@@ -78,6 +79,10 @@ export const TransformationDetailPage = () => {
             <JenkinsTransformation transformation={transformation} setTransformation={onChangeTransformation} />
           )}
 
+          {plugin === 'bitbucket' && (
+            <BitbucketTransformation transformation={transformation} setTransformation={onChangeTransformation} />
+          )}
+
           <div className="action">
             <ButtonGroup>
               <Button disabled={operating} outlined text="Cancel" onClick={() => history.push('/transformations')} />
diff --git a/config-ui/src/plugins/components/data-scope-list/use-data-scope-list.ts b/config-ui/src/plugins/components/data-scope-list/use-data-scope-list.ts
index 079be9a98..5cf3be985 100644
--- a/config-ui/src/plugins/components/data-scope-list/use-data-scope-list.ts
+++ b/config-ui/src/plugins/components/data-scope-list/use-data-scope-list.ts
@@ -64,6 +64,12 @@ export const useDataScopeList = ({ plugin, connectionId, scopeIds }: UseDataScop
             name: sc.jobFullName,
             transformationRuleName: sc.transformationRuleName,
           };
+        case plugin === 'bitbucket':
+          return {
+            id: sc.bitbucketId,
+            name: sc.name,
+            transformationRuleName: sc.transformationRuleName,
+          };
         default:
           return {
             id: sc.id,
diff --git a/config-ui/src/plugins/components/data-scope/index.tsx b/config-ui/src/plugins/components/data-scope/index.tsx
index 75f92a3fe..ffb5d1c05 100644
--- a/config-ui/src/plugins/components/data-scope/index.tsx
+++ b/config-ui/src/plugins/components/data-scope/index.tsx
@@ -29,6 +29,7 @@ import { MultiSelector } from '@/components';
 import type { UseDataScope } from './use-data-scope';
 import { useDataScope } from './use-data-scope';
 import * as S from './styled';
+import {BitbucketDataScope} from "@/plugins/register/bitbucket";
 
 interface Props extends UseDataScope {
   onCancel?: () => void;
@@ -60,6 +61,10 @@ export const DataScope = ({ plugin, connectionId, entities, onCancel, ...props }
         {plugin === 'jenkins' && (
           <JenkinsDataScope connectionId={connectionId} selectedItems={selectedScope} onChangeItems={onChangeScope} />
         )}
+
+        {plugin === 'bitbucket' && (
+          <BitbucketDataScope connectionId={connectionId} selectedItems={selectedScope} onChangeItems={onChangeScope} />
+        )}
       </div>
 
       <div className="block">
diff --git a/config-ui/src/plugins/components/data-scope/use-data-scope.ts b/config-ui/src/plugins/components/data-scope/use-data-scope.ts
index 0c4cf3cf3..a6bc7b0a6 100644
--- a/config-ui/src/plugins/components/data-scope/use-data-scope.ts
+++ b/config-ui/src/plugins/components/data-scope/use-data-scope.ts
@@ -63,6 +63,8 @@ export const useDataScope = ({ plugin, connectionId, entities, initialValues, on
         return scope.gitlabId;
       case plugin === 'jenkins':
         return scope.jobFullName;
+      case plugin === 'bitbucket':
+        return scope.bitbucketId;
     }
   };
 
diff --git a/config-ui/src/plugins/components/transformation/index.tsx b/config-ui/src/plugins/components/transformation/index.tsx
index 6c6bc53bb..959ad2e3b 100644
--- a/config-ui/src/plugins/components/transformation/index.tsx
+++ b/config-ui/src/plugins/components/transformation/index.tsx
@@ -31,6 +31,7 @@ import { TIPS_MAP } from './misc';
 import type { UseTransformationProps } from './use-transformation';
 import { useTransformation } from './use-transformation';
 import * as S from './styled';
+import {BitbucketTransformation} from "@/plugins/register/bitbucket";
 
 interface Props extends Omit<UseTransformationProps, 'name' | 'selectedRule' | 'setSelectedScope'> {
   from: 'create' | 'update';
@@ -165,6 +166,10 @@ export const Transformation = ({ from, plugin, connectionId, onCancel, ...props
               <JenkinsTransformation transformation={transformation} setTransformation={onChangeTransformation} />
             )}
 
+            {plugin === 'bitbucket' && (
+              <BitbucketTransformation transformation={transformation} setTransformation={onChangeTransformation} />
+            )}
+
             <ButtonGroup>
               <Button outlined intent={Intent.PRIMARY} text="Cancel and Go Back" onClick={onCancel} />
               <Button outlined disabled={!name} loading={saving} intent={Intent.PRIMARY} text="Save" onClick={onSave} />
diff --git a/config-ui/src/plugins/components/transformation/use-transformation.ts b/config-ui/src/plugins/components/transformation/use-transformation.ts
index f7231692c..1f29f325c 100644
--- a/config-ui/src/plugins/components/transformation/use-transformation.ts
+++ b/config-ui/src/plugins/components/transformation/use-transformation.ts
@@ -144,6 +144,8 @@ export const useTransformation = ({
             return sc.gitlabId;
           case plugin === 'jenkins':
             return sc.jobFullName;
+          case plugin === 'bitbucket':
+            return sc.bitbucketId;
         }
       },
       onSave: handleSave,
diff --git a/config-ui/src/plugins/register/bitbucket/config.tsx b/config-ui/src/plugins/register/bitbucket/config.tsx
index 3582ee64b..b951de53f 100644
--- a/config-ui/src/plugins/register/bitbucket/config.tsx
+++ b/config-ui/src/plugins/register/bitbucket/config.tsx
@@ -50,14 +50,18 @@ export const BitBucketConfig: PluginConfigType = {
         learnMore: 'https://devlake.apache.org/docs/Configuration/BitBucket#fixed-rate-limit-optional',
         externalInfo:
           'The maximum rate limit for different entities in BitBucket Cloud is 60,000 or 1,000 requests/hour.',
-        defaultValue: 10000,
+        defaultValue: 1000,
       },
     ],
   },
-  entities: ['TICKET', 'CROSS'],
+  entities: ['CODE', 'TICKET', 'CODEREVIEW', 'CROSS', 'CICD'],
   transformation: {
-    storyPointField: '',
-    remotelinkCommitShaPattern: '',
-    typeMappings: {},
+    issueStatusTodo: 'new,open',
+    issueStatusInProgress: '',
+    issueStatusDone: 'resolved,closed',
+    issueStatusOther: 'on hold,wontfix,duplicate,invalid',
+    productionPattern: '',
+    deploymentPattern: '',
+    refdiff: null,
   },
 };
diff --git a/config-ui/src/plugins/register/bitbucket/data-scope.tsx b/config-ui/src/plugins/register/bitbucket/data-scope.tsx
new file mode 100644
index 000000000..c1a96b178
--- /dev/null
+++ b/config-ui/src/plugins/register/bitbucket/data-scope.tsx
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ *
+ */
+
+import React, { useMemo } from 'react';
+
+import { DataScopeMillerColumns, DataScopeSearch } from '@/plugins';
+
+import type { ScopeItemType } from './types';
+
+interface Props {
+  connectionId: ID;
+  selectedItems: ScopeItemType[];
+  onChangeItems: (selectedItems: ScopeItemType[]) => void;
+}
+
+export const BitbucketDataScope = ({ connectionId, onChangeItems, ...props }: Props) => {
+  const selectedItems = useMemo(
+    () => props.selectedItems.map((it) => ({ id: `${it.bitbucketId}`, name: it.name, data: it })),
+    [props.selectedItems],
+  );
+
+  return (
+    <>
+      <h3>Repositories *</h3>
+      <p>Select the repositories you would like to sync.</p>
+      <DataScopeMillerColumns
+        plugin="bitbucket"
+        connectionId={connectionId}
+        selectedItems={selectedItems}
+        onChangeItems={onChangeItems}
+      />
+    </>
+  );
+};
diff --git a/config-ui/src/plugins/register/bitbucket/index.ts b/config-ui/src/plugins/register/bitbucket/index.ts
index de415db39..14fdb67cd 100644
--- a/config-ui/src/plugins/register/bitbucket/index.ts
+++ b/config-ui/src/plugins/register/bitbucket/index.ts
@@ -17,3 +17,5 @@
  */
 
 export * from './config';
+export * from './data-scope';
+export * from './transformation';
diff --git a/config-ui/src/plugins/register/bitbucket/styled.ts b/config-ui/src/plugins/register/bitbucket/styled.ts
new file mode 100644
index 000000000..876b88901
--- /dev/null
+++ b/config-ui/src/plugins/register/bitbucket/styled.ts
@@ -0,0 +1,89 @@
+/*
+ * 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.
+ *
+ */
+
+import styled from 'styled-components';
+
+export const TransformationWrapper = styled.div`
+  .issue-tracking {
+    .issue-type {
+      .title {
+        margin-bottom: 8px;
+      }
+
+      .list {
+        padding-left: 40px;
+      }
+    }
+  }
+
+  .ci-cd {
+    h3 {
+      margin-top: 16px;
+      margin-bottom: 8px;
+    }
+
+    .radio {
+      padding-left: 20px;
+    }
+  }
+
+  .additional-settings {
+    h2 {
+      display: flex;
+      align-items: center;
+      cursor: pointer;
+    }
+
+    .radio {
+      display: flex;
+      align-items: center;
+      margin: 8px 0 16px;
+
+      p {
+        margin: 0;
+      }
+
+      .bp4-control {
+        margin: 0;
+      }
+    }
+
+    .refdiff {
+      display: flex;
+      align-items: center;
+      padding-left: 20px;
+
+      .bp4-input-group {
+        margin: 0 8px;
+      }
+    }
+  }
+
+  .bp4-form-group {
+    display: flex;
+    align-items: center;
+
+    .bp4-label {
+      flex: 0 0 140px;
+    }
+
+    .bp4-form-content {
+      flex: auto;
+    }
+  }
+`;
diff --git a/config-ui/src/plugins/register/bitbucket/transformation.tsx b/config-ui/src/plugins/register/bitbucket/transformation.tsx
new file mode 100644
index 000000000..3a28b3bf2
--- /dev/null
+++ b/config-ui/src/plugins/register/bitbucket/transformation.tsx
@@ -0,0 +1,274 @@
+/*
+ * 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.
+ *
+ */
+
+import React, {useMemo, useState} from 'react';
+import {
+  FormGroup,
+  InputGroup,
+  TextArea,
+  Tag,
+  RadioGroup,
+  Radio,
+  Icon,
+  Collapse,
+  Intent,
+  Colors,
+  TagInput,
+} from '@blueprintjs/core';
+
+import { ExternalLink, HelpTooltip, Divider, MultiSelector } from '@/components';
+
+import * as S from './styled';
+
+interface Props {
+  transformation: any;
+  setTransformation: React.Dispatch<React.SetStateAction<any>>;
+}
+
+const ALL_STATES = ['new', 'open', 'resolved', 'closed', 'on hold', 'wontfix', 'duplicate', 'invalid'];
+
+export const BitbucketTransformation = ({ transformation, setTransformation }: Props) => {
+  const [enableCICD, setEnableCICD] = useState(1);
+  const [openAdditionalSettings, setOpenAdditionalSettings] = useState(false);
+  const selectedStates = useMemo(() => [
+    ...transformation.issueStatusTodo ? transformation.issueStatusTodo.split(',') : [],
+    ...transformation.issueStatusInProgress ? transformation.issueStatusInProgress.split(',') : [],
+    ...transformation.issueStatusDone ? transformation.issueStatusDone.split(',') : [],
+    ...transformation.issueStatusOther ? transformation.issueStatusOther.split(',') : [],
+  ], [transformation]);
+
+  const handleChangeCICDEnable = (e: number) => {
+    if (e === 0) {
+      setTransformation({
+        ...transformation,
+        deploymentPattern: undefined,
+        productionPattern: undefined,
+      });
+    } else {
+      setTransformation({
+        ...transformation,
+        deploymentPattern: '',
+        productionPattern: '',
+      });
+    }
+    setEnableCICD(e);
+  };
+
+  const handleChangeAdditionalSettingsOpen = () => {
+    setOpenAdditionalSettings(!openAdditionalSettings);
+    if (!openAdditionalSettings) {
+      setTransformation({
+        ...transformation,
+        refdiff: null,
+      });
+    }
+  };
+
+  return (
+    <S.TransformationWrapper>
+      {/* Issue Tracking */}
+      <div className="issue-tracking">
+        <h2>Issue Tracking</h2>
+        <div className="issue-type">
+          <div className="title">
+            <span>Issue Status Mapping</span>
+            <HelpTooltip content="Standardize your issue statuses to the following issue statuses to view metrics such as `Requirement Delivery Rate` in built-in dashboards." />
+          </div>
+          <div className="list">
+            <FormGroup inline label="TODO">
+              <MultiSelector
+                items={ALL_STATES}
+                disabledItems={selectedStates}
+                selectedItems={transformation.issueStatusTodo ? transformation.issueStatusTodo.split(',') : []}
+                onChangeItems={(selectedItems) =>
+                  setTransformation({
+                    ...transformation,
+                    issueStatusTodo: selectedItems.join(','),
+                  })
+                }
+              />
+            </FormGroup>
+            <FormGroup inline label="IN-PROGRESS">
+              <MultiSelector
+                items={ALL_STATES}
+                disabledItems={selectedStates}
+                selectedItems={
+                  transformation.issueStatusInProgress ? transformation.issueStatusInProgress.split(',') : []
+                }
+                onChangeItems={(selectedItems) =>
+                  setTransformation({
+                    ...transformation,
+                    issueStatusInProgress: selectedItems.join(','),
+                  })
+                }
+              />
+            </FormGroup>
+            <FormGroup inline label="DONE">
+              <MultiSelector
+                items={ALL_STATES}
+                disabledItems={selectedStates}
+                selectedItems={transformation.issueStatusDone ? transformation.issueStatusDone.split(',') : []}
+                onChangeItems={(selectedItems) =>
+                  setTransformation({
+                    ...transformation,
+                    issueStatusDone: selectedItems.join(','),
+                  })
+                }
+              />
+            </FormGroup>
+            <FormGroup inline label="OTHER">
+              <MultiSelector
+                items={ALL_STATES}
+                disabledItems={selectedStates}
+                selectedItems={transformation.issueStatusOther ? transformation.issueStatusOther.split(',') : []}
+                onChangeItems={(selectedItems) =>
+                  setTransformation({
+                    ...transformation,
+                    issueStatusOther: selectedItems.join(','),
+                  })
+                }
+              />
+            </FormGroup>
+          </div>
+        </div>
+      </div>
+      <Divider />
+      {/* CI/CD */}
+      <div className="ci-cd">
+        <h2>CI/CD</h2>
+        <h3>
+          <span>Deployment</span>
+          <Tag minimal intent={Intent.PRIMARY} style={{ marginLeft: 4, fontWeight: 400 }}>
+            DORA
+          </Tag>
+        </h3>
+        <p>Tell DevLake what CI jobs are Deployments.</p>
+        <RadioGroup
+          selectedValue={enableCICD}
+          onChange={(e) => handleChangeCICDEnable(+(e.target as HTMLInputElement).value)}
+        >
+          <Radio label="Detect Deployment from Builds in BitBucket" value={1} />
+          {enableCICD === 1 && (
+            <>
+              <p>
+                Not sure what a Bitbucket Action is?{' '}
+                <ExternalLink link="https://docs.github.com/en/actions/using-jobs/using-jobs-in-a-workflow">
+                  See it here
+                </ExternalLink>
+              </p>
+              <div className="radio">
+                <FormGroup
+                  inline
+                  label={
+                    <>
+                      <span>Deployment</span>
+                      <HelpTooltip content="A BitBucket build with a name that matches the RegEx will be considered as a deployment in DevLake." />
+                    </>
+                  }
+                >
+                  <InputGroup
+                    placeholder="(?i)deploy"
+                    value={transformation.deploymentPattern}
+                    onChange={(e) =>
+                      setTransformation({
+                        ...transformation,
+                        deploymentPattern: e.target.value,
+                      })
+                    }
+                  />
+                </FormGroup>
+                <FormGroup
+                  inline
+                  label={
+                    <>
+                      <span>Production</span>
+                      <HelpTooltip content="DevLake is only concerned with deployments in production environment when calculating DORA metrics. A BitBucket build with a name that matches the given RegEx will be considered as a job in the Production environment. If you leave this field empty, all data will be tagged as in the Production environment. " />
+                    </>
+                  }
+                >
+                  <InputGroup
+                    placeholder="(?i)production"
+                    value={transformation.productionPattern}
+                    onChange={(e) =>
+                      setTransformation({
+                        ...transformation,
+                        productionPattern: e.target.value,
+                      })
+                    }
+                  />
+                </FormGroup>
+              </div>
+            </>
+          )}
+          <Radio label="Not using any Bitbucket entities as Deployment" value={0} />
+        </RadioGroup>
+      </div>
+      <Divider />
+      {/* Additional Settings */}
+      <div className="additional-settings">
+        <h2 onClick={handleChangeAdditionalSettingsOpen}>
+          <Icon icon={openAdditionalSettings ? 'chevron-up' : 'chevron-down'} size={18} />
+          <span>Additional Settings</span>
+        </h2>
+        <Collapse isOpen={openAdditionalSettings}>
+          <div className="radio">
+            <Radio checked />
+            <p>
+              Enable the <ExternalLink link="https://devlake.apache.org/docs/Plugins/refdiff">RefDiff</ExternalLink>{' '}
+              plugin to pre-calculate version-based metrics
+              <HelpTooltip content="Calculate the commits diff between two consecutive tags that match the following RegEx. Issues closed by PRs which contain these commits will also be calculated. The result will be shown in table.refs_commits_diffs and table.refs_issues_diffs." />
+            </p>
+          </div>
+          <div className="refdiff">
+            Compare the last
+            <InputGroup
+              style={{ width: 60 }}
+              placeholder="10"
+              value={transformation.refdiff?.tagsOrder}
+              onChange={(e) =>
+                setTransformation({
+                  ...transformation,
+                  refdiff: {
+                    ...transformation?.refdiff,
+                    tagsOrder: e.target.value,
+                  },
+                })
+              }
+            />
+            tags that match the
+            <InputGroup
+              style={{ width: 200 }}
+              placeholder="v\d+\.\d+(\.\d+(-rc)*\d*)*$"
+              value={transformation.refdiff?.tagsPattern}
+              onChange={(e) =>
+                setTransformation({
+                  ...transformation,
+                  refdiff: {
+                    ...transformation?.refdiff,
+                    tagsPattern: e.target.value,
+                  },
+                })
+              }
+            />
+            for calculation
+          </div>
+        </Collapse>
+      </div>
+    </S.TransformationWrapper>
+  );
+};
diff --git a/config-ui/src/plugins/register/bitbucket/index.ts b/config-ui/src/plugins/register/bitbucket/types.ts
similarity index 88%
copy from config-ui/src/plugins/register/bitbucket/index.ts
copy to config-ui/src/plugins/register/bitbucket/types.ts
index de415db39..db949a1f9 100644
--- a/config-ui/src/plugins/register/bitbucket/index.ts
+++ b/config-ui/src/plugins/register/bitbucket/types.ts
@@ -16,4 +16,9 @@
  *
  */
 
-export * from './config';
+export type ScopeItemType = {
+  connectionId: ID;
+  bitbucketId: string;
+  name: string;
+  // and others
+};