You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by kl...@apache.org on 2022/11/22 12:14:31 UTC

[incubator-devlake] branch main updated: feat: scope and transformation rule for jira (#3753)

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

klesh 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 0ca624873 feat: scope and transformation rule for jira (#3753)
0ca624873 is described below

commit 0ca624873057f2f46ddfc27b7e5dd1b825c137fd
Author: mindlesscloud <li...@merico.dev>
AuthorDate: Tue Nov 22 20:14:27 2022 +0800

    feat: scope and transformation rule for jira (#3753)
    
    * feat: scope and transformation rule for jira
    
    * fix: revise based on review
    
    * fix: fix e2e test
    
    * fix: return the entity on creat/update
---
 plugins/core/plugin_meta.go                        |  13 +-
 plugins/jira/api/scope.go                          | 124 ++++++++++++++++
 plugins/jira/api/transformation_rule.go            | 158 +++++++++++++++++++++
 plugins/jira/e2e/epic_test.go                      |   2 +-
 plugins/jira/e2e/issue_test.go                     |   2 +-
 plugins/jira/e2e/remotelink_test.go                |   2 +-
 plugins/jira/impl/impl.go                          |  53 ++++++-
 ...go => 20221116_add_trasformation_rule_table.go} |  21 ++-
 .../transformation_rules.go}                       |  19 +--
 plugins/jira/models/migrationscripts/register.go   |   1 +
 .../register.go => transformation_rules.go}        |  19 +--
 plugins/jira/tasks/task_data.go                    |  51 ++++++-
 12 files changed, 427 insertions(+), 38 deletions(-)

diff --git a/plugins/core/plugin_meta.go b/plugins/core/plugin_meta.go
index 6bc05a53d..8c81b6352 100644
--- a/plugins/core/plugin_meta.go
+++ b/plugins/core/plugin_meta.go
@@ -17,7 +17,7 @@ limitations under the License.
 
 package core
 
