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 2023/03/16 04:58:20 UTC

[incubator-devlake] branch main updated: refactor(framework): add remote helper (#4677)

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

warren 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 d05c78d71 refactor(framework): add remote helper (#4677)
d05c78d71 is described below

commit d05c78d71bbb98b7e4833f5f60bc94f394eadab1
Author: Warren Chen <yi...@merico.dev>
AuthorDate: Thu Mar 16 12:58:14 2023 +0800

    refactor(framework): add remote helper (#4677)
---
 backend/core/plugin/plugin_blueprint.go            |  12 -
 .../plugin/plugin_connection_abstract.go}          |  13 +-
 .../plugin/plugin_scope_abstract.go}               |  47 ++--
 backend/helpers/pluginhelper/api/api_client.go     |   2 +-
 .../helpers/pluginhelper/api/connection_auths.go   |  28 +--
 .../helpers/pluginhelper/api/connection_helper.go  |   3 +-
 .../helpers/pluginhelper/api/remote_api_helper.go  | 255 +++++++++++++++++++++
 backend/helpers/pluginhelper/api/scope_helper.go   |  10 +-
 backend/plugins/bitbucket/api/remote.go            |  34 ++-
 backend/plugins/github/models/repo.go              |  12 +
 backend/plugins/gitlab/api/blueprint_v200.go       |   4 +-
 backend/plugins/gitlab/api/init.go                 |   7 +
 backend/plugins/gitlab/api/remote.go               | 160 +------------
 backend/plugins/gitlab/impl/impl.go                |   5 +-
 backend/plugins/gitlab/models/connection.go        |  65 +++++-
 backend/plugins/gitlab/models/project.go           |  78 +++++++
 .../migrationscripts/20230129_add_multi_auth.go    |   4 +-
 backend/plugins/sonarqube/models/connection.go     |   4 +-
 18 files changed, 502 insertions(+), 241 deletions(-)

diff --git a/backend/core/plugin/plugin_blueprint.go b/backend/core/plugin/plugin_blueprint.go
index 0c681829d..3a9fe34e6 100644
--- a/backend/core/plugin/plugin_blueprint.go
+++ b/backend/core/plugin/plugin_blueprint.go
@@ -105,18 +105,6 @@ step 3: framework should maintain the project_mapping table based on the []Scope
 	]
 */
 
-// Scope represents the top level entity for a data source, i.e. github repo,
-// gitlab project, jira board. They turn into repo, board in Domain Layer. In
-// Apache Devlake, a Project is essentially a set of these top level entities,
-// for the framework to maintain these relationships dynamically and
-// automatically, all Domain Layer Top Level Entities should implement this
-// interface
-type Scope interface {
-	ScopeId() string
-	ScopeName() string
-	TableName() string
-}
-
 // DataSourcePluginBlueprintV200 extends the V100 to provide support for
 // Project, so that complex metrics like DORA can be implemented based on a set
 // of Data Scopes
