You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by ab...@apache.org on 2022/06/21 01:19:50 UTC

[incubator-devlake] 01/04: feat: multi-conn for tapd and update migrationscripts

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

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

commit 00afb63952e193b998ebe18b7ab56534a3707521
Author: abeizn <zi...@merico.dev>
AuthorDate: Mon Jun 20 18:41:48 2022 +0800

    feat: multi-conn for tapd and update migrationscripts
---
 plugins/tapd/api/connection.go                     | 170 +++-------------
 plugins/tapd/api/init.go                           |  11 +
 plugins/tapd/api/proxy.go                          |  80 --------
 plugins/tapd/{tapd.go => impl/impl.go}             |  61 +-----
 plugins/tapd/models/connection.go                  |  15 +-
 .../migrationscripts/archived/tapd_connection.go   |  30 ++-
 .../tapd/models/migrationscripts/init_schema.go    |   8 +-
 .../migrationscripts/update_schemas20220531.go     |  78 --------
 .../migrationscripts/update_schemas20220613.go     |  45 -----
 plugins/tapd/tapd.go                               | 221 +++------------------
 plugins/tapd/tasks/api_client.go                   |  10 +-
 11 files changed, 104 insertions(+), 625 deletions(-)

diff --git a/plugins/tapd/api/connection.go b/plugins/tapd/api/connection.go
index 96c6bbba..3cad748e 100644
--- a/plugins/tapd/api/connection.go
+++ b/plugins/tapd/api/connection.go
@@ -19,22 +19,17 @@ package api
 
 import (
 	"fmt"
-	"github.com/apache/incubator-devlake/config"
-	"github.com/apache/incubator-devlake/models/common"
-	"github.com/apache/incubator-devlake/plugins/tapd/models"
 	"net/http"
-	"strconv"
 	"time"
 
+	"github.com/apache/incubator-devlake/plugins/tapd/models"
+
 	"github.com/apache/incubator-devlake/plugins/helper"
-	"github.com/go-playground/validator/v10"
 	"github.com/mitchellh/mapstructure"
 
 	"github.com/apache/incubator-devlake/plugins/core"
 )
 
-var vld = validator.New()
-
 /*
 POST /plugins/tapd/test
 */
@@ -78,131 +73,27 @@ func TestConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, erro
 	return nil, nil
 }
 