-// Minimal features a plugin should comply, should be implemented by all plugins
+// PluginMeta is the Minimal features a plugin should comply, should be implemented by all plugins
 type PluginMeta interface {
 	Description() string
 	// PkgPath information lost when compiled as plugin(.so)
@@ -31,12 +31,19 @@ type GrafanaDashboard struct {
 	GrafanaDashboardJson string
 }
 
-// PluginDashboard return it's dashboard which should be display at grafana
+// PluginDashboard return its dashboard which should be display at grafana
 type PluginDashboard interface {
 	Dashboards() []GrafanaDashboard
 }
 
-// PluginIcon return it's icon (.svg text)
+// PluginIcon return its icon (.svg text)
 type PluginIcon interface {
 	SvgIcon() string
 }
+
+// PluginSource abstracts data sources
+type PluginSource interface {
+	Connection() interface{}
+	Scope() interface{}
+	TransformationRule() interface{}
+}
diff --git a/plugins/jira/api/scope.go b/plugins/jira/api/scope.go
new file mode 100644
index 000000000..a6a575f7c
--- /dev/null
+++ b/plugins/jira/api/scope.go
@@ -0,0 +1,124 @@
+/*
+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 (
+	"net/http"
+	"strconv"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/jira/models"
+	"github.com/mitchellh/mapstructure"
+)
+
+type putBoardRequest struct {
+	ProjectId uint   `json:"projectId"`
+	Name      string `json:"name"`
+	Self      string `json:"self"`
+	Type      string `json:"type"`
+}
+
+// PutScope create or update jira board
+// @Summary create or update jira board
+// @Description Create or update Jira board
+// @Tags plugins/jira
+// @Accept application/json
+// @Param connectionId path int false "connection ID"
+// @Param boardId path int false "board ID"
+// @Param scope body putBoardRequest true "json"
+// @Success 200  {object} models.JiraBoard
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/connections/{connectionId}/scopes/{boardId} [PUT]
+func PutScope(input *core.ApiResourceInput) (*core.ApiResourceOutput, errors.Error) {
+	connectionId, boardId := extractParam(input.Params)
+	if connectionId*boardId == 0 {
+		return nil, errors.BadInput.New("invalid path params")
+	}
+	var req putBoardRequest
+	err := mapstructure.Decode(input.Body, &req)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error decoding map into putBoardRequest")
+	}
+	board := &models.JiraBoard{
+		ConnectionId: connectionId,
+		BoardId:      boardId,
+		ProjectId:    req.ProjectId,
+		Name:         req.Name,
+		Self:         req.Self,
+		Type:         req.Type,
+	}
+	err = basicRes.GetDal().CreateOrUpdate(&board)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving JiraBoard")
+	}
+	return &core.ApiResourceOutput{Body: board, Status: http.StatusOK}, nil
+}
+
+// GetScopeList get Jira boards
+// @Summary get Jira boards
+// @Description get Jira boards
+// @Tags plugins/jira
+// @Param connectionId path int false "connection ID"
+// @Success 200  {object} []models.JiraBoard
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/connections/{connectionId}/scopes/ [GET]
+func GetScopeList(input *core.ApiResourceInput) (*core.ApiResourceOutput, errors.Error) {
+	var boards []models.JiraBoard
+	connectionId, _ := extractParam(input.Params)
+	if connectionId == 0 {
+		return nil, errors.BadInput.New("invalid path params")
+	}
+	err := basicRes.GetDal().All(&boards, dal.Where("connection_id = ?", connectionId))
+	if err != nil {
+		return nil, err
+	}
+	return &core.ApiResourceOutput{Body: boards, Status: http.StatusOK}, nil
+}
+
+// GetScope get one Jira board
+// @Summary get one Jira board
+// @Description get one Jira board
+// @Tags plugins/jira
+// @Param connectionId path int false "connection ID"
+// @Param boardId path int false "board ID"
+// @Success 200  {object} models.JiraBoard
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/connections/{connectionId}/scopes/{boardId} [GET]
+func GetScope(input *core.ApiResourceInput) (*core.ApiResourceOutput, errors.Error) {
+	var board models.JiraBoard
+	connectionId, boardId := extractParam(input.Params)
+	if connectionId*boardId == 0 {
+		return nil, errors.BadInput.New("invalid path params")
+	}
+	err := basicRes.GetDal().First(&board, dal.Where("connection_id = ? AND board_id = ?", connectionId, board))
+	if err != nil {
+		return nil, err
+	}
+	return &core.ApiResourceOutput{Body: board, Status: http.StatusOK}, nil
+}
+
+func extractParam(params map[string]string) (uint64, uint64) {
+	connectionId, _ := strconv.ParseUint(params["connectionId"], 10, 64)
+	boardId, _ := strconv.ParseUint(params["boardId"], 10, 64)
+	return connectionId, boardId
+}
diff --git a/plugins/jira/api/transformation_rule.go b/plugins/jira/api/transformation_rule.go
new file mode 100644
index 000000000..b0c04b885
--- /dev/null
+++ b/plugins/jira/api/transformation_rule.go
@@ -0,0 +1,158 @@
+/*
+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"
+	"net/http"
+	"strconv"
+	"time"
+
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/jira/models"
+	"github.com/apache/incubator-devlake/plugins/jira/tasks"
+	"github.com/mitchellh/mapstructure"
+)
+
+// CreateTransformationRule create transformation rule for Jira
+// @Summary create transformation rule for Jira
+// @Description create transformation rule for Jira
+// @Tags plugins/jira
+// @Accept application/json
+// @Param transformationRule body tasks.TransformationRules true "transformation rule"
+// @Success 200  {object} models.JiraTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/transformation_rules [POST]
+func CreateTransformationRule(input *core.ApiResourceInput) (*core.ApiResourceOutput, errors.Error) {
+	rule, err := makeDbTransformationRuleFromInput(input)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error in makeJiraTransformationRule")
+	}
+	err = basicRes.GetDal().Create(&rule)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving TransformationRule")
+	}
+	return &core.ApiResourceOutput{Body: rule, Status: http.StatusOK}, nil
+}
+
+// UpdateTransformationRule update transformation rule for Jira
+// @Summary update transformation rule for Jira
+// @Description update transformation rule for Jira
+// @Tags plugins/jira
+// @Accept application/json
+// @Param id path int true "id"
+// @Param transformationRule body tasks.TransformationRules true "transformation rule"
+// @Success 200  {object} models.JiraTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/transformation_rules/{id} [PATCH]
+func UpdateTransformationRule(input *core.ApiResourceInput) (*core.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 old models.JiraTransformationRule
+	err = basicRes.GetDal().First(&old, dal.Where("id = ?", transformationRuleId))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving TransformationRule")
+	}
+	rule := new(tasks.TransformationRules)
+	_, err = rule.FromDb(&old)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "the transformation rule ID should be an integer")
+	}
+	err = mapstructure.Decode(input.Body, &rule)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error decoding map into putBoardRequest")
+	}
+	dbRule, err := makeDbTransformationRule(rule)
+	dbRule.Model = old.Model
+	dbRule.UpdatedAt = time.Now()
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error in makeJiraTransformationRule")
+	}
+
+	err = basicRes.GetDal().Update(&dbRule, dal.Where("id = ?", transformationRuleId))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on saving TransformationRule")
+	}
+	return &core.ApiResourceOutput{Body: rule, Status: http.StatusOK}, nil
+}
+
+func makeDbTransformationRuleFromInput(input *core.ApiResourceInput) (*models.JiraTransformationRule, errors.Error) {
+	var req tasks.TransformationRules
+	err := mapstructure.Decode(input.Body, &req)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error decoding map into putBoardRequest")
+	}
+	return makeDbTransformationRule(&req)
+}
+func makeDbTransformationRule(rule *tasks.TransformationRules) (*models.JiraTransformationRule, errors.Error) {
+	blob, err := json.Marshal(rule.TypeMappings)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error marshaling TypeMappings")
+	}
+	return &models.JiraTransformationRule{
+		EpicKeyField:               rule.EpicKeyField,
+		StoryPointField:            rule.StoryPointField,
+		RemotelinkCommitShaPattern: rule.RemotelinkCommitShaPattern,
+		TypeMappings:               blob,
+	}, nil
+}
+
+// GetTransformationRule return one transformation rule
+// @Summary return one transformation rule
+// @Description return one transformation rule
+// @Tags plugins/jira
+// @Param id path int true "id"
+// @Success 200  {object} models.JiraTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/transformation_rules/{id} [GET]
+func GetTransformationRule(input *core.ApiResourceInput) (*core.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.JiraTransformationRule
+	err = basicRes.GetDal().First(&rule, dal.Where("id = ?", transformationRuleId))
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on get TransformationRule")
+	}
+	return &core.ApiResourceOutput{Body: rule, Status: http.StatusOK}, nil
+}
+
+// GetTransformationRuleList return all transformation rules
+// @Summary return all transformation rules
+// @Description return all transformation rules
+// @Tags plugins/jira
+// @Success 200  {object} []models.JiraTransformationRule
+// @Failure 400  {object} shared.ApiBody "Bad Request"
+// @Failure 500  {object} shared.ApiBody "Internal Error"
+// @Router /plugins/jira/transformation_rules [GET]
+func GetTransformationRuleList(input *core.ApiResourceInput) (*core.ApiResourceOutput, errors.Error) {
+	var rules []models.JiraTransformationRule
+	err := basicRes.GetDal().All(&rules)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error on get TransformationRule list")
+	}
+	return &core.ApiResourceOutput{Body: rules, Status: http.StatusOK}, nil
+}
diff --git a/plugins/jira/e2e/epic_test.go b/plugins/jira/e2e/epic_test.go
index 91a1489a3..dedd5d39e 100644
--- a/plugins/jira/e2e/epic_test.go
+++ b/plugins/jira/e2e/epic_test.go
@@ -35,7 +35,7 @@ func TestEpicDataflow(t *testing.T) {
 		Options: &tasks.JiraOptions{
 			ConnectionId:        1,
 			BoardId:             93,
-			TransformationRules: tasks.TransformationRules{StoryPointField: "customfield_10024"},
+			TransformationRules: &tasks.TransformationRules{StoryPointField: "customfield_10024"},
 		},
 	}
 
diff --git a/plugins/jira/e2e/issue_test.go b/plugins/jira/e2e/issue_test.go
index 5e05a3e8f..a712460e8 100644
--- a/plugins/jira/e2e/issue_test.go
+++ b/plugins/jira/e2e/issue_test.go
@@ -35,7 +35,7 @@ func TestIssueDataFlow(t *testing.T) {
 		Options: &tasks.JiraOptions{
 			ConnectionId: 2,
 			BoardId:      8,
-			TransformationRules: tasks.TransformationRules{
+			TransformationRules: &tasks.TransformationRules{
 				StoryPointField: "customfield_10024",
 				TypeMappings: map[string]tasks.TypeMapping{
 					"子任务": {
diff --git a/plugins/jira/e2e/remotelink_test.go b/plugins/jira/e2e/remotelink_test.go
index 82ea62b14..1e096efa4 100644
--- a/plugins/jira/e2e/remotelink_test.go
+++ b/plugins/jira/e2e/remotelink_test.go
@@ -34,7 +34,7 @@ func TestRemotelinkDataFlow(t *testing.T) {
 		Options: &tasks.JiraOptions{
 			ConnectionId:        2,
 			BoardId:             8,
-			TransformationRules: tasks.TransformationRules{RemotelinkCommitShaPattern: ".*/commit/(.*)"},
+			TransformationRules: &tasks.TransformationRules{RemotelinkCommitShaPattern: ".*/commit/(.*)"},
 		},
 	}
 
