You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by zh...@apache.org on 2022/09/16 12:35:53 UTC

[incubator-devlake] branch main updated: add Bitbucket Pipeline Collector&Extractor&Convertor (#3071)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new d7fe9329 add Bitbucket Pipeline Collector&Extractor&Convertor (#3071)
d7fe9329 is described below

commit d7fe9329bd975f067dd7e3f7535e2dfecc641adf
Author: tsoc <47...@users.noreply.github.com>
AuthorDate: Fri Sep 16 20:35:48 2022 +0800

    add Bitbucket Pipeline Collector&Extractor&Convertor (#3071)
    
    * fix: deal with conflict
    
    * fix: deal with conflict
    
    * fix: deal with conflict
    
    * fix: fix models in migrationscripts and filled field `durationsec` and `finish_date` etc.
    
    * fix: ignore HttpStatus403
    
    * fix: change statusRule and resultRule for bitbucket pipeline
    
    * fix: use archived model during migration
    
    * fix: add json for models in pipeline_extractor.go
---
 plugins/bitbucket/impl/impl.go                     |   3 +
 ...register.go => 20220914_add_pipeline_tables.go} |  24 +++-
 .../{register.go => archived/pipeline.go}          |  27 +++-
 .../bitbucket/models/migrationscripts/register.go  |   1 +
 .../{migrationscripts/register.go => pipeline.go}  |  27 +++-
 plugins/bitbucket/tasks/api_common.go              |   7 +
 plugins/bitbucket/tasks/pipeline_collector.go      |  56 ++++++++
 plugins/bitbucket/tasks/pipeline_convertor.go      | 108 +++++++++++++++
 plugins/bitbucket/tasks/pipeline_extractor.go      | 145 +++++++++++++++++++++
 9 files changed, 379 insertions(+), 19 deletions(-)

diff --git a/plugins/bitbucket/impl/impl.go b/plugins/bitbucket/impl/impl.go
index 8bb957b2..5f311f48 100644
--- a/plugins/bitbucket/impl/impl.go
+++ b/plugins/bitbucket/impl/impl.go
@@ -60,12 +60,15 @@ func (plugin Bitbucket) SubTaskMetas() []core.SubTaskMeta {
 		tasks.ExtractApiPrCommentsMeta,
 		tasks.CollectApiIssueCommentsMeta,
 		tasks.ExtractApiIssueCommentsMeta,
+		tasks.CollectApiPipelinesMeta,
+		tasks.ExtractApiPipelinesMeta,
 		tasks.ConvertRepoMeta,
 		tasks.ConvertAccountsMeta,
 		tasks.ConvertPullRequestsMeta,
 		tasks.ConvertPrCommentsMeta,
 		tasks.ConvertIssuesMeta,
 		tasks.ConvertIssueCommentsMeta,
+		tasks.ConvertPipelineMeta,
 	}
 }
 
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/migrationscripts/20220914_add_pipeline_tables.go
similarity index 58%
copy from plugins/bitbucket/models/migrationscripts/register.go
copy to plugins/bitbucket/models/migrationscripts/20220914_add_pipeline_tables.go
index c1365f7d..45db9377 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/migrationscripts/20220914_add_pipeline_tables.go
@@ -18,12 +18,26 @@ limitations under the License.
 package migrationscripts
 
 import (
-	"github.com/apache/incubator-devlake/migration"
+	"context"
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models/migrationscripts/archived"
+	"gorm.io/gorm"
 )
 