-func findConnectionByInputParam(input *core.ApiResourceInput) (*models.TapdConnection, error) {
-	connectionId := input.Params["connectionId"]
-	if connectionId == "" {
-		return nil, fmt.Errorf("missing connectionsid")
-	}
-	tapdConnectionId, err := strconv.ParseUint(connectionId, 10, 64)
-	if err != nil {
-		return nil, fmt.Errorf("invalid connectionId")
-	}
-
-	return getTapdConnectionById(tapdConnectionId)
-}
-
-func getTapdConnectionById(id uint64) (*models.TapdConnection, error) {
-	tapdConnection := &models.TapdConnection{}
-	err := db.First(tapdConnection, id).Error
-	if err != nil {
-		return nil, err
-	}
-
-	// decrypt
-	v := config.GetConfig()
-	encKey := v.GetString(core.EncodeKeyEnvStr)
-	tapdConnection.BasicAuthEncoded, err = core.Decrypt(encKey, tapdConnection.BasicAuthEncoded)
-	if err != nil {
-		return nil, err
-	}
-
-	return tapdConnection, nil
-}
-
-func mergeFieldsToTapdConnection(tapdConnection *models.TapdConnection, connections ...map[string]interface{}) error {
-	// decode
-	for _, connections := range connections {
-		err := mapstructure.Decode(connections, tapdConnection)
-		if err != nil {
-			return err
-		}
-	}
-
-	// validate
-	vld := validator.New()
-	err := vld.Struct(tapdConnection)
-	if err != nil {
-		return err
-	}
-
-	return nil
-}
-
-func refreshAndSaveTapdConnection(tapdConnection *models.TapdConnection, data map[string]interface{}) error {
-	var err error
-	// update fields from request body
-	err = mergeFieldsToTapdConnection(tapdConnection, data)
-	if err != nil {
-		return err
-	}
-
-	// encrypt
-	v := config.GetConfig()
-	encKey := v.GetString(core.EncodeKeyEnvStr)
-	if encKey == "" {
-		// Randomly generate a bunch of encryption keys and set them to config
-		encKey = core.RandomEncKey()
-		v.Set(core.EncodeKeyEnvStr, encKey)
-		err := config.WriteConfig(v)
-		if err != nil {
-			return err
-		}
-	}
-	tapdConnection.BasicAuthEncoded, err = core.Encrypt(encKey, tapdConnection.BasicAuthEncoded)
-	if err != nil {
-		return err
-	}
-
-	// transaction for nested operations
-	tx := db.Begin()
-	defer func() {
-		if err != nil {
-			tx.Rollback()
-		} else {
-			tx.Commit()
-		}
-	}()
-	if tapdConnection.RateLimit == 0 {
-		tapdConnection.RateLimit = 6480
-	}
-	if tapdConnection.ID > 0 {
-		err = tx.Save(tapdConnection).Error
-	} else {
-		err = tx.Create(tapdConnection).Error
-	}
-	if err != nil {
-		if common.IsDuplicateError(err) {
-			return fmt.Errorf("tapd connections with name %s already exists", tapdConnection.Name)
-		}
-		return err
-	}
-	tapdConnection.BasicAuthEncoded, err = core.Decrypt(encKey, tapdConnection.BasicAuthEncoded)
-	if err != nil {
-		return err
-	}
-	return nil
-}
-
 /*
 POST /plugins/tapd/connections
 {
 	"name": "tapd data connections name",
 	"endpoint": "tapd api endpoint, i.e. https://merico.atlassian.net/rest",
-	"basicAuthEncoded": "generated by `echo -n <tapd login email>:<tapd token> | base64`",
+	"Token": "generated by `echo -n <tapd login email>:<tapd token> | base64`",
 	"rateLimit": 10800,
 }
 */
 func PostConnections(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
 	// create a new connections
-	tapdConnection := &models.TapdConnection{}
+	connection := &models.TapdConnection{}
 
 	// update from request and save to database
-	err := refreshAndSaveTapdConnection(tapdConnection, input.Body)
+	//err := refreshAndSaveTapdConnection(tapdConnection, input.Body)
+	err := connectionHelper.Create(connection, input)
 	if err != nil {
 		return nil, err
 	}
 
-	return &core.ApiResourceOutput{Body: tapdConnection, Status: http.StatusOK}, nil
+	return &core.ApiResourceOutput{Body: connection, Status: http.StatusOK}, nil
 }
 
 /*
@@ -210,53 +101,44 @@ PATCH /plugins/tapd/connections/:connectionId
 {
 	"name": "tapd data connections name",
 	"endpoint": "tapd api endpoint, i.e. https://merico.atlassian.net/rest",
-	"basicAuthEncoded": "generated by `echo -n <tapd login email>:<tapd token> | base64`",
+	"Token": "generated by `echo -n <tapd login email>:<tapd token> | base64`",
 	"rateLimit": 10800,
 }
 */
 func PatchConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
-	// load from db
-	tapdConnection, err := findConnectionByInputParam(input)
-	if err != nil {
-		return nil, err
-	}
-
-	// update from request and save to database
-	err = refreshAndSaveTapdConnection(tapdConnection, input.Body)
+	connection := &models.TapdConnection{}
+	err := connectionHelper.Patch(connection, input)
 	if err != nil {
 		return nil, err
 	}
 
-	return &core.ApiResourceOutput{Body: tapdConnection}, nil
+	return &core.ApiResourceOutput{Body: connection}, nil
 }
 
 /*
 DELETE /plugins/tapd/connections/:connectionId
 */
 func DeleteConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
-	// load from db
-	tapdConnection, err := findConnectionByInputParam(input)
-	if err != nil {
-		return nil, err
-	}
-	err = db.Delete(tapdConnection).Error
+	connection := &models.TapdConnection{}
+	err := connectionHelper.First(connection, input.Params)
 	if err != nil {
 		return nil, err
 	}
-
-	return &core.ApiResourceOutput{Body: tapdConnection}, nil
+	err = connectionHelper.Delete(connection)
+	return &core.ApiResourceOutput{Body: connection}, err
 }
 
 /*
 GET /plugins/tapd/connections
 */
 func ListConnections(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
-	tapdConnections := make([]models.TapdConnection, 0)
-	err := db.Find(&tapdConnections).Error
+	var connections []models.TapdConnection
+	err := connectionHelper.List(&connections)
 	if err != nil {
 		return nil, err
 	}
-	return &core.ApiResourceOutput{Body: tapdConnections}, nil
+
+	return &core.ApiResourceOutput{Body: connections, Status: http.StatusOK}, nil
 }
 
 /*
@@ -266,18 +148,12 @@ GET /plugins/tapd/connections/:connectionId
 {
 	"name": "tapd data connections name",
 	"endpoint": "tapd api endpoint, i.e. https://merico.atlassian.net/rest",
-	"basicAuthEncoded": "generated by `echo -n <tapd login email>:<tapd token> | base64`",
+	"Token": "generated by `echo -n <tapd login email>:<tapd token> | base64`",
 	"rateLimit": 10800,
 }
 */
 func GetConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