diff --git a/plugins/jira/impl/impl.go b/plugins/jira/impl/impl.go
index c7705aa5a..5b2b957ef 100644
--- a/plugins/jira/impl/impl.go
+++ b/plugins/jira/impl/impl.go
@@ -20,11 +20,12 @@ package impl
 import (
 	"fmt"
 	"net/http"
+	"strconv"
 	"time"
 
 	"github.com/apache/incubator-devlake/errors"
-
 	"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/jira/api"
 	"github.com/apache/incubator-devlake/plugins/jira/models"
@@ -42,10 +43,24 @@ var _ core.PluginModel = (*Jira)(nil)
 var _ core.PluginMigration = (*Jira)(nil)
 var _ core.PluginBlueprintV100 = (*Jira)(nil)
 var _ core.CloseablePluginTask = (*Jira)(nil)
+var _ core.PluginSource = (*Jira)(nil)
+
+type Jira struct {
+}
+
+func (plugin Jira) Connection() interface{} {
+	return &models.JiraConnection{}
+}
 
-type Jira struct{}
+func (plugin Jira) Scope() interface{} {
+	return &models.JiraBoard{}
+}
+
+func (plugin Jira) TransformationRule() interface{} {
+	return &models.JiraTransformationRule{}
+}
 
-func (plugin Jira) Init(config *viper.Viper, logger core.Logger, db *gorm.DB) errors.Error {
+func (plugin *Jira) Init(config *viper.Viper, logger core.Logger, db *gorm.DB) errors.Error {
 	api.Init(config, logger, db)
 	return nil
 }
@@ -163,6 +178,23 @@ func (plugin Jira) PrepareTaskData(taskCtx core.TaskContext, options map[string]
 			return nil, errors.BadInput.Wrap(err, "invalid value for `since`")
 		}
 	}
+	if op.BoardId == 0 && op.ScopeId != "" {
+		op.BoardId, err = strconv.ParseUint(op.ScopeId, 10, 64)
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "invalid value for scopeId")
+		}
+	}
+	if op.TransformationRules == nil && op.TransformationRuleId != 0 {
+		var transformationRule models.JiraTransformationRule
+		err = taskCtx.GetDal().First(&transformationRule, dal.Where("id = ?", op.TransformationRuleId))
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "fail to get transformationRule")
+		}
+		op.TransformationRules, err = tasks.MakeTransformationRules(transformationRule)
+		if err != nil {
+			return nil, errors.BadInput.Wrap(err, "fail to make transformationRule")
+		}
+	}
 	jiraApiClient, err := tasks.NewJiraApiClient(taskCtx, connection)
 	if err != nil {
 		return nil, errors.Default.Wrap(err, "failed to create jira api client")
@@ -217,6 +249,21 @@ func (plugin Jira) ApiResources() map[string]map[string]core.ApiResourceHandler
 		"connections/:connectionId/proxy/rest/*path": {
 			"GET": api.Proxy,
 		},
+		"connections/:connectionId/scopes/:boardId": {
+			"GET": api.GetScope,
+			"PUT": api.PutScope,
+		},
+		"connections/:connectionId/scopes": {
+			"GET": api.GetScopeList,
+		},
+		"transformation_rules": {
+			"POST": api.CreateTransformationRule,
+			"GET":  api.GetTransformationRuleList,
+		},
+		"transformation_rules/:id": {
+			"PATCH": api.UpdateTransformationRule,
+			"GET":   api.GetTransformationRule,
+		},
 	}
 }
 
