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 2022/10/17 06:20:20 UTC

[incubator-devlake] 12/12: feat:add zentao bugs

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

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

commit 6a500469fbd5df7d2b5c8abbf6a6bc078c6ac272
Author: yuqiangabab <11...@qq.com>
AuthorDate: Wed Oct 5 12:57:25 2022 +0800

    feat:add zentao bugs
---
 plugins/zentao/impl/impl.go                        |   3 +
 plugins/zentao/models/archived/bug.go              | 112 +++++++++++++++++++++
 plugins/zentao/models/bug.go                       | 112 +++++++++++++++++++++
 .../migrationscripts/20220906_add_init_tables.go   |   1 +
 plugins/zentao/tasks/bug_collector.go              |  84 ++++++++++++++++
 plugins/zentao/tasks/bug_convertor.go              | 110 ++++++++++++++++++++
 plugins/zentao/tasks/bug_extractor.go              |  67 ++++++++++++
 7 files changed, 489 insertions(+)

diff --git a/plugins/zentao/impl/impl.go b/plugins/zentao/impl/impl.go
index aaf3c922..8091e898 100644
--- a/plugins/zentao/impl/impl.go
+++ b/plugins/zentao/impl/impl.go
@@ -60,6 +60,9 @@ func (plugin Zentao) SubTaskMetas() []core.SubTaskMeta {
 		tasks.CollectStoriesMeta,
 		tasks.ExtractStoriesMeta,
 		tasks.ConvertStoriesMeta,
+		tasks.CollectBugMeta,
+		tasks.ExtractBugMeta,
+		tasks.ConvertBugMeta,
 	}
 }
 
