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:13 UTC

[incubator-devlake] 05/12: feat:add zentao project extractor

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 34298a21aad879dac19bc884f53f6742ccdfd6ee
Author: yuqiangabab <11...@qq.com>
AuthorDate: Sun Sep 18 18:54:15 2022 +0800

    feat:add zentao project extractor
---
 plugins/zentao/impl/impl.go                        |  83 ++++++++-------
 plugins/zentao/models/archived/project.go          | 113 +++++++++++++++++++++
 .../migrationscripts/20220906_add_init_tables.go   |   1 +
 plugins/zentao/tasks/project_extractor.go          |  63 ++++++++++++
 4 files changed, 218 insertions(+), 42 deletions(-)

diff --git a/plugins/zentao/impl/impl.go b/plugins/zentao/impl/impl.go
index cc81c226..f24f6afe 100644
--- a/plugins/zentao/impl/impl.go
+++ b/plugins/zentao/impl/impl.go
@@ -21,11 +21,11 @@ import (
 	"fmt"
 	"github.com/apache/incubator-devlake/migration"
 	"github.com/apache/incubator-devlake/plugins/core"
-    "github.com/apache/incubator-devlake/plugins/zentao/api"
-    "github.com/apache/incubator-devlake/plugins/zentao/models"
-    "github.com/apache/incubator-devlake/plugins/zentao/models/migrationscripts"
-	"github.com/apache/incubator-devlake/plugins/zentao/tasks"
 	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/zentao/api"
+	"github.com/apache/incubator-devlake/plugins/zentao/models"
+	"github.com/apache/incubator-devlake/plugins/zentao/models/migrationscripts"
+	"github.com/apache/incubator-devlake/plugins/zentao/tasks"
 	"github.com/spf13/viper"
 	"gorm.io/gorm"
 )
@@ -38,8 +38,6 @@ var _ core.PluginApi = (*Zentao)(nil)
 var _ core.PluginBlueprintV100 = (*Zentao)(nil)
 var _ core.CloseablePluginTask = (*Zentao)(nil)
 