-	tapdConnection, err := findConnectionByInputParam(input)
-	if err != nil {
-		return nil, err
-	}
-
-	detail := &models.TapdConnectionDetail{
-		TapdConnection: *tapdConnection,
-	}
-	return &core.ApiResourceOutput{Body: detail}, nil
+	connection := &models.TapdConnection{}
+	err := connectionHelper.First(connection, input.Params)
+	return &core.ApiResourceOutput{Body: connection}, err
 }
diff --git a/plugins/tapd/api/init.go b/plugins/tapd/api/init.go
index fc8588d5..ef6c55f7 100644
--- a/plugins/tapd/api/init.go
+++ b/plugins/tapd/api/init.go
@@ -19,14 +19,25 @@ package api
 
 import (
 	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/go-playground/validator/v10"
 	"github.com/spf13/viper"
 	"gorm.io/gorm"
 )
 
 var db *gorm.DB
 var cfg *viper.Viper
+var vld *validator.Validate
+var connectionHelper *helper.ConnectionApiHelper
+var basicRes core.BasicRes
 
 func Init(config *viper.Viper, logger core.Logger, database *gorm.DB) {
 	db = database
 	cfg = config
+	basicRes = helper.NewDefaultBasicRes(cfg, logger, db)
+	vld = validator.New()
+	connectionHelper = helper.NewConnectionHelper(
+		basicRes,
+		vld,
+	)
 }