diff --git a/plugins/jira/models/migrationscripts/register.go b/plugins/jira/models/migrationscripts/20221116_add_trasformation_rule_table.go
similarity index 61%
copy from plugins/jira/models/migrationscripts/register.go
copy to plugins/jira/models/migrationscripts/20221116_add_trasformation_rule_table.go
index 6401fd9f4..8f1079e51 100644
--- a/plugins/jira/models/migrationscripts/register.go
+++ b/plugins/jira/models/migrationscripts/20221116_add_trasformation_rule_table.go
@@ -18,14 +18,21 @@ limitations under the License.
 package migrationscripts
 
 import (
+	"github.com/apache/incubator-devlake/errors"
 	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/jira/models/migrationscripts/archived"
 )
 
-// All return all the migration scripts
-func All() []core.MigrationScript {
-	return []core.MigrationScript{
-		new(addSourceTable20220407),
-		new(renameSourceTable20220505),
-		new(addInitTables20220716),
-	}
+type addTransformationRule20221116 struct{}
+
+func (script *addTransformationRule20221116) Up(basicRes core.BasicRes) errors.Error {
+	return basicRes.GetDal().AutoMigrate(&archived.JiraTransformationRule{})
+}
+
+func (*addTransformationRule20221116) Version() uint64 {
+	return 20221117122532
+}
+
+func (*addTransformationRule20221116) Name() string {
+	return "add table _tool_jira_transformation_rules"
 }
diff --git a/plugins/jira/models/migrationscripts/register.go b/plugins/jira/models/migrationscripts/archived/transformation_rules.go
similarity index 60%
copy from plugins/jira/models/migrationscripts/register.go
copy to plugins/jira/models/migrationscripts/archived/transformation_rules.go
index 6401fd9f4..a13420baf 100644
--- a/plugins/jira/models/migrationscripts/register.go
+++ b/plugins/jira/models/migrationscripts/archived/transformation_rules.go
@@ -15,17 +15,18 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package migrationscripts
+package archived
 
 import (
-	"github.com/apache/incubator-devlake/plugins/core"
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
 )
 
-// All return all the migration scripts
-func All() []core.MigrationScript {
-	return []core.MigrationScript{
-		new(addSourceTable20220407),
-		new(renameSourceTable20220505),
-		new(addInitTables20220716),
-	}
+type JiraTransformationRule struct {
+	archived.Model
+	EpicKeyField               string          `json:"epicKeyField" gorm:"type:varchar(255)"`
+	StoryPointField            string          `json:"storyPointField" gorm:"type:varchar(255)"`
+	RemotelinkCommitShaPattern string          `json:"remotelinkCommitShaPattern" gorm:"type:varchar(255)"`
+	TypeMappings               json.RawMessage `json:"typeMappings"`
 }
diff --git a/plugins/jira/models/migrationscripts/register.go b/plugins/jira/models/migrationscripts/register.go
index 6401fd9f4..15f990b14 100644
--- a/plugins/jira/models/migrationscripts/register.go
+++ b/plugins/jira/models/migrationscripts/register.go
@@ -27,5 +27,6 @@ func All() []core.MigrationScript {
 		new(addSourceTable20220407),
 		new(renameSourceTable20220505),
 		new(addInitTables20220716),
+		new(addTransformationRule20221116),
 	}
 }
diff --git a/plugins/jira/models/migrationscripts/register.go b/plugins/jira/models/transformation_rules.go
similarity index 61%
copy from plugins/jira/models/migrationscripts/register.go
copy to plugins/jira/models/transformation_rules.go
index 6401fd9f4..1aa3aa055 100644
--- a/plugins/jira/models/migrationscripts/register.go
+++ b/plugins/jira/models/transformation_rules.go
@@ -15,17 +15,18 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package migrationscripts
+package models
 
 import (
-	"github.com/apache/incubator-devlake/plugins/core"
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/models/common"
 )
 
-// All return all the migration scripts
-func All() []core.MigrationScript {
-	return []core.MigrationScript{
-		new(addSourceTable20220407),
-		new(renameSourceTable20220505),
-		new(addInitTables20220716),
-	}
+type JiraTransformationRule struct {
+	common.Model
+	EpicKeyField               string          `json:"epicKeyField" gorm:"type:varchar(255)"`
+	StoryPointField            string          `json:"storyPointField" gorm:"type:varchar(255)"`
+	RemotelinkCommitShaPattern string          `json:"remotelinkCommitShaPattern" gorm:"type:varchar(255)"`
+	TypeMappings               json.RawMessage `json:"typeMappings"`
 }
diff --git a/plugins/jira/tasks/task_data.go b/plugins/jira/tasks/task_data.go
index 82937225d..4ee28ffbd 100644
--- a/plugins/jira/tasks/task_data.go
+++ b/plugins/jira/tasks/task_data.go
@@ -18,6 +18,7 @@ limitations under the License.
 package tasks
 
 import (
+	"encoding/json"
 	"fmt"
 	"github.com/apache/incubator-devlake/errors"
 	"time"
@@ -46,11 +47,53 @@ type TransformationRules struct {
 	TypeMappings               TypeMappings `json:"typeMappings"`
 }
 
+func (r *TransformationRules) ToDb() (rule *models.JiraTransformationRule, error2 errors.Error) {
+	blob, err := json.Marshal(r.TypeMappings)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error marshaling TypeMappings")
+	}
+	return &models.JiraTransformationRule{
+		EpicKeyField:               r.EpicKeyField,
+		StoryPointField:            r.StoryPointField,
+		RemotelinkCommitShaPattern: r.RemotelinkCommitShaPattern,
+		TypeMappings:               blob,
+	}, nil
+}
+func (r *TransformationRules) FromDb(rule *models.JiraTransformationRule) (*TransformationRules, errors.Error) {
+	mappings := make(map[string]TypeMapping)
+	err := json.Unmarshal(rule.TypeMappings, &mappings)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "error marshaling TypeMappings")
+	}
+	r.EpicKeyField = rule.EpicKeyField
+	r.StoryPointField = rule.StoryPointField
+	r.RemotelinkCommitShaPattern = rule.RemotelinkCommitShaPattern
+	r.TypeMappings = mappings
+	return r, nil
+}
+
+func MakeTransformationRules(rule models.JiraTransformationRule) (*TransformationRules, errors.Error) {
+	var typeMapping TypeMappings
+	err := json.Unmarshal(rule.TypeMappings, &typeMapping)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, "unable to unmarshal the typeMapping")
+	}
+	result := &TransformationRules{
+		EpicKeyField:               rule.EpicKeyField,
+		StoryPointField:            rule.StoryPointField,
+		RemotelinkCommitShaPattern: rule.RemotelinkCommitShaPattern,
+		TypeMappings:               typeMapping,
+	}
+	return result, nil
+}
+
 type JiraOptions struct {
-	ConnectionId        uint64 `json:"connectionId"`
-	BoardId             uint64 `json:"boardId"`
-	Since               string
-	TransformationRules TransformationRules `json:"transformationRules"`
+	ConnectionId         uint64 `json:"connectionId"`
+	BoardId              uint64 `json:"boardId"`
+	Since                string
+	TransformationRules  *TransformationRules `json:"transformationRules"`
+	ScopeId              string
+	TransformationRuleId uint64
 }
 
 type JiraTaskData struct {