diff --git a/plugins/zentao/models/archived/bug.go b/plugins/zentao/models/archived/bug.go
new file mode 100644
index 00000000..3743b761
--- /dev/null
+++ b/plugins/zentao/models/archived/bug.go
@@ -0,0 +1,112 @@
+/*
+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/models/migrationscripts/archived"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+type ZentaoBug struct {
+	archived.NoPKModel
+	ConnectionId   uint64   `gorm:"primaryKey;type:BIGINT  NOT NULL"`
+	ID             int      `json:"id" gorm:"primaryKey"`
+	Project        int      `json:"project"`
+	Product        int      `json:"product"`
+	Injection      int      `json:"injection"`
+	Identify       int      `json:"identify"`
+	Branch         int      `json:"branch"`
+	Module         int      `json:"module"`
+	Execution      int      `json:"execution"`
+	Plan           int      `json:"plan"`
+	Story          int      `json:"story"`
+	StoryVersion   int      `json:"storyVersion"`
+	Task           int      `json:"task"`
+	ToTask         int      `json:"toTask"`
+	ToStory        int      `json:"toStory"`
+	Title          string   `json:"title"`
+	Keywords       string   `json:"keywords"`
+	Severity       int      `json:"severity"`
+	Pri            int      `json:"pri"`
+	Type           string   `json:"type"`
+	Os             string   `json:"os"`
+	Browser        string   `json:"browser"`
+	Hardware       string   `json:"hardware"`
+	Found          string   `json:"found"`
+	Steps          string   `json:"steps"`
+	Status         string   `json:"status"`
+	SubStatus      string   `json:"subStatus"`
+	Color          string   `json:"color"`
+	Confirmed      int      `json:"confirmed"`
+	ActivatedCount int      `json:"activatedCount"`
+	ActivatedDate  string   `json:"activatedDate"`
+	FeedbackBy     string   `json:"feedbackBy"`
+	NotifyEmail    string   `json:"notifyEmail"`
+	Mailto         []Mailto `json:"mailto" gorm:"-:all"`
+	OpenedBy       `json:"openedBy"`
+	OpenedDate     *helper.Iso8601Time `json:"openedDate"`
+	OpenedBuild    string              `json:"openedBuild"`
+	AssignedTo     `json:"assignedTo"`
+	AssignedDate   *helper.Iso8601Time `json:"assignedDate"`
+	Deadline       string              `json:"deadline"`
+	ResolvedBy     `json:"resolvedBy"`
+	Resolution     string              `json:"resolution"`
+	ResolvedBuild  string              `json:"resolvedBuild"`
+	ResolvedDate   *helper.Iso8601Time `json:"resolvedDate"`
+	ClosedBy       `json:"closedBy"`
+	ClosedDate     *helper.Iso8601Time `json:"closedDate"`
+	DuplicateBug   int                 `json:"duplicateBug"`
+	LinkBug        string              `json:"linkBug"`
+	Case           int                 `json:"case"`
+	CaseVersion    int                 `json:"caseVersion"`
+	Feedback       int                 `json:"feedback"`
+	Result         int                 `json:"result"`
+	Repo           int                 `json:"repo"`
+	Mr             int                 `json:"mr"`
+	Entry          string              `json:"entry"`
+	Lines          string              `json:"lines"`
+	V1             string              `json:"v1"`
+	V2             string              `json:"v2"`
+	RepoType       string              `json:"repoType"`
+	IssueKey       string              `json:"issueKey"`
+	Testtask       int                 `json:"testtask"`
+	LastEditedBy   `json:"lastEditedBy"`
+	LastEditedDate *helper.Iso8601Time `json:"lastEditedDate"`
+	Deleted        bool                `json:"deleted"`
+	PriOrder       string              `json:"priOrder"`
+	SeverityOrder  int                 `json:"severityOrder"`
+	Needconfirm    bool                `json:"needconfirm"`
+	StatusName     string              `json:"statusName"`
+	ProductStatus  string              `json:"productStatus"`
+}
+type ResolvedBy struct {
+	ResolvedByID       int    `json:"id"`
+	ResolvedByAccount  string `json:"account"`
+	ResolvedByAvatar   string `json:"avatar"`
+	ResolvedByRealname string `json:"realname"`
+}
+type Mailto struct {
+	MailtoID       int    `json:"id"`
+	MailtoAccount  string `json:"account"`
+	MailtoAvatar   string `json:"avatar"`
+	MailtoRealname string `json:"realname"`
+}
+
+func (ZentaoBug) TableName() string {
+	return "_tool_zentao_bugs"
+}
diff --git a/plugins/zentao/models/bug.go b/plugins/zentao/models/bug.go
new file mode 100644
index 00000000..c081ed9c
--- /dev/null
+++ b/plugins/zentao/models/bug.go
@@ -0,0 +1,112 @@
+/*
+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/models/common"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+type ZentaoBug struct {
+	common.NoPKModel
+	ConnectionId   uint64   `gorm:"primaryKey;type:BIGINT  NOT NULL"`
+	ID             int      `json:"id" gorm:"primaryKey"`
+	Project        int      `json:"project"`
+	Product        int      `json:"product"`
+	Injection      int      `json:"injection"`
+	Identify       int      `json:"identify"`
+	Branch         int      `json:"branch"`
+	Module         int      `json:"module"`
+	Execution      int      `json:"execution"`
+	Plan           int      `json:"plan"`
+	Story          int      `json:"story"`
+	StoryVersion   int      `json:"storyVersion"`
+	Task           int      `json:"task"`
+	ToTask         int      `json:"toTask"`
+	ToStory        int      `json:"toStory"`
+	Title          string   `json:"title"`
+	Keywords       string   `json:"keywords"`
+	Severity       int      `json:"severity"`
+	Pri            int      `json:"pri"`
+	Type           string   `json:"type"`
+	Os             string   `json:"os"`
+	Browser        string   `json:"browser"`
+	Hardware       string   `json:"hardware"`
+	Found          string   `json:"found"`
+	Steps          string   `json:"steps"`
+	Status         string   `json:"status"`
+	SubStatus      string   `json:"subStatus"`
+	Color          string   `json:"color"`
+	Confirmed      int      `json:"confirmed"`
+	ActivatedCount int      `json:"activatedCount"`
+	ActivatedDate  string   `json:"activatedDate"`
+	FeedbackBy     string   `json:"feedbackBy"`
+	NotifyEmail    string   `json:"notifyEmail"`
+	Mailto         []Mailto `json:"mailto" gorm:"-:all"`
+	OpenedBy       `json:"openedBy"`
+	OpenedDate     *helper.Iso8601Time `json:"openedDate"`
+	OpenedBuild    string              `json:"openedBuild"`
+	AssignedTo     `json:"assignedTo"`
+	AssignedDate   *helper.Iso8601Time `json:"assignedDate"`
+	Deadline       string              `json:"deadline"`
+	ResolvedBy     `json:"resolvedBy"`
+	Resolution     string              `json:"resolution"`
+	ResolvedBuild  string              `json:"resolvedBuild"`
+	ResolvedDate   *helper.Iso8601Time `json:"resolvedDate"`
+	ClosedBy       `json:"closedBy"`
+	ClosedDate     *helper.Iso8601Time `json:"closedDate"`
+	DuplicateBug   int                 `json:"duplicateBug"`
+	LinkBug        string              `json:"linkBug"`
+	Case           int                 `json:"case"`
+	CaseVersion    int                 `json:"caseVersion"`
+	Feedback       int                 `json:"feedback"`
+	Result         int                 `json:"result"`
+	Repo           int                 `json:"repo"`
+	Mr             int                 `json:"mr"`
+	Entry          string              `json:"entry"`
+	Lines          string              `json:"lines"`
+	V1             string              `json:"v1"`
+	V2             string              `json:"v2"`
+	RepoType       string              `json:"repoType"`
+	IssueKey       string              `json:"issueKey"`
+	Testtask       int                 `json:"testtask"`
+	LastEditedBy   `json:"lastEditedBy"`
+	LastEditedDate *helper.Iso8601Time `json:"lastEditedDate"`
+	Deleted        bool                `json:"deleted"`
+	PriOrder       string              `json:"priOrder"`
+	SeverityOrder  int                 `json:"severityOrder"`
+	Needconfirm    bool                `json:"needconfirm"`
+	StatusName     string              `json:"statusName"`
+	ProductStatus  string              `json:"productStatus"`
+}
+type ResolvedBy struct {
+	ResolvedByID       int    `json:"id"`
+	ResolvedByAccount  string `json:"account"`
+	ResolvedByAvatar   string `json:"avatar"`
+	ResolvedByRealname string `json:"realname"`
+}
+type Mailto struct {
+	MailtoID       int    `json:"id"`
+	MailtoAccount  string `json:"account"`
+	MailtoAvatar   string `json:"avatar"`
+	MailtoRealname string `json:"realname"`
+}
+
+func (ZentaoBug) TableName() string {
+	return "_tool_zentao_bugs"
+}
diff --git a/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go b/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go
index 3d51dcde..ade33e29 100644
--- a/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go
+++ b/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go
@@ -31,6 +31,7 @@ func (u *addInitTables) Up(ctx context.Context, db *gorm.DB) error {
 		archived.ZentaoProject{},
 		archived.ZentaoExecution{},
 		archived.ZentaoStories{},
+		archived.ZentaoBug{},
 	)
 }
 
diff --git a/plugins/zentao/tasks/bug_collector.go b/plugins/zentao/tasks/bug_collector.go
new file mode 100644
index 00000000..d47c3bdd
--- /dev/null
+++ b/plugins/zentao/tasks/bug_collector.go
@@ -0,0 +1,84 @@
+/*
+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/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"io"
+	"net/http"
+	"net/url"
+)
+
+const RAW_BUG_TABLE = "zentao_bug"
+
+var _ core.SubTaskEntryPoint = CollectExecution
+
+func CollectBug(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*ZentaoTaskData)
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: ZentaoApiParams{
+				ProductId:   data.Options.ProductId,
+				ExecutionId: data.Options.ExecutionId,
+				ProjectId:   data.Options.ProjectId,
+			},
+			Table: RAW_BUG_TABLE,
+		},
+		ApiClient:   data.ApiClient,
+		Incremental: false,
+		PageSize:    100,
+		// TODO write which api would you want request
+		UrlTemplate: "/products/{{ .Params.ProductId }}/bugs",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("limit", fmt.Sprintf("%v", reqData.Pager.Size))
+			return query, nil
+		},
+		GetTotalPages: GetTotalPagesFromResponse,
+		ResponseParser: func(res *http.Response) ([]json.RawMessage, error) {
+			var data struct {
+				Bugs []json.RawMessage `json:"bugs"`
+			}
+			body, err := io.ReadAll(res.Body)
+			json.Unmarshal(body, &data)
+			res.Body.Close()
+			if err != nil {
+				return nil, err
+			}
+			return data.Bugs, nil
+			//return []json.RawMessage{body}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
+
+var CollectBugMeta = core.SubTaskMeta{
+	Name:             "CollectBug",
+	EntryPoint:       CollectBug,
+	EnabledByDefault: true,
+	Description:      "Collect Bug data from Zentao api",
+}
diff --git a/plugins/zentao/tasks/bug_convertor.go b/plugins/zentao/tasks/bug_convertor.go
new file mode 100644
index 00000000..beb99cfd
--- /dev/null
+++ b/plugins/zentao/tasks/bug_convertor.go
@@ -0,0 +1,110 @@
+/*
+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/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/zentao/models"
+	"reflect"
+)
+
+var _ core.SubTaskEntryPoint = ConvertExecutions
+
+var ConvertBugMeta = core.SubTaskMeta{
+	Name:             "convertBug",
+	EntryPoint:       ConvertBug,
+	EnabledByDefault: true,
+	Description:      "convert Zentao bug",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func ConvertBug(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*ZentaoTaskData)
+	db := taskCtx.GetDal()
+	boardIdGen := didgen.NewDomainIdGenerator(&models.ZentaoBug{})
+	cursor, err := db.Cursor(
+		dal.From(&models.ZentaoBug{}),
+		dal.Where(`_tool_zentao_bugs.product = ? and
+			_tool_zentao_bugs.connection_id = ?`, data.Options.ProductId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+	convertor, err := helper.NewDataConverter(helper.DataConverterArgs{
+		InputRowType: reflect.TypeOf(models.ZentaoBug{}),
+		Input:        cursor,
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: ZentaoApiParams{
+				ProductId:   data.Options.ProductId,
+				ExecutionId: data.Options.ExecutionId,
+				ProjectId:   data.Options.ProjectId,
+			},
+			Table: RAW_BUG_TABLE,
+		},
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			toolBug := inputRow.(*models.ZentaoBug)
+			domainBoard := &ticket.Issue{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: boardIdGen.Generate(toolBug.ConnectionId, toolBug.ID),
+				},
+				Url:                     "",
+				IconURL:                 "",
+				IssueKey:                toolBug.IssueKey,
+				Title:                   toolBug.Title,
+				Description:             "",
+				EpicKey:                 "",
+				Type:                    toolBug.Type,
+				Status:                  toolBug.Status,
+				OriginalStatus:          "",
+				StoryPoint:              0,
+				ResolutionDate:          toolBug.ResolvedDate.ToNullableTime(),
+				CreatedDate:             toolBug.OpenedDate.ToNullableTime(),
+				UpdatedDate:             toolBug.LastEditedDate.ToNullableTime(),
+				LeadTimeMinutes:         0,
+				ParentIssueId:           "",
+				Priority:                string(toolBug.Pri),
+				OriginalEstimateMinutes: 0,
+				TimeSpentMinutes:        0,
+				TimeRemainingMinutes:    0,
+				CreatorId:               string(toolBug.OpenedBy.OpenedByID),
+				CreatorName:             toolBug.OpenedBy.OpenedByRealname,
+				AssigneeId:              string(toolBug.AssignedTo.ID),
+				AssigneeName:            toolBug.AssignedTo.Realname,
+				Severity:                string(toolBug.Severity),
+				Component:               "",
+				DeploymentId:            "",
+			}
+			results := make([]interface{}, 0)
+			results = append(results, domainBoard)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return convertor.Execute()
+}
diff --git a/plugins/zentao/tasks/bug_extractor.go b/plugins/zentao/tasks/bug_extractor.go
new file mode 100644
index 00000000..4dbb3205
--- /dev/null
+++ b/plugins/zentao/tasks/bug_extractor.go
@@ -0,0 +1,67 @@
+/*
+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/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/zentao/models"
+)
+
+var _ core.SubTaskEntryPoint = ExtractBug
+
+var ExtractBugMeta = core.SubTaskMeta{
+	Name:             "extractBug",
+	EntryPoint:       ExtractBug,
+	EnabledByDefault: true,
+	Description:      "extract Zentao bug",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func ExtractBug(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*ZentaoTaskData)
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: ZentaoApiParams{
+				ProductId:   data.Options.ProductId,
+				ExecutionId: data.Options.ExecutionId,
+				ProjectId:   data.Options.ProjectId,
+			},
+			Table: RAW_BUG_TABLE,
+		},
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			bug := &models.ZentaoBug{}
+			err := json.Unmarshal(row.Data, bug)
+			if err != nil {
+				return nil, err
+			}
+			bug.ConnectionId = data.Options.ConnectionId
+			results := make([]interface{}, 0)
+			results = append(results, bug)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}