diff --git a/plugins/tapd/api/proxy.go b/plugins/tapd/api/proxy.go
deleted file mode 100644
index 7c1f16b7..00000000
--- a/plugins/tapd/api/proxy.go
+++ /dev/null
@@ -1,80 +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 (
-	"encoding/json"
-	"fmt"
-	"io/ioutil"
-	"strconv"
-	"time"
-
-	"github.com/apache/incubator-devlake/plugins/core"
-	"github.com/apache/incubator-devlake/plugins/helper"
-	"github.com/apache/incubator-devlake/plugins/tapd/models"
-)
-
-func Proxy(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
-	connectionId := input.Params["connectionId"]
-	if connectionId == "" {
-		return nil, fmt.Errorf("missing connectionId")
-	}
-	tapdConnectionId, err := strconv.ParseUint(connectionId, 10, 64)
-	if err != nil {
-		return nil, err
-	}
-	tapdConnection := &models.TapdConnection{}
-	err = db.First(tapdConnection, tapdConnectionId).Error
-	if err != nil {
-		return nil, err
-	}
-	encKey := cfg.GetString(core.EncodeKeyEnvStr)
-	basicAuth, err := core.Decrypt(encKey, tapdConnection.BasicAuthEncoded)
-	if err != nil {
-		return nil, err
-	}
-	apiClient, err := helper.NewApiClient(
-		tapdConnection.Endpoint,
-		map[string]string{
-			"Authorization": fmt.Sprintf("Basic %v", basicAuth),
-		},
-		30*time.Second,
-		"",
-		nil,
-	)
-	if err != nil {
-		return nil, err
-	}
-	resp, err := apiClient.Get(input.Params["path"], input.Query, nil)
-	if err != nil {
-		return nil, err
-	}
-	defer resp.Body.Close()
-
-	body, err := ioutil.ReadAll(resp.Body)
-	if err != nil {
-		return nil, err
-	}
-	// verify response body is json
-	var tmp interface{}
-	err = json.Unmarshal(body, &tmp)
-	if err != nil {
-		return nil, err
-	}
-	return &core.ApiResourceOutput{Status: resp.StatusCode, Body: json.RawMessage(body)}, nil
-}
diff --git a/plugins/tapd/tapd.go b/plugins/tapd/impl/impl.go
similarity index 75%
copy from plugins/tapd/tapd.go
copy to plugins/tapd/impl/impl.go
index aa029332..e3c01faa 100644
--- a/plugins/tapd/tapd.go
+++ b/plugins/tapd/impl/impl.go
@@ -15,15 +15,12 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package main
+package impl
 
 import (
 	"fmt"
 	"time"
 
-	"github.com/apache/incubator-devlake/config"
-	"github.com/apache/incubator-devlake/logger"
-
 	"github.com/apache/incubator-devlake/migration"
 	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
 	"github.com/apache/incubator-devlake/plugins/core"
@@ -31,9 +28,7 @@ import (
 	"github.com/apache/incubator-devlake/plugins/tapd/models"
 	"github.com/apache/incubator-devlake/plugins/tapd/models/migrationscripts"
 	"github.com/apache/incubator-devlake/plugins/tapd/tasks"
-	"github.com/apache/incubator-devlake/runner"
 	"github.com/mitchellh/mapstructure"
-	"github.com/spf13/cobra"
 	"github.com/spf13/viper"
 	"gorm.io/gorm"
 )
@@ -165,9 +160,7 @@ func (plugin Tapd) RootPkgPath() string {
 }
 
 func (plugin Tapd) MigrationScripts() []migration.Script {
-	return []migration.Script{new(migrationscripts.InitSchemas),
-		new(migrationscripts.UpdateSchemas20220531),
-		new(migrationscripts.UpdateSchemas20220613)}
+	return []migration.Script{new(migrationscripts.InitSchemas)}
 }
 
 func (plugin Tapd) ApiResources() map[string]map[string]core.ApiResourceHandler {
@@ -184,55 +177,5 @@ func (plugin Tapd) ApiResources() map[string]map[string]core.ApiResourceHandler
 			"DELETE": api.DeleteConnection,
 			"GET":    api.GetConnection,
 		},
-		"connections/:connectionId/proxy/rest/*path": {
-			"GET": api.Proxy,
-		},
-	}
-}
-
-// Export a variable named PluginEntry for Framework to search and load
-var PluginEntry Tapd //nolint
-
-// standalone mode for debugging
-func main() {
-	cmd := &cobra.Command{Use: "tapd"}
-	connectionId := cmd.Flags().Uint64P("connection", "c", 0, "tapd connection id")
-	workspaceId := cmd.Flags().Uint64P("workspace", "w", 0, "tapd workspace id")
-	companyId := cmd.Flags().Uint64P("company", "o", 0, "tapd company id")
-	err := cmd.MarkFlagRequired("connection")
-	if err != nil {
-		panic(err)
-	}
-	err = cmd.MarkFlagRequired("workspace")
-	if err != nil {
-		panic(err)
-	}
-
-	cmd.Run = func(c *cobra.Command, args []string) {
-		//runner.DirectRun(c, args, PluginEntry, map[string]interface{}{
-		//	"connectionId": *connectionId,
-		//	"workspaceId":  *workspaceId,
-		//	"companyId":    *companyId,
-		//})
-		cfg := config.GetConfig()
-		log := logger.Global.Nested(cmd.Use)
-		db, err := runner.NewGormDb(cfg, log)
-		if err != nil {
-			panic(err)
-		}
-		wsList := make([]*models.TapdWorkspace, 0)
-		err = db.Find(&wsList, "parent_id = ?", 59169984).Error //nolint TODO: fix the unused err
-		if err != nil {
-			panic(err)
-		}
-		for _, v := range wsList {
-			*workspaceId = v.ID
-			runner.DirectRun(c, args, PluginEntry, map[string]interface{}{
-				"connectionId": *connectionId,
-				"workspaceId":  *workspaceId,
-				"companyId":    *companyId,
-			})
-		}
 	}
-	runner.RunCmd(cmd)
 }
diff --git a/plugins/tapd/models/connection.go b/plugins/tapd/models/connection.go
index 5f346463..5e49cd2b 100644
--- a/plugins/tapd/models/connection.go
+++ b/plugins/tapd/models/connection.go
@@ -17,9 +17,7 @@ limitations under the License.
 
 package models
 