-// All return all the migration scripts
-func All() []migration.Script {
-	return []migration.Script{
-		new(addInitTables),
+type addPipeline20220914 struct{}
+
+func (*addPipeline20220914) Up(ctx context.Context, db *gorm.DB) errors.Error {
+	err := db.Migrator().AutoMigrate(&archived.BitbucketPipeline{})
+	if err != nil {
+		return errors.Convert(err)
 	}
+	return nil
+}
+
+func (*addPipeline20220914) Version() uint64 {
+	return 20220914111223
+}
+
+func (*addPipeline20220914) Name() string {
+	return "bitbucket add _tool_bitbucket_pipelines table"
 }
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/migrationscripts/archived/pipeline.go
similarity index 55%
copy from plugins/bitbucket/models/migrationscripts/register.go
copy to plugins/bitbucket/models/migrationscripts/archived/pipeline.go
index c1365f7d..0b9f56bc 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/migrationscripts/archived/pipeline.go
@@ -15,15 +15,28 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package migrationscripts
+package archived
 
 import (
-	"github.com/apache/incubator-devlake/migration"
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
+	"time"
 )
 
-// All return all the migration scripts
-func All() []migration.Script {
-	return []migration.Script{
-		new(addInitTables),
-	}
+type BitbucketPipeline struct {
+	ConnectionId      uint64 `gorm:"primaryKey"`
+	BitbucketId       string `gorm:"primaryKey"`
+	Status            string `gorm:"type:varchar(100)"`
+	Result            string `gorm:"type:varchar(100)"`
+	RefName           string `gorm:"type:varchar(255)"`
+	WebUrl            string `gorm:"type:varchar(255)"`
+	DurationInSeconds uint64
+
+	BitbucketCreatedOn  *time.Time
+	BitbucketCompleteOn *time.Time
+
+	archived.NoPKModel
+}
+
+func (BitbucketPipeline) TableName() string {
+	return "_tool_bitbucket_pipelines"
 }
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/migrationscripts/register.go
index c1365f7d..7218481b 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/migrationscripts/register.go
@@ -25,5 +25,6 @@ import (
 func All() []migration.Script {
 	return []migration.Script{
 		new(addInitTables),
+		new(addPipeline20220914),
 	}
 }
diff --git a/plugins/bitbucket/models/migrationscripts/register.go b/plugins/bitbucket/models/pipeline.go
similarity index 56%
copy from plugins/bitbucket/models/migrationscripts/register.go
copy to plugins/bitbucket/models/pipeline.go
index c1365f7d..12db320f 100644
--- a/plugins/bitbucket/models/migrationscripts/register.go
+++ b/plugins/bitbucket/models/pipeline.go
@@ -15,15 +15,28 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package migrationscripts
+package models
 
 import (
-	"github.com/apache/incubator-devlake/migration"
+	"github.com/apache/incubator-devlake/models/common"
+	"time"
 )
 
-// All return all the migration scripts
-func All() []migration.Script {
-	return []migration.Script{
-		new(addInitTables),
-	}
+type BitbucketPipeline struct {
+	ConnectionId      uint64 `gorm:"primaryKey"`
+	BitbucketId       string `gorm:"primaryKey"`
+	Status            string `gorm:"type:varchar(100)"`
+	Result            string `gorm:"type:varchar(100)"`
+	RefName           string `gorm:"type:varchar(255)"`
+	WebUrl            string `gorm:"type:varchar(255)"`
+	DurationInSeconds uint64
+
+	BitbucketCreatedOn  *time.Time
+	BitbucketCompleteOn *time.Time
+
+	common.NoPKModel
+}
+
+func (BitbucketPipeline) TableName() string {
+	return "_tool_bitbucket_pipelines"
 }
diff --git a/plugins/bitbucket/tasks/api_common.go b/plugins/bitbucket/tasks/api_common.go
index b8b487e1..000089da 100644
--- a/plugins/bitbucket/tasks/api_common.go
+++ b/plugins/bitbucket/tasks/api_common.go
@@ -154,3 +154,10 @@ func ignoreHTTPStatus404(res *http.Response) errors.Error {
 	}
 	return nil
 }
+
+func ignoreHTTPStatus403(res *http.Response) errors.Error {
+	if res.StatusCode == http.StatusForbidden {
+		return helper.ErrIgnoreAndContinue
+	}
+	return nil
+}
diff --git a/plugins/bitbucket/tasks/pipeline_collector.go b/plugins/bitbucket/tasks/pipeline_collector.go
new file mode 100644
index 00000000..c4284e7d
--- /dev/null
+++ b/plugins/bitbucket/tasks/pipeline_collector.go
@@ -0,0 +1,56 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+const RAW_PIPELINE_TABLE = "bitbucket_api_pipelines"
+
+var CollectApiPipelinesMeta = core.SubTaskMeta{
+	Name:             "collectApiPipelines",
+	EntryPoint:       CollectApiPipelines,
+	EnabledByDefault: true,
+	Description:      "Collect pipeline data from bitbucket api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CICD},
+}
+
+func CollectApiPipelines(taskCtx core.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PIPELINE_TABLE)
+
+	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,
+		AfterResponse:      ignoreHTTPStatus403, // ignore 403 for CI/CD disable
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
diff --git a/plugins/bitbucket/tasks/pipeline_convertor.go b/plugins/bitbucket/tasks/pipeline_convertor.go
new file mode 100644
index 00000000..e3caf9b1
--- /dev/null
+++ b/plugins/bitbucket/tasks/pipeline_convertor.go
@@ -0,0 +1,108 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"github.com/apache/incubator-devlake/errors"
+	"reflect"
+	"time"
+
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/devops"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	bitbucketModels "github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertPipelineMeta = core.SubTaskMeta{
+	Name:             "convertPipelines",
+	EntryPoint:       ConvertPipelines,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table bitbucket_pipeline into domain layer table pipeline",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CROSS},
+}
+
+func ConvertPipelines(taskCtx core.SubTaskContext) errors.Error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*BitbucketTaskData)
+
+	cursor, err := db.Cursor(dal.From(bitbucketModels.BitbucketPipeline{}))
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	pipelineIdGen := didgen.NewDomainIdGenerator(&bitbucketModels.BitbucketPipeline{})
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		InputRowType: reflect.TypeOf(bitbucketModels.BitbucketPipeline{}),
+		Input:        cursor,
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: BitbucketApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_PIPELINE_TABLE,
+		},
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			bitbucketPipeline := inputRow.(*bitbucketModels.BitbucketPipeline)
+
+			createdAt := time.Now()
+			if bitbucketPipeline.BitbucketCreatedOn != nil {
+				createdAt = *bitbucketPipeline.BitbucketCreatedOn
+			}
+
+			domainPipeline := &devops.CICDPipeline{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: pipelineIdGen.Generate(data.Options.ConnectionId, bitbucketPipeline.BitbucketId),
+				},
+				Name: didgen.NewDomainIdGenerator(&bitbucketModels.BitbucketPipeline{}).
+					Generate(data.Options.ConnectionId, bitbucketPipeline.RefName),
+				Result: devops.GetResult(&devops.ResultRule{
+					Failed:  []string{"FAILED", "ERROR"},
+					Abort:   []string{"STOPPED", "SKIPPED"},
+					Success: []string{"SUCCESSFUL", "PASSED"},
+					Manual:  []string{"PAUSED", "HALTED"},
+					Default: devops.SUCCESS,
+				}, bitbucketPipeline.Result),
+				Status: devops.GetStatus(&devops.StatusRule{
+					InProgress: []string{"IN_PROGRESS"},
+					Default:    devops.DONE,
+				}, bitbucketPipeline.Status),
+				Type:         "CI/CD",
+				CreatedDate:  createdAt,
+				DurationSec:  bitbucketPipeline.DurationInSeconds,
+				FinishedDate: bitbucketPipeline.BitbucketCompleteOn,
+			}
+
+			return []interface{}{
+				domainPipeline,
+			}, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/bitbucket/tasks/pipeline_extractor.go b/plugins/bitbucket/tasks/pipeline_extractor.go
new file mode 100644
index 00000000..26ce73d6
--- /dev/null
+++ b/plugins/bitbucket/tasks/pipeline_extractor.go
@@ -0,0 +1,145 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/bitbucket/models"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"time"
+)
+
+type bitbucketApiCommit struct {
+	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 bitbucketApiPipelineTarget struct {
+	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"`
+	State struct {
+		Name   string `json:"name"`
+		Type   string `json:"type"`
+		Result *struct {
+			Name string `json:"name"`
+			Type string `json:"type"`
+		} `json:"result"`
+		Stage *struct {
+			Name string `json:"name"`
+			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         *time.Time `json:"created_on"`
+	CompletedOn       *time.Time `json:"completed_on"`
+	RunNumber         int        `json:"run_number"`
+	DurationInSeconds uint64     `json:"duration_in_seconds"`
+	BuildSecondsUsed  int        `json:"build_seconds_used"`
+	FirstSuccessful   bool       `json:"first_successful"`
+	Expired           bool       `json:"expired"`
+	HasVariables      bool       `json:"has_variables"`
+	Links             struct {
+		Self struct {
+			Href string `json:"href"`
+		} `json:"self"`
+		Steps struct {
+			Href string `json:"href"`
+		} `json:"steps"`
+	} `json:"links"`
+}
+
+var ExtractApiPipelinesMeta = core.SubTaskMeta{
+	Name:             "extractApiPipelines",
+	EntryPoint:       ExtractApiPipelines,
+	EnabledByDefault: true,
+	Description:      "Extract raw pipelines data into tool layer table BitbucketPipeline",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CICD},
+}
+
+func ExtractApiPipelines(taskCtx core.SubTaskContext) errors.Error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_PIPELINE_TABLE)
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Extract: func(row *helper.RawData) ([]interface{}, errors.Error) {
+			// create bitbucket commit
+			bitbucketApiPipeline := &BitbucketApiPipeline{}
+			err := errors.Convert(json.Unmarshal(row.Data, bitbucketApiPipeline))
+			if err != nil {
+				return nil, err
+			}
+
+			bitbucketPipeline := &models.BitbucketPipeline{
+				ConnectionId:        data.Options.ConnectionId,
+				BitbucketId:         bitbucketApiPipeline.Uuid,
+				WebUrl:              bitbucketApiPipeline.Links.Self.Href,
+				Status:              bitbucketApiPipeline.State.Name,
+				RefName:             bitbucketApiPipeline.Target.RefName,
+				DurationInSeconds:   bitbucketApiPipeline.DurationInSeconds,
+				BitbucketCreatedOn:  bitbucketApiPipeline.CreatedOn,
+				BitbucketCompleteOn: bitbucketApiPipeline.CompletedOn,
+			}
+			if err != nil {
+				return nil, err
+			}
+			if bitbucketApiPipeline.State.Result != nil {
+				bitbucketPipeline.Result = bitbucketApiPipeline.State.Result.Name
+			} else if bitbucketApiPipeline.State.Stage != nil {
+				bitbucketPipeline.Result = bitbucketApiPipeline.State.Stage.Name
+			}
+
+			results := make([]interface{}, 0, 2)
+			results = append(results, bitbucketPipeline)
+
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}