-
-
 type Zentao struct{}
 
 func (plugin Zentao) Description() string {
@@ -55,33 +53,34 @@ func (plugin Zentao) SubTaskMetas() []core.SubTaskMeta {
 	// TODO add your sub task here
 	return []core.SubTaskMeta{
 		tasks.CollectProjectMeta,
+		tasks.ExtractProjectsMeta,
 	}
 }
 
 func (plugin Zentao) PrepareTaskData(taskCtx core.TaskContext, options map[string]interface{}) (interface{}, error) {
 	op, err := tasks.DecodeAndValidateTaskOptions(options)
-    if err != nil {
-        return nil, err
-    }
-    connectionHelper := helper.NewConnectionHelper(
-        taskCtx,
-        nil,
-    )
-    connection := &models.ZentaoConnection{}
-    err = connectionHelper.FirstById(connection, op.ConnectionId)
-    if err != nil {
-        return nil, fmt.Errorf("unable to get Zentao connection by the given connection ID: %v", err)
-    }
-
-    apiClient, err := tasks.NewZentaoApiClient(taskCtx, connection)
-    if err != nil {
-        return nil, fmt.Errorf("unable to get Zentao API client instance: %v", err)
-    }
-
-    return &tasks.ZentaoTaskData{
-        Options:   op,
-        ApiClient: apiClient,
-    }, nil
+	if err != nil {
+		return nil, err
+	}
+	connectionHelper := helper.NewConnectionHelper(
+		taskCtx,
+		nil,
+	)
+	connection := &models.ZentaoConnection{}
+	err = connectionHelper.FirstById(connection, op.ConnectionId)
+	if err != nil {
+		return nil, fmt.Errorf("unable to get Zentao connection by the given connection ID: %v", err)
+	}
+
+	apiClient, err := tasks.NewZentaoApiClient(taskCtx, connection)
+	if err != nil {
+		return nil, fmt.Errorf("unable to get Zentao API client instance: %v", err)
+	}
+
+	return &tasks.ZentaoTaskData{
+		Options:   op,
+		ApiClient: apiClient,
+	}, nil
 }
 
 // PkgPath information lost when compiled as plugin(.so)
@@ -94,20 +93,20 @@ func (plugin Zentao) MigrationScripts() []migration.Script {
 }
 
 func (plugin Zentao) ApiResources() map[string]map[string]core.ApiResourceHandler {
-    return map[string]map[string]core.ApiResourceHandler{
-        "test": {
-            "POST": api.TestConnection,
-        },
-        "connections": {
-            "POST": api.PostConnections,
-            "GET":  api.ListConnections,
-        },
-        "connections/:connectionId": {
-            "GET":    api.GetConnection,
-            "PATCH":  api.PatchConnection,
-            "DELETE": api.DeleteConnection,
-        },
-    }
+	return map[string]map[string]core.ApiResourceHandler{
+		"test": {
+			"POST": api.TestConnection,
+		},
+		"connections": {
+			"POST": api.PostConnections,
+			"GET":  api.ListConnections,
+		},
+		"connections/:connectionId": {
+			"GET":    api.GetConnection,
+			"PATCH":  api.PatchConnection,
+			"DELETE": api.DeleteConnection,
+		},
+	}
 }
 
 func (plugin Zentao) MakePipelinePlan(connectionId uint64, scope []*core.BlueprintScopeV100) (core.PipelinePlan, error) {
diff --git a/plugins/zentao/models/archived/project.go b/plugins/zentao/models/archived/project.go
new file mode 100644
index 00000000..db79cceb
--- /dev/null
+++ b/plugins/zentao/models/archived/project.go
@@ -0,0 +1,113 @@
+/*
+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/common"
+	"time"
+)
+
+type ZentaoProject struct {
+	common.NoPKModel
+	ConnectionId  uint64 `gorm:"primaryKey;type:BIGINT  NOT NULL"`
+	ID            int    `json:"id"`
+	Project       int    `json:"project"`
+	Model         string `json:"model"`
+	Type          string `json:"type"`
+	Lifetime      string `json:"lifetime"`
+	Budget        string `json:"budget"`
+	BudgetUnit    string `json:"budgetUnit"`
+	Attribute     string `json:"attribute"`
+	Percent       int    `json:"percent"`
+	Milestone     string `json:"milestone"`
+	Output        string `json:"output"`
+	Auth          string `json:"auth"`
+	Parent        int    `json:"parent"`
+	Path          string `json:"path"`
+	Grade         int    `json:"grade"`
+	Name          string `json:"name"`
+	Code          string `json:"code"`
+	Begin         string `json:"begin"`
+	End           string `json:"end"`
+	RealBegan     string `json:"realBegan"`
+	RealEnd       string `json:"realEnd"`
+	Days          int    `json:"days"`
+	Status        string `json:"status"`
+	SubStatus     string `json:"subStatus"`
+	Pri           string `json:"pri"`
+	Desc          string `json:"desc"`
+	Version       int    `json:"version"`
+	ParentVersion int    `json:"parentVersion"`
+	PlanDuration  int    `json:"planDuration"`
+	RealDuration  int    `json:"realDuration"`
+	//OpenedBy       string    `json:"openedBy"`
+	OpenedDate     time.Time `json:"openedDate"`
+	OpenedVersion  string    `json:"openedVersion"`
+	LastEditedBy   string    `json:"lastEditedBy"`
+	LastEditedDate time.Time `json:"lastEditedDate"`
+	//ClosedBy       string    `json:"closedBy"`
+	//ClosedDate     string    `json:"closedDate"`
+	//CanceledBy string `json:"canceledBy"`
+	//CanceledDate   string    `json:"canceledDate"`
+	SuspendedDate string `json:"suspendedDate"`
+	PO            string `json:"PO"`
+	PM            `json:"PM"`
+	QD            string `json:"QD"`
+	RD            string `json:"RD"`
+	Team          string `json:"team"`
+	Acl           string `json:"acl"`
+	Whitelist     `json:"whitelist" gorm:"-"`
+	Order         int    `json:"order"`
+	Vision        string `json:"vision"`
+	DisplayCards  int    `json:"displayCards"`
+	FluidBoard    string `json:"fluidBoard"`
+	Deleted       bool   `json:"deleted"`
+	Delay         int    `json:"delay"`
+	Hours         `json:"hours"`
+	TeamCount     int    `json:"teamCount"`
+	LeftTasks     string `json:"leftTasks"`
+	//TeamMembers   []interface{} `json:"teamMembers" gorm:"-"`
+	TotalEstimate int `json:"totalEstimate"`
+	TotalConsumed int `json:"totalConsumed"`
+	TotalLeft     int `json:"totalLeft"`
+	Progress      int `json:"progress"`
+	TotalReal     int `json:"totalReal"`
+}
+type PM struct {
+	PmId       int    `json:"id"`
+	PmAccount  string `json:"account"`
+	PmAvatar   string `json:"avatar"`
+	PmRealname string `json:"realname"`
+}
+type Whitelist []struct {
+	WhitelistID       int    `json:"id"`
+	WhitelistAccount  string `json:"account"`
+	WhitelistAvatar   string `json:"avatar"`
+	WhitelistRealname string `json:"realname"`
+}
+type Hours struct {
+	HoursTotalEstimate int `json:"totalEstimate"`
+	HoursTotalConsumed int `json:"totalConsumed"`
+	HoursTotalLeft     int `json:"totalLeft"`
+	HoursProgress      int `json:"progress"`
+	HoursTotalReal     int `json:"totalReal"`
+}
+
+func (ZentaoProject) TableName() string {
+	return "_tool_zentao_project"
+}
diff --git a/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go b/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go
index fa9f9684..1a743c46 100644
--- a/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go
+++ b/plugins/zentao/models/migrationscripts/20220906_add_init_tables.go
@@ -28,6 +28,7 @@ type addInitTables struct{}
 func (u *addInitTables) Up(ctx context.Context, db *gorm.DB) error {
 	return db.Migrator().AutoMigrate(
 		archived.ZentaoConnection{},
+		archived.ZentaoProject{},
 	)
 }
 
diff --git a/plugins/zentao/tasks/project_extractor.go b/plugins/zentao/tasks/project_extractor.go
new file mode 100644
index 00000000..6cc9d9e5
--- /dev/null
+++ b/plugins/zentao/tasks/project_extractor.go
@@ -0,0 +1,63 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package 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/archived"
+)
+
+var _ core.SubTaskEntryPoint = ExtractProjects
+
+var ExtractProjectsMeta = core.SubTaskMeta{
+	Name:             "extractProjects",
+	EntryPoint:       ExtractProjects,
+	EnabledByDefault: true,
+	Description:      "extract Zentao projects",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func ExtractProjects(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*ZentaoTaskData)
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx:    taskCtx,
+			Params: ZentaoApiParams{},
+			Table:  RAW_PROJECT_TABLE,
+		},
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			project := &archived.ZentaoProject{}
+			err := json.Unmarshal(row.Data, project)
+			if err != nil {
+				return nil, err
+			}
+			project.ConnectionId = data.Options.ConnectionId
+			results := make([]interface{}, 0)
+			results = append(results, project)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}