-import (
-	"github.com/apache/incubator-devlake/models/common"
-)
+import "github.com/apache/incubator-devlake/plugins/helper"
 
 type TestConnectionRequest struct {
 	Endpoint string `json:"endpoint" validate:"required,url"`
@@ -33,12 +31,13 @@ type WorkspaceResponse struct {
 	Value string
 }
 
+type AccessToken struct {
+	Token string `mapstructure:"token" validate:"required" json:"token"`
+}
+
 type TapdConnection struct {
-	common.Model
-	Name             string `gorm:"type:varchar(100);uniqueIndex" json:"name" validate:"required"`
-	Endpoint         string `gorm:"type:varchar(255)" json:"endpoint"`
-	BasicAuthEncoded string `gorm:"type:varchar(255)" json:"basicAuthEncoded"`
-	RateLimit        int    `comment:"api request rate limt per hour" json:"rateLimit"`
+	helper.RestConnection `mapstructure:",squash"`
+	AccessToken           `mapstructure:",squash"`
 }
 
 type TapdConnectionDetail struct {
diff --git a/plugins/tapd/models/migrationscripts/archived/tapd_connection.go b/plugins/tapd/models/migrationscripts/archived/tapd_connection.go
index 82a59e96..86f1f7fb 100644
--- a/plugins/tapd/models/migrationscripts/archived/tapd_connection.go
+++ b/plugins/tapd/models/migrationscripts/archived/tapd_connection.go
@@ -18,15 +18,33 @@ limitations under the License.
 package archived
 
 import (
-	"github.com/apache/incubator-devlake/models/common"
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
 )
 
+type BaseConnection struct {
+	Name string `gorm:"type:varchar(100);uniqueIndex" json:"name" validate:"required"`
+	archived.Model
+}
+
+type AccessToken struct {
+	Token string `mapstructure:"token" validate:"required" json:"token" encrypt:"yes"`
+}
+
+type BasicAuth struct {
+	Username string `mapstructure:"username" validate:"required" json:"username"`
+	Password string `mapstructure:"password" validate:"required" json:"password" encrypt:"yes"`
+}
+
+type RestConnection struct {
+	BaseConnection `mapstructure:",squash"`
+	Endpoint       string `mapstructure:"endpoint" validate:"required" json:"endpoint"`
+	Proxy          string `mapstructure:"proxy" json:"proxy"`
+	RateLimit      int    `comment:"api request rate limt per hour" json:"rateLimit"`
+}
+
 type TapdConnection struct {
-	common.Model
-	Name             string `gorm:"type:varchar(100);uniqueIndex" json:"name" validate:"required"`
-	Endpoint         string `gorm:"type:varchar(255)"`
-	BasicAuthEncoded string `gorm:"type:varchar(255)"`
-	RateLimit        int    `comment:"api request rate limt per second"`
+	RestConnection `mapstructure:",squash"`
+	AccessToken    `mapstructure:",squash"`
 }
 
 type TapdConnectionDetail struct {
diff --git a/plugins/tapd/models/migrationscripts/init_schema.go b/plugins/tapd/models/migrationscripts/init_schema.go
index 543ffdef..9c4d4184 100644
--- a/plugins/tapd/models/migrationscripts/init_schema.go
+++ b/plugins/tapd/models/migrationscripts/init_schema.go
@@ -19,6 +19,7 @@ package migrationscripts
 
 import (
 	"context"
+
 	"github.com/apache/incubator-devlake/plugins/tapd/models/migrationscripts/archived"
 	"gorm.io/gorm"
 )
@@ -57,11 +58,16 @@ func (*InitSchemas) Up(ctx context.Context, db *gorm.DB) error {
 		&archived.TapdIterationBug{},
 		&archived.TapdIterationStory{},
 		&archived.TapdIterationTask{},
+		&archived.TapdStoryCustomFields{},
+		&archived.TapdBugCustomFields{},
+		&archived.TapdTaskCustomFields{},
+		&archived.TapdStoryCategory{},
+		&archived.TapdStoryBug{},
 	)
 }
 
 func (*InitSchemas) Version() uint64 {
-	return 20220420231138
+	return 20220620231138
 }
 
 func (*InitSchemas) Name() string {
diff --git a/plugins/tapd/models/migrationscripts/update_schemas20220531.go b/plugins/tapd/models/migrationscripts/update_schemas20220531.go
deleted file mode 100644
index 144dafc5..00000000
--- a/plugins/tapd/models/migrationscripts/update_schemas20220531.go
+++ /dev/null
@@ -1,78 +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 migrationscripts
-
-import (
-	"context"
-	"github.com/apache/incubator-devlake/plugins/tapd/models/migrationscripts/archived"
-	"gorm.io/gorm"
-)
-
-type UpdateSchemas20220531 struct{}
-
-func (*UpdateSchemas20220531) Up(ctx context.Context, db *gorm.DB) error {
-	err := db.Migrator().AutoMigrate(
-		&archived.TapdWorkspace{},
-		&archived.TapdWorklog{},
-		&archived.TapdWorkspaceIteration{},
-		&archived.TapdUser{},
-		&archived.TapdBugChangelog{},
-		&archived.TapdBugChangelogItem{},
-		&archived.TapdStoryChangelog{},
-		&archived.TapdStoryChangelogItem{},
-		&archived.TapdTaskChangelog{},
-		&archived.TapdTaskChangelogItem{},
-		&archived.TapdIssue{},
-		&archived.TapdIteration{},
-		&archived.TapdConnection{},
-		&archived.TapdBug{},
-		&archived.TapdStory{},
-		&archived.TapdTask{},
-		&archived.TapdTaskLabel{},
-		&archived.TapdBugLabel{},
-		&archived.TapdStoryLabel{},
-		&archived.TapdBugStatus{},
-		&archived.TapdStoryStatus{},
-		&archived.TapdBugCommit{},
-		&archived.TapdStoryCommit{},
-		&archived.TapdTaskCommit{},
-		&archived.TapdWorkSpaceBug{},
-		&archived.TapdWorkSpaceStory{},
-		&archived.TapdWorkSpaceTask{},
-		&archived.TapdIterationBug{},
-		&archived.TapdIterationStory{},
-		&archived.TapdIterationTask{},
-		&archived.TapdStoryCustomFields{},
-		&archived.TapdBugCustomFields{},
-		&archived.TapdTaskCustomFields{},
-		&archived.TapdStoryCategory{},
-	)
-	return err
-}
-
-func (*UpdateSchemas20220531) Version() uint64 {
-	return 20220531132510
-}
-
-func (*UpdateSchemas20220531) Name() string {
-	return "rebuild tapd tables"
-}
-
-func (*UpdateSchemas20220531) Owner() string {
-	return "tapd"
-}
diff --git a/plugins/tapd/models/migrationscripts/update_schemas20220613.go b/plugins/tapd/models/migrationscripts/update_schemas20220613.go
deleted file mode 100644
index c583c750..00000000
--- a/plugins/tapd/models/migrationscripts/update_schemas20220613.go
+++ /dev/null
@@ -1,45 +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 migrationscripts
-
-import (
-	"context"
-	"github.com/apache/incubator-devlake/plugins/tapd/models/migrationscripts/archived"
-	"gorm.io/gorm"
-)
-
-type UpdateSchemas20220613 struct{}
-
-func (*UpdateSchemas20220613) Up(ctx context.Context, db *gorm.DB) error {
-	err := db.Migrator().AutoMigrate(
-		&archived.TapdStoryBug{},
-	)
-	return err
-}
-
-func (*UpdateSchemas20220613) Version() uint64 {
-	return 20220613132510
-}
-
-func (*UpdateSchemas20220613) Name() string {
-	return "add story bug"
-}
-
-func (*UpdateSchemas20220613) Owner() string {
-	return "tapd"
-}
diff --git a/plugins/tapd/tapd.go b/plugins/tapd/tapd.go
index aa029332..b4db1216 100644
--- a/plugins/tapd/tapd.go
+++ b/plugins/tapd/tapd.go
@@ -18,180 +18,12 @@ limitations under the License.
 package main
 
 import (
-	"fmt"
-	"time"
-
-	"github.com/apache/incubator-devlake/config"
-	"github.com/apache/incubator-devlake/logger"
-
-	"github.com/apache/incubator-devlake/migration"
-	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
-	"github.com/apache/incubator-devlake/plugins/core"
-	"github.com/apache/incubator-devlake/plugins/tapd/api"
-	"github.com/apache/incubator-devlake/plugins/tapd/models"
-	"github.com/apache/incubator-devlake/plugins/tapd/models/migrationscripts"
-	"github.com/apache/incubator-devlake/plugins/tapd/tasks"
+	"github.com/apache/incubator-devlake/plugins/tapd/impl"
 	"github.com/apache/incubator-devlake/runner"
-	"github.com/mitchellh/mapstructure"
 	"github.com/spf13/cobra"
-	"github.com/spf13/viper"
-	"gorm.io/gorm"
 )
 
-var _ core.PluginMeta = (*Tapd)(nil)
-var _ core.PluginInit = (*Tapd)(nil)
-var _ core.PluginTask = (*Tapd)(nil)
-var _ core.PluginApi = (*Tapd)(nil)
-var _ core.Migratable = (*Tapd)(nil)
-
-type Tapd struct{}
-
-func (plugin Tapd) Init(config *viper.Viper, logger core.Logger, db *gorm.DB) error {
-	api.Init(config, logger, db)
-	return nil
-}
-
-func (plugin Tapd) Description() string {
-	return "To collect and enrich data from Tapd"
-}
-
-func (plugin Tapd) SubTaskMetas() []core.SubTaskMeta {
-	return []core.SubTaskMeta{
-		tasks.CollectCompanyMeta,
-		tasks.ExtractCompanyMeta,
-		tasks.CollectWorkspaceMeta,
-		tasks.ExtractWorkspaceMeta,
-		tasks.CollectStoryCustomFieldsMeta,
-		tasks.ExtractStoryCustomFieldsMeta,
-		tasks.CollectTaskCustomFieldsMeta,
-		tasks.ExtractTaskCustomFieldsMeta,
-		tasks.CollectBugCustomFieldsMeta,
-		tasks.ExtractBugCustomFieldsMeta,
-		tasks.CollectStoryCategoriesMeta,
-		tasks.ExtractStoryCategoriesMeta,
-		tasks.CollectStoryStatusMeta,
-		tasks.ExtractStoryStatusMeta,
-		tasks.CollectBugStatusMeta,
-		tasks.ExtractBugStatusMeta,
-		tasks.CollectUserMeta,
-		tasks.ExtractUserMeta,
-		tasks.CollectIterationMeta,
-		tasks.ExtractIterationMeta,
-		tasks.CollectStoryMeta,
-		tasks.CollectBugMeta,
-		tasks.CollectTaskMeta,
-		tasks.ExtractStoryMeta,
-		tasks.ExtractBugMeta,
-		tasks.ExtractTaskMeta,
-		tasks.CollectBugChangelogMeta,
-		tasks.ExtractBugChangelogMeta,
-		tasks.CollectStoryChangelogMeta,
-		tasks.ExtractStoryChangelogMeta,
-		tasks.CollectTaskChangelogMeta,
-		tasks.ExtractTaskChangelogMeta,
-		tasks.CollectWorklogMeta,
-		tasks.ExtractWorklogMeta,
-		tasks.CollectBugCommitMeta,
-		tasks.ExtractBugCommitMeta,
-		tasks.CollectStoryCommitMeta,
-		tasks.ExtractStoryCommitMeta,
-		tasks.CollectTaskCommitMeta,
-		tasks.ExtractTaskCommitMeta,
-		tasks.CollectStoryBugMeta,
-		tasks.ExtractStoryBugsMeta,
-		tasks.ConvertWorkspaceMeta,
-		tasks.ConvertUserMeta,
-		tasks.ConvertIterationMeta,
-		tasks.ConvertStoryMeta,
-		tasks.ConvertBugMeta,
-		tasks.ConvertTaskMeta,
-		tasks.ConvertWorklogMeta,
-		tasks.ConvertBugChangelogMeta,
-		tasks.ConvertStoryChangelogMeta,
-		tasks.ConvertTaskChangelogMeta,
-		tasks.ConvertBugCommitMeta,
-		tasks.ConvertStoryCommitMeta,
-		tasks.ConvertTaskCommitMeta,
-		tasks.ConvertStoryLabelsMeta,
-		tasks.ConvertTaskLabelsMeta,
-		tasks.ConvertBugLabelsMeta,
-	}
-}
-
-func (plugin Tapd) PrepareTaskData(taskCtx core.TaskContext, options map[string]interface{}) (interface{}, error) {
-	db := taskCtx.GetDb()
-	var op tasks.TapdOptions
-	err := mapstructure.Decode(options, &op)
-	if err != nil {
-		return nil, err
-	}
-	if op.ConnectionId == 0 {
-		return nil, fmt.Errorf("ConnectionId is required for Tapd execution")
-	}
-	connection := &models.TapdConnection{}
-	err = db.First(connection, op.ConnectionId).Error
-	if err != nil {
-		return nil, err
-	}
-	var since time.Time
-	if op.Since != "" {
-		since, err = time.Parse("2006-01-02T15:04:05Z", op.Since)
-		if err != nil {
-			return nil, fmt.Errorf("invalid value for `since`: %w", err)
-		}
-	}
-	tapdApiClient, err := tasks.NewTapdApiClient(taskCtx, connection)
-	if err != nil {
-		return nil, fmt.Errorf("failed to create tapd api client: %w", err)
-	}
-	taskData := &tasks.TapdTaskData{
-		Options:    &op,
-		ApiClient:  tapdApiClient,
-		Connection: connection,
-	}
-	if !since.IsZero() {
-		taskData.Since = &since
-	}
-	tasks.UserIdGen = didgen.NewDomainIdGenerator(&models.TapdUser{})
-	tasks.WorkspaceIdGen = didgen.NewDomainIdGenerator(&models.TapdWorkspace{})
-	tasks.IssueIdGen = didgen.NewDomainIdGenerator(&models.TapdIssue{})
-	tasks.IterIdGen = didgen.NewDomainIdGenerator(&models.TapdIteration{})
-
-	return taskData, nil
-}
-
-func (plugin Tapd) RootPkgPath() string {
-	return "github.com/apache/incubator-devlake/plugins/tapd"
-}
-
-func (plugin Tapd) MigrationScripts() []migration.Script {
-	return []migration.Script{new(migrationscripts.InitSchemas),
-		new(migrationscripts.UpdateSchemas20220531),
-		new(migrationscripts.UpdateSchemas20220613)}
-}
-
-func (plugin Tapd) 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": {
-			"PATCH":  api.PatchConnection,
-			"DELETE": api.DeleteConnection,
-			"GET":    api.GetConnection,
-		},
-		"connections/:connectionId/proxy/rest/*path": {
-			"GET": api.Proxy,
-		},
-	}
-}
-
-// Export a variable named PluginEntry for Framework to search and load
-var PluginEntry Tapd //nolint
+var PluginEntry impl.Tapd
 
 // standalone mode for debugging
 func main() {
@@ -209,30 +41,31 @@ func main() {
 	}
 
 	cmd.Run = func(c *cobra.Command, args []string) {
-		//runner.DirectRun(c, args, PluginEntry, map[string]interface{}{
-		//	"connectionId": *connectionId,
-		//	"workspaceId":  *workspaceId,
-		//	"companyId":    *companyId,
-		//})
-		cfg := config.GetConfig()
-		log := logger.Global.Nested(cmd.Use)
-		db, err := runner.NewGormDb(cfg, log)
-		if err != nil {
-			panic(err)
-		}
-		wsList := make([]*models.TapdWorkspace, 0)
-		err = db.Find(&wsList, "parent_id = ?", 59169984).Error //nolint TODO: fix the unused err
-		if err != nil {
-			panic(err)
-		}
-		for _, v := range wsList {
-			*workspaceId = v.ID
-			runner.DirectRun(c, args, PluginEntry, map[string]interface{}{
-				"connectionId": *connectionId,
-				"workspaceId":  *workspaceId,
-				"companyId":    *companyId,
-			})
-		}
+		runner.DirectRun(c, args, PluginEntry, map[string]interface{}{
+			"connectionId": *connectionId,
+			"workspaceId":  *workspaceId,
+			"companyId":    *companyId,
+		})
+
+		// cfg := config.GetConfig()
+		// log := logger.Global.Nested(cmd.Use)
+		// db, err := runner.NewGormDb(cfg, log)
+		// if err != nil {
+		// 	panic(err)
+		// }
+		// wsList := make([]*models.TapdWorkspace, 0)
+		// err = db.Find(&wsList, "parent_id = ?", 59169984).Error //nolint TODO: fix the unused err
+		// if err != nil {
+		// 	panic(err)
+		// }
+		// for _, v := range wsList {
+		// 	*workspaceId = v.ID
+		// 	runner.DirectRun(c, args, PluginEntry, map[string]interface{}{
+		// 		"connectionId": *connectionId,
+		// 		"workspaceId":  *workspaceId,
+		// 		"companyId":    *companyId,
+		// 	})
+		// }
 	}
 	runner.RunCmd(cmd)
 }
diff --git a/plugins/tapd/tasks/api_client.go b/plugins/tapd/tasks/api_client.go
index 977d076f..d2be604a 100644
--- a/plugins/tapd/tasks/api_client.go
+++ b/plugins/tapd/tasks/api_client.go
@@ -18,6 +18,7 @@ limitations under the License.
 package tasks
 
 import (
+	"encoding/base64"
 	"fmt"
 	"net/http"
 
@@ -27,13 +28,8 @@ import (
 )
 
 func NewTapdApiClient(taskCtx core.TaskContext, connection *models.TapdConnection) (*helper.ApiAsyncClient, error) {
-	// load configuration
-	encKey := taskCtx.GetConfig(core.EncodeKeyEnvStr)
-	auth, err := core.Decrypt(encKey, connection.BasicAuthEncoded)
-	if err != nil {
-		return nil, fmt.Errorf("Failed to decrypt Auth AccessToken: %w", err)
-	}
-
+	
+	auth := base64.StdEncoding.EncodeToString([]byte(connection.Token))
 	// create synchronize api client so we can calculate api rate limit dynamically
 	headers := map[string]string{
 		"Authorization": fmt.Sprintf("Basic %v", auth),