diff --git a/backend/helpers/pluginhelper/api/apihelperabstract/connection_abstract.go b/backend/core/plugin/plugin_connection_abstract.go
similarity index 84%
rename from backend/helpers/pluginhelper/api/apihelperabstract/connection_abstract.go
rename to backend/core/plugin/plugin_connection_abstract.go
index 39feaad0d..92e4a5a00 100644
--- a/backend/helpers/pluginhelper/api/apihelperabstract/connection_abstract.go
+++ b/backend/core/plugin/plugin_connection_abstract.go
@@ -15,10 +15,13 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package apihelperabstract
+package plugin
 
 import (
+	context2 "github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 	"net/http"
+	"net/url"
 
 	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/go-playground/validator/v10"
@@ -31,6 +34,12 @@ type ApiConnection interface {
 	GetRateLimitPerHour() int
 }
 
+type ApiConnectionForRemote[T ApiGroup, S ApiScope] interface {
+	ApiConnection
+	GetGroup(basicRes context2.BasicRes, gid string, query url.Values) ([]T, errors.Error)
+	GetScope(basicRes context2.BasicRes, gid string, query url.Values) ([]S, errors.Error)
+}
+
 // ApiAuthenticator is to be implemented by a Concreate Connection if Authorization is required
 type ApiAuthenticator interface {
 	// SetupAuthentication is a hook function for connection to set up authentication for the HTTP request
@@ -46,7 +55,7 @@ type ConnectionValidator interface {
 // PrepareApiClient is to be implemented by the concrete Connection which requires
 // preparation for the ApiClient created by NewApiClientFromConnection, i.e. fetch token for future requests
 type PrepareApiClient interface {
-	PrepareApiClient(apiClient ApiClientAbstract) errors.Error
+	PrepareApiClient(apiClient apihelperabstract.ApiClientAbstract) errors.Error
 }
 
 // MultiAuth
diff --git a/backend/plugins/gitlab/api/init.go b/backend/core/plugin/plugin_scope_abstract.go
similarity index 50%
copy from backend/plugins/gitlab/api/init.go
copy to backend/core/plugin/plugin_scope_abstract.go
index b4190c90b..772afb046 100644
--- a/backend/plugins/gitlab/api/init.go
+++ b/backend/core/plugin/plugin_scope_abstract.go
@@ -15,30 +15,31 @@ See the License for the specific language governing permissions and
 limitations under the License.
 */
 
-package api
+package plugin
 
-import (
-	"github.com/apache/incubator-devlake/core/context"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	"github.com/apache/incubator-devlake/plugins/gitlab/models"
-	"github.com/go-playground/validator/v10"
-)
+// Scope represents the top level entity for a data source, i.e. github repo,
+// gitlab project, jira board. They turn into repo, board in Domain Layer. In
+// Apache Devlake, a Project is essentially a set of these top level entities,
+// for the framework to maintain these relationships dynamically and
+// automatically, all Domain Layer Top Level Entities should implement this
+// interface
+type Scope interface {
+	ScopeId() string
+	ScopeName() string
+	TableName() string
+}
+
+type ToolLayerScope interface {
+	ScopeId() string
+	ScopeName() string
+	TableName() string
+}
 
-var vld *validator.Validate
-var connectionHelper *api.ConnectionApiHelper
-var scopeHelper *api.ScopeApiHelper[models.GitlabConnection, models.GitlabProject, models.GitlabTransformationRule]
-var basicRes context.BasicRes
+type ApiScope interface {
+	ConvertApiScope() ToolLayerScope
+}
 
-func Init(br context.BasicRes) {
-	basicRes = br
-	vld = validator.New()
-	connectionHelper = api.NewConnectionHelper(
-		basicRes,
-		vld,
-	)
-	scopeHelper = api.NewScopeHelper[models.GitlabConnection, models.GitlabProject, models.GitlabTransformationRule](
-		basicRes,
-		vld,
-		connectionHelper,
-	)
+type ApiGroup interface {
+	GroupId() string
+	GroupName() string
 }
diff --git a/backend/helpers/pluginhelper/api/api_client.go b/backend/helpers/pluginhelper/api/api_client.go
index b7bda0bd5..cc5bc08b8 100644
--- a/backend/helpers/pluginhelper/api/api_client.go
+++ b/backend/helpers/pluginhelper/api/api_client.go
@@ -24,6 +24,7 @@ import (
 	"encoding/json"
 	"encoding/xml"
 	"fmt"
+	aha "github.com/apache/incubator-devlake/core/plugin"
 	"io"
 	"net/http"
 	"net/url"
@@ -38,7 +39,6 @@ import (
 	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/apache/incubator-devlake/core/log"
 	"github.com/apache/incubator-devlake/core/utils"
-	aha "github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/common"
 )
 
diff --git a/backend/helpers/pluginhelper/api/connection_auths.go b/backend/helpers/pluginhelper/api/connection_auths.go
index 50c0973bb..9b7860208 100644
--- a/backend/helpers/pluginhelper/api/connection_auths.go
+++ b/backend/helpers/pluginhelper/api/connection_auths.go
@@ -20,11 +20,11 @@ package api
 import (
 	"encoding/base64"
 	"fmt"
+	"github.com/apache/incubator-devlake/core/plugin"
 	"net/http"
 	"strings"
 
 	"github.com/apache/incubator-devlake/core/errors"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 	"github.com/go-playground/validator/v10"
 )
 
@@ -49,7 +49,7 @@ func (ba *BasicAuth) SetupAuthentication(request *http.Request) errors.Error {
 // it looks odd to return itself with a different type, this is necessary because Callers
 // might call the method from the Outer-Struct(`connection.SetupAuthentication(...)`)
 // which would lead to a Stack Overflow  error
-func (ba *BasicAuth) GetBasicAuthenticator() apihelperabstract.ApiAuthenticator {
+func (ba *BasicAuth) GetBasicAuthenticator() plugin.ApiAuthenticator {
 	return ba
 }
 
@@ -65,7 +65,7 @@ func (at *AccessToken) SetupAuthentication(request *http.Request) errors.Error {
 }
 
 // GetAccessTokenAuthenticator returns SetupAuthentication
-func (at *AccessToken) GetAccessTokenAuthenticator() apihelperabstract.ApiAuthenticator {
+func (at *AccessToken) GetAccessTokenAuthenticator() plugin.ApiAuthenticator {
 	return at
 }
 
@@ -84,7 +84,7 @@ func (ak *AppKey) SetupAuthentication(request *http.Request) errors.Error {
 }
 
 // GetAppKeyAuthenticator returns SetupAuthentication
-func (ak *AppKey) GetAppKeyAuthenticator() apihelperabstract.ApiAuthenticator {
+func (ak *AppKey) GetAppKeyAuthenticator() plugin.ApiAuthenticator {
 	// no universal way to implement AppKey authentication, plugin should alias AppKey and
 	// define its own implementation
 	return ak
@@ -93,33 +93,33 @@ func (ak *AppKey) GetAppKeyAuthenticator() apihelperabstract.ApiAuthenticator {
 // MultiAuth implements the MultiAuthenticator interface
 type MultiAuth struct {
 	AuthMethod       string `mapstructure:"authMethod" json:"authMethod" validate:"required,oneof=BasicAuth AccessToken AppKey"`
-	apiAuthenticator apihelperabstract.ApiAuthenticator
+	apiAuthenticator plugin.ApiAuthenticator
 }
 
-func (ma *MultiAuth) GetApiAuthenticator(connection apihelperabstract.ApiConnection) (apihelperabstract.ApiAuthenticator, errors.Error) {
+func (ma *MultiAuth) GetApiAuthenticator(connection plugin.ApiConnection) (plugin.ApiAuthenticator, errors.Error) {
 	// cache the ApiAuthenticator for performance
 	if ma.apiAuthenticator != nil {
 		return ma.apiAuthenticator, nil
 	}
 	// cache missed
 	switch ma.AuthMethod {
-	case apihelperabstract.AUTH_METHOD_BASIC:
-		basicAuth, ok := connection.(apihelperabstract.BasicAuthenticator)
+	case plugin.AUTH_METHOD_BASIC:
+		basicAuth, ok := connection.(plugin.BasicAuthenticator)
 		if !ok {
 			return nil, errors.Default.New("connection doesn't support Basic Authentication")
 		}
 		ma.apiAuthenticator = basicAuth.GetBasicAuthenticator()
-	case apihelperabstract.AUTH_METHOD_TOKEN:
-		accessToken, ok := connection.(apihelperabstract.AccessTokenAuthenticator)
+	case plugin.AUTH_METHOD_TOKEN:
+		accessToken, ok := connection.(plugin.AccessTokenAuthenticator)
 		if !ok {
 			return nil, errors.Default.New("connection doesn't support AccessToken Authentication")
 		}
 		ma.apiAuthenticator = accessToken.GetAccessTokenAuthenticator()
-	case apihelperabstract.AUTH_METHOD_APPKEY:
+	case plugin.AUTH_METHOD_APPKEY:
 		// Note that AppKey Authentication requires complex logic like signing the request with timestamp
 		// so, there is no way to solve them once and for all, each Specific Connection should implement
 		// on its own.
-		appKey, ok := connection.(apihelperabstract.AppKeyAuthenticator)
+		appKey, ok := connection.(plugin.AppKeyAuthenticator)
 		if !ok {
 			return nil, errors.Default.New("connection doesn't support AppKey Authentication")
 		}
@@ -135,7 +135,7 @@ func (ma *MultiAuth) GetApiAuthenticator(connection apihelperabstract.ApiConnect
 // Specific Connection should implement IAuthentication and then call this method for MultiAuth to work properly,
 // check jira/models/connection.go:JiraConn if you needed an example
 // Note: this method would be called for each request, so it is performance-sensitive, do NOT use reflection here
-func (ma *MultiAuth) SetupAuthenticationForConnection(connection apihelperabstract.ApiConnection, req *http.Request) errors.Error {
+func (ma *MultiAuth) SetupAuthenticationForConnection(connection plugin.ApiConnection, req *http.Request) errors.Error {
 	apiAuthenticator, err := ma.GetApiAuthenticator(connection)
 	if err != nil {
 		return err
@@ -154,7 +154,7 @@ func (ma *MultiAuth) ValidateConnection(connection interface{}, v *validator.Val
 			if len(ns) > 1 {
 				// BasicAuth
 				authName := ns[len(ns)-2]
-				if apihelperabstract.ALL_AUTH[authName] && authName != ma.AuthMethod {
+				if plugin.ALL_AUTH[authName] && authName != ma.AuthMethod {
 					continue
 				}
 				filteredValidationErrors = append(filteredValidationErrors, e)
diff --git a/backend/helpers/pluginhelper/api/connection_helper.go b/backend/helpers/pluginhelper/api/connection_helper.go
index 96f3fbdb9..8d7764f1c 100644
--- a/backend/helpers/pluginhelper/api/connection_helper.go
+++ b/backend/helpers/pluginhelper/api/connection_helper.go
@@ -24,7 +24,6 @@ import (
 	"github.com/apache/incubator-devlake/core/log"
 	"github.com/apache/incubator-devlake/core/models"
 	plugin "github.com/apache/incubator-devlake/core/plugin"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 	"github.com/go-playground/validator/v10"
 	"strconv"
 )
@@ -107,7 +106,7 @@ func (c *ConnectionApiHelper) Delete(connection interface{}) errors.Error {
 
 func (c *ConnectionApiHelper) merge(connection interface{}, body map[string]interface{}) errors.Error {
 	connection = models.UnwrapObject(connection)
-	if connectionValidator, ok := connection.(apihelperabstract.ConnectionValidator); ok {
+	if connectionValidator, ok := connection.(plugin.ConnectionValidator); ok {
 		err := Decode(body, connection, nil)
 		if err != nil {
 			return err
diff --git a/backend/helpers/pluginhelper/api/remote_api_helper.go b/backend/helpers/pluginhelper/api/remote_api_helper.go
new file mode 100644
index 000000000..931ccdbae
--- /dev/null
+++ b/backend/helpers/pluginhelper/api/remote_api_helper.go
@@ -0,0 +1,255 @@
+/*
+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/base64"
+	"encoding/json"
+	"fmt"
+	coreContext "github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/go-playground/validator/v10"
+	"net/http"
+	"net/url"
+)
+
+// RemoteApiHelper is used to write the CURD of connection
+type RemoteApiHelper[Conn plugin.ApiConnectionForRemote[Group, ApiScope], Scope plugin.ToolLayerScope, ApiScope plugin.ApiScope, Group plugin.ApiGroup] struct {
+	basicRes   coreContext.BasicRes
+	validator  *validator.Validate
+	connHelper *ConnectionApiHelper
+}
+
+// NewRemoteHelper creates a ScopeHelper for connection management
+func NewRemoteHelper[Conn plugin.ApiConnectionForRemote[Group, ApiScope], Scope plugin.ToolLayerScope, ApiScope plugin.ApiScope, Group plugin.ApiGroup](
+	basicRes coreContext.BasicRes,
+	vld *validator.Validate,
+	connHelper *ConnectionApiHelper,
+) *RemoteApiHelper[Conn, Scope, ApiScope, Group] {
+	if vld == nil {
+		vld = validator.New()
+	}
+	if connHelper == nil {
+		return nil
+	}
+	return &RemoteApiHelper[Conn, Scope, ApiScope, Group]{
+		basicRes:   basicRes,
+		validator:  vld,
+		connHelper: connHelper,
+	}
+}
+
+type RemoteScopesChild struct {
+	Type     string      `json:"type"`
+	ParentId *string     `json:"parentId"`
+	Id       string      `json:"id"`
+	Name     string      `json:"name"`
+	Data     interface{} `json:"data"`
+}
+
+type RemoteScopesOutput struct {
+	Children      []RemoteScopesChild `json:"children"`
+	NextPageToken string              `json:"nextPageToken"`
+}
+
+type SearchRemoteScopesOutput struct {
+	Children []RemoteScopesChild `json:"children"`
+	Page     int                 `json:"page"`
+	PageSize int                 `json:"pageSize"`
+}
+
+type PageData struct {
+	Page    int    `json:"page"`
+	PerPage int    `json:"per_page"`
+	Tag     string `json:"tag"`
+}
+
+const remoteScopesPerPage int = 100
+const TypeProject string = "scope"
+const TypeGroup string = "group"
+
+func (r *RemoteApiHelper[Conn, Scope, ApiScope, Group]) GetScopesFromRemote(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connectionId, _ := ExtractFromReqParam(input.Params)
+	if connectionId == 0 {
+		return nil, errors.BadInput.New("invalid connectionId")
+	}
+
+	var connection Conn
+	err := r.connHelper.First(&connection, input.Params)
+	if err != nil {
+		return nil, err
+	}
+
+	groupId, ok := input.Query["groupId"]
+	if !ok || len(groupId) == 0 {
+		groupId = []string{""}
+	}
+
+	pageToken, ok := input.Query["pageToken"]
+	if !ok || len(pageToken) == 0 {
+		pageToken = []string{""}
+	}
+
+	// get gid and pageData
+	gid := groupId[0]
+	pageData, err := getPageDataFromPageToken(pageToken[0])
+	if err != nil {
+		return nil, errors.BadInput.New("failed to get paget token")
+	}
+
+	outputBody := &RemoteScopesOutput{}
+
+	// list groups part
+	if pageData.Tag == TypeGroup {
+		query, err := getQueryFromPageData(pageData)
+		if err != nil {
+			return nil, err
+		}
+		var resBody []Group
+		resBody, err = connection.GetGroup(r.basicRes, gid, query)
+		if err != nil {
+			return nil, err
+		}
+
+		// append group to output
+		for _, group := range resBody {
+			child := RemoteScopesChild{
+				Type: TypeGroup,
+				Id:   group.GroupId(),
+				Name: group.GroupName(),
+				// don't need to save group into data
+				Data: nil,
+			}
+			child.ParentId = &gid
+			if *child.ParentId == "" {
+				child.ParentId = nil
+			}
+			outputBody.Children = append(outputBody.Children, child)
+		}
+		// check groups count
+		if len(resBody) < pageData.PerPage {
+			pageData.Tag = TypeProject
+			pageData.Page = 1
+			pageData.PerPage = pageData.PerPage - len(resBody)
+		}
+	}
+
+	// list projects part
+	if pageData.Tag == TypeProject {
+		query, err := getQueryFromPageData(pageData)
+		if err != nil {
+			return nil, err
+		}
+		var resBody []ApiScope
+
+		resBody, err = connection.GetScope(r.basicRes, gid, query)
+		if err != nil {
+			return nil, err
+		}
+
+		// append project to output
+		for _, project := range resBody {
+			scope := project.ConvertApiScope()
+			child := RemoteScopesChild{
+				Type: TypeProject,
+				Id:   scope.ScopeId(),
+				Name: scope.ScopeName(),
+				Data: &scope,
+			}
+			child.ParentId = &gid
+			if *child.ParentId == "" {
+				child.ParentId = nil
+			}
+
+			outputBody.Children = append(outputBody.Children, child)
+		}
+
+		// check project count
+		if len(resBody) < pageData.PerPage {
+			pageData = nil
+		}
+	}
+
+	// get the next page token
+	outputBody.NextPageToken = ""
+	if pageData != nil {
+		pageData.Page += 1
+		pageData.PerPage = remoteScopesPerPage
+
+		outputBody.NextPageToken, err = getPageTokenFromPageData(pageData)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	return &plugin.ApiResourceOutput{Body: outputBody, Status: http.StatusOK}, nil
+}
+
+func getPageTokenFromPageData(pageData *PageData) (string, errors.Error) {
+	// Marshal json
+	pageTokenDecode, err := json.Marshal(pageData)
+	if err != nil {
+		return "", errors.Default.Wrap(err, fmt.Sprintf("Marshal pageToken failed %+v", pageData))
+	}
+
+	// Encode pageToken Base64
+	return base64.StdEncoding.EncodeToString(pageTokenDecode), nil
+}
+
+func getPageDataFromPageToken(pageToken string) (*PageData, errors.Error) {
+	if pageToken == "" {
+		return &PageData{
+			Page:    1,
+			PerPage: remoteScopesPerPage,
+			Tag:     "group",
+		}, nil
+	}
+
+	// Decode pageToken Base64
+	pageTokenDecode, err := base64.StdEncoding.DecodeString(pageToken)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, fmt.Sprintf("decode pageToken failed %s", pageToken))
+	}
+	// Unmarshal json
+	pt := &PageData{}
+	err = json.Unmarshal(pageTokenDecode, pt)
+	if err != nil {
+		return nil, errors.Default.Wrap(err, fmt.Sprintf("json Unmarshal pageTokenDecode failed %s", pageTokenDecode))
+	}
+
+	return pt, nil
+}
+
+func getQueryFromPageData(pageData *PageData) (url.Values, errors.Error) {
+	query := url.Values{}
+	query.Set("page", fmt.Sprintf("%v", pageData.Page))
+	query.Set("per_page", fmt.Sprintf("%v", pageData.PerPage))
+	return query, nil
+}
+
+func GetQueryForSearchProject(search string, page int, perPage int) (url.Values, errors.Error) {
+	query, err := getQueryFromPageData(&PageData{Page: page, PerPage: perPage})
+	if err != nil {
+		return nil, err
+	}
+	query.Set("search", search)
+	query.Set("scope", "projects")
+
+	return query, nil
+}
diff --git a/backend/helpers/pluginhelper/api/scope_helper.go b/backend/helpers/pluginhelper/api/scope_helper.go
index 749372831..d34069ffd 100644
--- a/backend/helpers/pluginhelper/api/scope_helper.go
+++ b/backend/helpers/pluginhelper/api/scope_helper.go
@@ -86,7 +86,7 @@ func (c *ScopeApiHelper[Conn, Scope, Tr]) Put(input *plugin.ApiResourceInput) (*
 		return nil, errors.BadInput.Wrap(err, "decoding Github repo error")
 	}
 	// Extract the connection ID from the input.Params map
-	connectionId, _ := extractFromReqParam(input.Params)
+	connectionId, _ := ExtractFromReqParam(input.Params)
 	if connectionId == 0 {
 		return nil, errors.BadInput.New("invalid connectionId or scopeId")
 	}
@@ -127,7 +127,7 @@ func (c *ScopeApiHelper[Conn, Scope, Tr]) Put(input *plugin.ApiResourceInput) (*
 }
 
 func (c *ScopeApiHelper[Conn, Scope, Tr]) Update(input *plugin.ApiResourceInput, fieldName string) (*plugin.ApiResourceOutput, errors.Error) {
-	connectionId, scopeId := extractFromReqParam(input.Params)
+	connectionId, scopeId := ExtractFromReqParam(input.Params)
 
 	if connectionId == 0 || len(scopeId) == 0 || scopeId == "0" {
 		return &plugin.ApiResourceOutput{Body: nil, Status: http.StatusInternalServerError}, errors.BadInput.New("invalid connectionId")
@@ -158,7 +158,7 @@ func (c *ScopeApiHelper[Conn, Scope, Tr]) Update(input *plugin.ApiResourceInput,
 }
 
 func (c *ScopeApiHelper[Conn, Scope, Tr]) GetScopeList(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
-	connectionId, _ := extractFromReqParam(input.Params)
+	connectionId, _ := ExtractFromReqParam(input.Params)
 	if connectionId == 0 {
 		return nil, errors.BadInput.New("invalid path params")
 	}
@@ -204,7 +204,7 @@ func (c *ScopeApiHelper[Conn, Scope, Tr]) GetScopeList(input *plugin.ApiResource
 }
 
 func (c *ScopeApiHelper[Conn, Scope, Tr]) GetScope(input *plugin.ApiResourceInput, fieldName string) (*plugin.ApiResourceOutput, errors.Error) {
-	connectionId, scopeId := extractFromReqParam(input.Params)
+	connectionId, scopeId := ExtractFromReqParam(input.Params)
 	if connectionId == 0 || len(scopeId) == 0 || scopeId == "0" {
 		return nil, errors.BadInput.New("invalid path params")
 	}
@@ -262,7 +262,7 @@ func (c *ScopeApiHelper[Conn, Scope, Tr]) save(scope interface{}) errors.Error {
 	return nil
 }
 
-func extractFromReqParam(params map[string]string) (uint64, string) {
+func ExtractFromReqParam(params map[string]string) (uint64, string) {
 	connectionId, err := strconv.ParseUint(params["connectionId"], 10, 64)
 	if err != nil {
 		return 0, ""
diff --git a/backend/plugins/bitbucket/api/remote.go b/backend/plugins/bitbucket/api/remote.go
index 49f85b8a7..c94cb1c18 100644
--- a/backend/plugins/bitbucket/api/remote.go
+++ b/backend/plugins/bitbucket/api/remote.go
@@ -58,23 +58,33 @@ type PageData struct {
 }
 
 type WorkspaceResponse struct {
-	Pagelen int `json:"pagelen"`
-	Page    int `json:"page"`
-	Size    int `json:"size"`
-	Values  []struct {
-		//Type       string `json:"type"`
-		//Permission string `json:"permission"`
-		//LastAccessed time.Time `json:"last_accessed"`
-		//AddedOn      time.Time `json:"added_on"`
-		Workspace WorkspaceItem `json:"workspace"`
-	} `json:"values"`
+	Pagelen int             `json:"pagelen"`
+	Page    int             `json:"page"`
+	Size    int             `json:"size"`
+	Values  []GroupResponse `json:"values"`
+}
+
+type GroupResponse struct {
+	//Type       string `json:"type"`
+	//Permission string `json:"permission"`
+	//LastAccessed time.Time `json:"last_accessed"`
+	//AddedOn      time.Time `json:"added_on"`
+	Workspace WorkspaceItem `json:"workspace"`
+}
+
+func (p GroupResponse) ScopeId() string {
+	return p.Workspace.Slug
+}
+
+func (p GroupResponse) ScopeName() string {
+	return p.Workspace.Name
 }
 
 type WorkspaceItem struct {
 	//Type string `json:"type"`
 	//Uuid string `json:"uuid"`
-	Slug string `json:"slug"`
-	Name string `json:"name"`
+	Slug string `json:"slug" group:"id"`
+	Name string `json:"name" group:"name"`
 }
 
 type ReposResponse struct {
diff --git a/backend/plugins/github/models/repo.go b/backend/plugins/github/models/repo.go
index 9eda5e118..4b4cf6a30 100644
--- a/backend/plugins/github/models/repo.go
+++ b/backend/plugins/github/models/repo.go
@@ -19,9 +19,13 @@ package models
 
 import (
 	"github.com/apache/incubator-devlake/core/models/common"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"strconv"
 	"time"
 )
 
+var _ plugin.ToolLayerScope = (*GithubRepo)(nil)
+
 type GithubRepo struct {
 	ConnectionId         uint64     `json:"connectionId" gorm:"primaryKey" validate:"required" mapstructure:"connectionId,omitempty"`
 	GithubId             int        `json:"githubId" gorm:"primaryKey" validate:"required" mapstructure:"githubId"`
@@ -39,6 +43,14 @@ type GithubRepo struct {
 	common.NoPKModel     `json:"-" mapstructure:"-"`
 }
 
+func (r GithubRepo) ScopeId() string {
+	return strconv.Itoa(r.GithubId)
+}
+
+func (r GithubRepo) ScopeName() string {
+	return r.Name
+}
+
 func (GithubRepo) TableName() string {
 	return "_tool_github_repos"
 }
diff --git a/backend/plugins/gitlab/api/blueprint_v200.go b/backend/plugins/gitlab/api/blueprint_v200.go
index 477cce1ad..6950ae772 100644
--- a/backend/plugins/gitlab/api/blueprint_v200.go
+++ b/backend/plugins/gitlab/api/blueprint_v200.go
@@ -234,8 +234,8 @@ func GetTransformationRuleByRepo(repo *models.GitlabProject) (*models.GitlabTran
 func GetApiProject(
 	op *tasks.GitlabOptions,
 	apiClient aha.ApiClientAbstract,
-) (*tasks.GitlabApiProject, errors.Error) {
-	repoRes := &tasks.GitlabApiProject{}
+) (*models.GitlabApiProject, errors.Error) {
+	repoRes := &models.GitlabApiProject{}
 	res, err := apiClient.Get(fmt.Sprintf("projects/%d", op.ProjectId), nil, nil)
 	if err != nil {
 		return nil, err
diff --git a/backend/plugins/gitlab/api/init.go b/backend/plugins/gitlab/api/init.go
index b4190c90b..444534799 100644
--- a/backend/plugins/gitlab/api/init.go
+++ b/backend/plugins/gitlab/api/init.go
@@ -27,6 +27,7 @@ import (
 var vld *validator.Validate
 var connectionHelper *api.ConnectionApiHelper
 var scopeHelper *api.ScopeApiHelper[models.GitlabConnection, models.GitlabProject, models.GitlabTransformationRule]
+var remoteHelper *api.RemoteApiHelper[models.GitlabConnection, models.GitlabProject, models.GitlabApiProject, models.GroupResponse]
 var basicRes context.BasicRes
 
 func Init(br context.BasicRes) {
@@ -41,4 +42,10 @@ func Init(br context.BasicRes) {
 		vld,
 		connectionHelper,
 	)
+
+	remoteHelper = api.NewRemoteHelper[models.GitlabConnection, models.GitlabProject, models.GitlabApiProject, models.GroupResponse](
+		basicRes,
+		vld,
+		connectionHelper,
+	)
 }
diff --git a/backend/plugins/gitlab/api/remote.go b/backend/plugins/gitlab/api/remote.go
index a0aadcb96..609e072f7 100644
--- a/backend/plugins/gitlab/api/remote.go
+++ b/backend/plugins/gitlab/api/remote.go
@@ -92,165 +92,7 @@ const TypeGroup string = "group"
 // @Failure 500  {object} shared.ApiBody "Internal Error"
 // @Router /plugins/gitlab/connections/{connectionId}/remote-scopes [GET]
 func RemoteScopes(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
-	connectionId, _ := extractParam(input.Params)
-	if connectionId == 0 {
-		return nil, errors.BadInput.New("invalid connectionId")
-	}
-
-	connection := &models.GitlabConnection{}
-	err := connectionHelper.First(connection, input.Params)
-	if err != nil {
-		return nil, err
-	}
-
-	groupId, ok := input.Query["groupId"]
-	if !ok || len(groupId) == 0 {
-		groupId = []string{""}
-	}
-
-	pageToken, ok := input.Query["pageToken"]
-	if !ok || len(pageToken) == 0 {
-		pageToken = []string{""}
-	}
-
-	// get gid and pageData
-	gid := groupId[0]
-	pageData, err := GetPageDataFromPageToken(pageToken[0])
-	if err != nil {
-		return nil, errors.BadInput.New("failed to get paget token")
-	}
-
-	// create api client
-	apiClient, err := api.NewApiClientFromConnection(context.TODO(), basicRes, connection)
-	if err != nil {
-		return nil, err
-	}
-
-	var res *http.Response
-	outputBody := &RemoteScopesOutput{}
-
-	// list groups part
-	if pageData.Tag == TypeGroup {
-		query, err := GetQueryFromPageData(pageData)
-		if err != nil {
-			return nil, err
-		}
-
-		if gid == "" {
-			query.Set("top_level_only", "true")
-			res, err = apiClient.Get("groups", query, nil)
-		} else {
-			res, err = apiClient.Get(fmt.Sprintf("groups/%s/subgroups", gid), query, nil)
-		}
-		if err != nil {
-			return nil, err
-		}
-
-		resBody := []GroupResponse{}
-		err = api.UnmarshalResponse(res, &resBody)
-		if err != nil {
-			return nil, err
-		}
-
-		// append group to output
-		for _, group := range resBody {
-			child := RemoteScopesChild{
-				Type: TypeGroup,
-				Id:   strconv.Itoa(group.Id),
-				Name: group.Name,
-				// don't need to save group into data
-				Data: nil,
-			}
-
-			// ignore not top_level
-			if group.ParentId == nil {
-				if gid != "" {
-					continue
-				}
-			} else {
-				if strconv.Itoa(*group.ParentId) != gid {
-					continue
-				}
-			}
-
-			// ignore self
-			if gid == child.Id {
-				continue
-			}
-
-			child.ParentId = &gid
-			if *child.ParentId == "" {
-				child.ParentId = nil
-			}
-
-			outputBody.Children = append(outputBody.Children, child)
-		}
-
-		// check groups count
-		if len(resBody) < pageData.PerPage {
-			pageData.Tag = TypeProject
-			pageData.Page = 1
-			pageData.PerPage = pageData.PerPage - len(resBody)
-		}
-	}
-
-	// list projects part
-	if pageData.Tag == TypeProject {
-		query, err := GetQueryFromPageData(pageData)
-		if err != nil {
-			return nil, err
-		}
-		if gid == "" {
-			res, err = apiClient.Get(fmt.Sprintf("users/%d/projects", apiClient.GetData(models.GitlabApiClientData_UserId)), query, nil)
-		} else {
-			query.Set("with_shared", "false")
-			res, err = apiClient.Get(fmt.Sprintf("/groups/%s/projects", gid), query, nil)
-		}
-		if err != nil {
-			return nil, err
-		}
-
-		resBody := []tasks.GitlabApiProject{}
-		err = api.UnmarshalResponse(res, &resBody)
-		if err != nil {
-			return nil, err
-		}
-
-		// append project to output
-		for _, project := range resBody {
-			child := RemoteScopesChild{
-				Type: TypeProject,
-				Id:   strconv.Itoa(project.GitlabId),
-				Name: project.Name,
-				Data: tasks.ConvertProject(&project),
-			}
-			child.ParentId = &gid
-			if *child.ParentId == "" {
-				child.ParentId = nil
-			}
-
-			outputBody.Children = append(outputBody.Children, child)
-		}
-
-		// check project count
-		if len(resBody) < pageData.PerPage {
-			pageData = nil
-		}
-	}
-
-	// get the next page token
-	outputBody.NextPageToken = ""
-	if pageData != nil {
-		pageData.Page += 1
-		pageData.PerPage = GitlabRemoteScopesPerPage
-
-		outputBody.NextPageToken, err = GetPageTokenFromPageData(pageData)
-		if err != nil {
-			return nil, err
-		}
-	}
-
-	return &plugin.ApiResourceOutput{Body: outputBody, Status: http.StatusOK}, nil
+	return remoteHelper.GetScopesFromRemote(input)
 }
 
 // SearchRemoteScopes use the Search API and only return project
diff --git a/backend/plugins/gitlab/impl/impl.go b/backend/plugins/gitlab/impl/impl.go
index 99c2dd968..aa1f38d67 100644
--- a/backend/plugins/gitlab/impl/impl.go
+++ b/backend/plugins/gitlab/impl/impl.go
@@ -177,13 +177,14 @@ func (p Gitlab) PrepareTaskData(taskCtx plugin.TaskContext, options map[string]i
 		db := taskCtx.GetDal()
 		err = db.First(&scope, dal.Where("connection_id = ? AND gitlab_id = ?", op.ConnectionId, op.ProjectId))
 		if err != nil && db.IsErrorNotFound(err) {
-			var project *tasks.GitlabApiProject
+			var project *models.GitlabApiProject
 			project, err = api.GetApiProject(op, apiClient)
 			if err != nil {
 				return nil, err
 			}
 			logger.Debug(fmt.Sprintf("Current project: %d", project.GitlabId))
-			scope = tasks.ConvertProject(project)
+			i := project.ConvertApiScope()
+			scope = i.(*models.GitlabProject)
 			scope.ConnectionId = op.ConnectionId
 			err = taskCtx.GetDal().CreateIfNotExist(&scope)
 			if err != nil {
diff --git a/backend/plugins/gitlab/models/connection.go b/backend/plugins/gitlab/models/connection.go
index 1dcb19863..aa55a79d2 100644
--- a/backend/plugins/gitlab/models/connection.go
+++ b/backend/plugins/gitlab/models/connection.go
@@ -18,8 +18,12 @@ limitations under the License.
 package models
 
 import (
+	"context"
 	"fmt"
+	context2 "github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/plugin"
 	"net/http"
+	"net/url"
 
 	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
@@ -47,10 +51,10 @@ func (conn *GitlabConn) PrepareApiClient(apiClient apihelperabstract.ApiClientAb
 	// test request for access token
 	userResBody := &ApiUserResponse{}
 	res, err := apiClient.Get("user", nil, header1)
+	if err != nil {
+		return err
+	}
 	if res.StatusCode != http.StatusUnauthorized {
-		if err != nil {
-			return errors.Convert(err)
-		}
 		err = api.UnmarshalResponse(res, userResBody)
 		if err != nil {
 			return errors.Convert(err)
@@ -104,6 +108,9 @@ func (conn *GitlabConn) PrepareApiClient(apiClient apihelperabstract.ApiClientAb
 	return nil
 }
 
+var _ plugin.ApiConnectionForRemote[GroupResponse, GitlabApiProject] = (*GitlabConnection)(nil)
+var _ plugin.ApiGroup = (*GroupResponse)(nil)
+
 // GitlabConnection holds GitlabConn plus ID/Name for database storage
 type GitlabConnection struct {
 	api.BaseConnection `mapstructure:",squash"`
@@ -131,3 +138,55 @@ type ApiUserResponse struct {
 func (GitlabConnection) TableName() string {
 	return "_tool_gitlab_connections"
 }
+
+func (g GitlabConnection) GetGroup(basicRes context2.BasicRes, gid string, query url.Values) ([]GroupResponse, errors.Error) {
+	apiClient, err := api.NewApiClientFromConnection(context.TODO(), basicRes, &g)
+	if err != nil {
+		return nil, errors.BadInput.Wrap(err, "failed to get create apiClient")
+	}
+	var res *http.Response
+	if gid == "" {
+		query.Set("top_level_only", "true")
+		res, err = apiClient.Get("groups", query, nil)
+		if err != nil {
+			return nil, err
+		}
+	} else {
+		res, err = apiClient.Get(fmt.Sprintf("groups/%s/subgroups", gid), query, nil)
+		if err != nil {
+			return nil, err
+		}
+	}
+	var resBody []GroupResponse
+	err = api.UnmarshalResponse(res, &resBody)
+	if err != nil {
+		return nil, err
+	}
+	return resBody, err
+}
+
+func (g GitlabConnection) GetScope(basicRes context2.BasicRes, gid string, query url.Values) ([]GitlabApiProject, errors.Error) {
+	apiClient, err := api.NewApiClientFromConnection(context.TODO(), basicRes, &g)
+	if err != nil {
+		return nil, errors.BadInput.Wrap(err, "failed to get create apiClient")
+	}
+	var res *http.Response
+	if gid == "" {
+		res, err = apiClient.Get(fmt.Sprintf("users/%d/projects", apiClient.GetData("UserId")), query, nil)
+		if err != nil {
+			return nil, err
+		}
+	} else {
+		query.Set("with_shared", "false")
+		res, err = apiClient.Get(fmt.Sprintf("/groups/%s/projects", gid), query, nil)
+		if err != nil {
+			return nil, err
+		}
+	}
+	var resBody []GitlabApiProject
+	err = api.UnmarshalResponse(res, &resBody)
+	if err != nil {
+		return nil, err
+	}
+	return resBody, err
+}
diff --git a/backend/plugins/gitlab/models/project.go b/backend/plugins/gitlab/models/project.go
index 8db83c195..af08f9128 100644
--- a/backend/plugins/gitlab/models/project.go
+++ b/backend/plugins/gitlab/models/project.go
@@ -18,11 +18,16 @@ limitations under the License.
 package models
 
 import (
+	"github.com/apache/incubator-devlake/core/plugin"
+	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"strconv"
 	"time"
 
 	"github.com/apache/incubator-devlake/core/models/common"
 )
 
+var _ plugin.ToolLayerScope = (*GitlabProject)(nil)
+
 type GitlabProject struct {
 	ConnectionId            uint64 `json:"connectionId" mapstructure:"connectionId" validate:"required" gorm:"primaryKey"`
 	TransformationRuleId    uint64 `json:"transformationRuleId,omitempty" mapstructure:"transformationRuleId"`
@@ -48,3 +53,76 @@ type GitlabProject struct {
 func (GitlabProject) TableName() string {
 	return "_tool_gitlab_projects"
 }
+
+func (p GitlabProject) ScopeId() string {
+	return strconv.Itoa(p.GitlabId)
+}
+
+func (p GitlabProject) ScopeName() string {
+	return p.Name
+}
+
+// Convert the API response to our DB model instance
+func (gitlabApiProject GitlabApiProject) ConvertApiScope() plugin.ToolLayerScope {
+	p := &GitlabProject{}
+	p.GitlabId = gitlabApiProject.GitlabId
+	p.Name = gitlabApiProject.Name
+	p.Description = gitlabApiProject.Description
+	p.DefaultBranch = gitlabApiProject.DefaultBranch
+	p.CreatorId = gitlabApiProject.CreatorId
+	p.PathWithNamespace = gitlabApiProject.PathWithNamespace
+	p.WebUrl = gitlabApiProject.WebUrl
+	p.HttpUrlToRepo = gitlabApiProject.HttpUrlToRepo
+	p.Visibility = gitlabApiProject.Visibility
+	p.OpenIssuesCount = gitlabApiProject.OpenIssuesCount
+	p.StarCount = gitlabApiProject.StarCount
+	p.CreatedDate = gitlabApiProject.CreatedAt.ToNullableTime()
+	p.UpdatedDate = helper.Iso8601TimeToTime(gitlabApiProject.LastActivityAt)
+	if gitlabApiProject.ForkedFromProject != nil {
+		p.ForkedFromProjectId = gitlabApiProject.ForkedFromProject.GitlabId
+		p.ForkedFromProjectWebUrl = gitlabApiProject.ForkedFromProject.WebUrl
+	}
+	return p
+}
+
+type GitlabApiProject struct {
+	GitlabId          int    `json:"id"`
+	Name              string `josn:"name"`
+	Description       string `json:"description"`
+	DefaultBranch     string `json:"default_branch"`
+	PathWithNamespace string `json:"path_with_namespace"`
+	WebUrl            string `json:"web_url"`
+	CreatorId         int
+	Visibility        string              `json:"visibility"`
+	OpenIssuesCount   int                 `json:"open_issues_count"`
+	StarCount         int                 `json:"star_count"`
+	ForkedFromProject *GitlabApiProject   `json:"forked_from_project"`
+	CreatedAt         helper.Iso8601Time  `json:"created_at"`
+	LastActivityAt    *helper.Iso8601Time `json:"last_activity_at"`
+	HttpUrlToRepo     string              `json:"http_url_to_repo"`
+}
+
+type GroupResponse struct {
+	Id                   int    `json:"id" group:"id"`
+	WebUrl               string `json:"web_url"`
+	Name                 string `json:"name" group:"name"`
+	Path                 string `json:"path"`
+	Description          string `json:"description"`
+	Visibility           string `json:"visibility"`
+	LfsEnabled           bool   `json:"lfs_enabled"`
+	AvatarUrl            string `json:"avatar_url"`
+	RequestAccessEnabled bool   `json:"request_access_enabled"`
+	FullName             string `json:"full_name"`
+	FullPath             string `json:"full_path"`
+	ParentId             *int   `json:"parent_id"`
+	LdapCN               string `json:"ldap_cn"`
+	LdapAccess           string `json:"ldap_access"`
+}
+
+func (p GroupResponse) GroupId() string {
+	return strconv.Itoa(p.Id)
+}
+
+func (p GroupResponse) GroupName() string {
+	return p.Name
+}
diff --git a/backend/plugins/jira/models/migrationscripts/20230129_add_multi_auth.go b/backend/plugins/jira/models/migrationscripts/20230129_add_multi_auth.go
index 868eeb918..a3ac1f262 100644
--- a/backend/plugins/jira/models/migrationscripts/20230129_add_multi_auth.go
+++ b/backend/plugins/jira/models/migrationscripts/20230129_add_multi_auth.go
@@ -21,8 +21,8 @@ import (
 	"github.com/apache/incubator-devlake/core/context"
 	"github.com/apache/incubator-devlake/core/dal"
 	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
 	"github.com/apache/incubator-devlake/helpers/migrationhelper"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 )
 
 type jiraMultiAuth20230129 struct {
@@ -43,7 +43,7 @@ func (script *addJiraMultiAuth20230129) Up(basicRes context.BasicRes) errors.Err
 	}
 	return basicRes.GetDal().UpdateColumn(
 		&jiraMultiAuth20230129{},
-		"auth_method", apihelperabstract.AUTH_METHOD_BASIC,
+		"auth_method", plugin.AUTH_METHOD_BASIC,
 		dal.Where("auth_method IS NULL"),
 	)
 }
diff --git a/backend/plugins/sonarqube/models/connection.go b/backend/plugins/sonarqube/models/connection.go
index 6d29013de..3309d5d28 100644
--- a/backend/plugins/sonarqube/models/connection.go
+++ b/backend/plugins/sonarqube/models/connection.go
@@ -21,8 +21,8 @@ import (
 	"encoding/base64"
 	"fmt"
 	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
 	helper "github.com/apache/incubator-devlake/helpers/pluginhelper/api"
-	"github.com/apache/incubator-devlake/helpers/pluginhelper/api/apihelperabstract"
 	"net/http"
 )
 
@@ -34,7 +34,7 @@ func (sat SonarqubeAccessToken) SetupAuthentication(req *http.Request) errors.Er
 	return nil
 }
 
-func (sat SonarqubeAccessToken) GetAccessTokenAuthenticator() apihelperabstract.ApiAuthenticator {
+func (sat SonarqubeAccessToken) GetAccessTokenAuthenticator() plugin.ApiAuthenticator {
 	return sat
 }