You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by wa...@apache.org on 2022/09/10 02:35:48 UTC

[incubator-devlake] 01/02: feat: new plugin for gitea

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

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

commit 1a0ac51fe752f2a1b9d4b61ec6c3e3f10aa3de6d
Author: tk103331 <tk...@gmail.com>
AuthorDate: Wed Sep 7 13:07:29 2022 +0800

    feat: new plugin for gitea
---
 plugins/gitea/README.md                            |  12 ++
 plugins/gitea/api/blueprint.go                     | 152 ++++++++++++++++++
 plugins/gitea/api/connection.go                    | 164 +++++++++++++++++++
 plugins/gitea/api/init.go                          |  39 +++++
 plugins/gitea/api/swagger.go                       |  49 ++++++
 plugins/gitea/gitea.go                             |  46 ++++++
 plugins/gitea/impl/impl.go                         | 173 +++++++++++++++++++++
 plugins/gitea/models/account.go                    |  42 +++++
 plugins/gitea/models/commit.go                     |  47 ++++++
 plugins/gitea/models/commit_stats.go               |  37 +++++
 plugins/gitea/models/connection.go                 |  46 ++++++
 plugins/gitea/models/issue.go                      |  52 +++++++
 plugins/gitea/models/issue_comment.go              |  43 +++++
 plugins/gitea/models/issue_label.go                |  39 +++++
 .../migrationscripts/20220830_add_init_tables.go   |  88 +++++++++++
 .../models/migrationscripts/archived/account.go    |  37 +++++
 .../models/migrationscripts/archived/commit.go     |  47 ++++++
 .../migrationscripts/archived/commit_stats.go      |  37 +++++
 .../models/migrationscripts/archived/connection.go |  58 +++++++
 .../models/migrationscripts/archived/issue.go      |  52 +++++++
 .../migrationscripts/archived/issue_comment.go     |  43 +++++
 .../migrationscripts/archived/issue_label.go       |  39 +++++
 .../gitea/models/migrationscripts/archived/repo.go |  45 ++++++
 .../migrationscripts/archived/repo_commit.go       |  33 ++++
 .../models/migrationscripts/archived/reviewer.go   |  35 +++++
 plugins/gitea/models/migrationscripts/register.go  |  29 ++++
 plugins/gitea/models/repo.go                       |  45 ++++++
 plugins/gitea/models/repo_commit.go                |  33 ++++
 plugins/gitea/models/reviewer.go                   |  35 +++++
 plugins/gitea/tasks/account_convertor.go           |  74 +++++++++
 plugins/gitea/tasks/api_client.go                  |  51 ++++++
 plugins/gitea/tasks/commit_collector.go            |  89 +++++++++++
 plugins/gitea/tasks/commit_convertor.go            | 103 ++++++++++++
 plugins/gitea/tasks/commit_extractor.go            | 126 +++++++++++++++
 plugins/gitea/tasks/commit_stats_collector.go      | 110 +++++++++++++
 plugins/gitea/tasks/commit_stats_extractor.go      | 102 ++++++++++++
 plugins/gitea/tasks/issue_collector.go             | 102 ++++++++++++
 plugins/gitea/tasks/issue_comment_collector.go     |  93 +++++++++++
 plugins/gitea/tasks/issue_comment_convertor.go     |  85 ++++++++++
 plugins/gitea/tasks/issue_comment_extractor.go     | 114 ++++++++++++++
 plugins/gitea/tasks/issue_convertor.go             | 104 +++++++++++++
 plugins/gitea/tasks/issue_extractor.go             | 167 ++++++++++++++++++++
 plugins/gitea/tasks/issue_label_convertor.go       |  78 ++++++++++
 plugins/gitea/tasks/repo_collector.go              |  72 +++++++++
 plugins/gitea/tasks/repo_convertor.go              |  99 ++++++++++++
 plugins/gitea/tasks/repo_extractor.go              |  93 +++++++++++
 plugins/gitea/tasks/shared.go                      | 101 ++++++++++++
 plugins/gitea/tasks/task_data.go                   |  60 +++++++
 48 files changed, 3420 insertions(+)

diff --git a/plugins/gitea/README.md b/plugins/gitea/README.md
new file mode 100644
index 00000000..e6f204fe
--- /dev/null
+++ b/plugins/gitea/README.md
@@ -0,0 +1,12 @@
+# Gitea Pond
+
+<div align="center">
+
+| [English](README.md) | [中文](README-zh-CN.md) |
+| --- | --- |
+
+</div>
+
+<br>
+
+Please see details in the [Apache DevLake website](https://devlake.apache.org/docs/Plugins/gitea)
\ No newline at end of file
diff --git a/plugins/gitea/api/blueprint.go b/plugins/gitea/api/blueprint.go
new file mode 100644
index 00000000..00e927d2
--- /dev/null
+++ b/plugins/gitea/api/blueprint.go
@@ -0,0 +1,152 @@
+/*
+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 (
+	"context"
+	"encoding/json"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"net/url"
+	"strings"
+	"time"
+
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/gitea/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/utils"
+)
+
+func MakePipelinePlan(subtaskMetas []core.SubTaskMeta, connectionId uint64, scope []*core.BlueprintScopeV100) (core.PipelinePlan, error) {
+	var err error
+	plan := make(core.PipelinePlan, len(scope))
+	for i, scopeElem := range scope {
+		// handle taskOptions and transformationRules, by dumping them to taskOptions
+		transformationRules := make(map[string]interface{})
+		if len(scopeElem.Transformation) > 0 {
+			err = json.Unmarshal(scopeElem.Transformation, &transformationRules)
+			if err != nil {
+				return nil, err
+			}
+		}
+		// refdiff
+		if refdiffRules, ok := transformationRules["refdiff"]; ok && refdiffRules != nil {
+			// add a new task to next stage
+			j := i + 1
+			if j == len(plan) {
+				plan = append(plan, nil)
+			}
+			plan[j] = core.PipelineStage{
+				{
+					Plugin:  "refdiff",
+					Options: refdiffRules.(map[string]interface{}),
+				},
+			}
+			// remove it from github transformationRules
+			delete(transformationRules, "refdiff")
+		}
+		// construct task options for github
+		options := make(map[string]interface{})
+		err = json.Unmarshal(scopeElem.Options, &options)
+		if err != nil {
+			return nil, err
+		}
+		options["connectionId"] = connectionId
+		options["transformationRules"] = transformationRules
+		// make sure task options is valid
+		op, err := tasks.DecodeAndValidateTaskOptions(options)
+		if err != nil {
+			return nil, err
+		}
+		// construct subtasks
+		subtasks, err := helper.MakePipelinePlanSubtasks(subtaskMetas, scopeElem.Entities)
+		if err != nil {
+			return nil, err
+		}
+		stage := plan[i]
+		if stage == nil {
+			stage = core.PipelineStage{}
+		}
+		stage = append(stage, &core.PipelineTask{
+			Plugin:   "gitea",
+			Subtasks: subtasks,
+			Options:  options,
+		})
+		// collect git data by gitextractor if CODE was requested
+		if utils.StringsContains(scopeElem.Entities, core.DOMAIN_TYPE_CODE) {
+			// here is the tricky part, we have to obtain the repo id beforehand
+			connection := new(models.GiteaConnection)
+			err = connectionHelper.FirstById(connection, connectionId)
+			if err != nil {
+				return nil, err
+			}
+			token := strings.Split(connection.Token, ",")[0]
+			apiClient, err := helper.NewApiClient(
+				context.TODO(),
+				connection.Endpoint,
+				map[string]string{
+					"Authorization": fmt.Sprintf("token %s", token),
+				},
+				10*time.Second,
+				connection.Proxy,
+				basicRes,
+			)
+			if err != nil {
+				return nil, err
+			}
+			res, err := apiClient.Get(fmt.Sprintf("repos/%s/%s", op.Owner, op.Repo), nil, nil)
+			if err != nil {
+				return nil, err
+			}
+			defer res.Body.Close()
+			if res.StatusCode != http.StatusOK {
+				return nil, fmt.Errorf(
+					"unexpected status code when requesting repo detail %d %s",
+					res.StatusCode, res.Request.URL.String(),
+				)
+			}
+			body, err := ioutil.ReadAll(res.Body)
+			if err != nil {
+				return nil, err
+			}
+			apiRepo := new(tasks.GiteaApiRepoResponse)
+			err = json.Unmarshal(body, apiRepo)
+			if err != nil {
+				return nil, err
+			}
+			cloneUrl, err := url.Parse(apiRepo.CloneUrl)
+			if err != nil {
+				return nil, err
+			}
+			cloneUrl.User = url.UserPassword("git", token)
+			stage = append(stage, &core.PipelineTask{
+				Plugin: "gitextractor",
+				Options: map[string]interface{}{
+					"url":    cloneUrl.String(),
+					"repoId": didgen.NewDomainIdGenerator(&models.GiteaRepo{}).Generate(connectionId, apiRepo.GiteaId),
+					"proxy":  connection.Proxy,
+				},
+			})
+		}
+		plan[i] = stage
+	}
+	return plan, nil
+}
diff --git a/plugins/gitea/api/connection.go b/plugins/gitea/api/connection.go
new file mode 100644
index 00000000..e3c3fd36
--- /dev/null
+++ b/plugins/gitea/api/connection.go
@@ -0,0 +1,164 @@
+/*
+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 (
+	"context"
+	"fmt"
+	"net/http"
+	"net/url"
+	"time"
+
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/mitchellh/mapstructure"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+)
+
+// @Summary test gitea connection
+// @Description Test gitea Connection
+// @Tags plugins/gitea
+// @Param body body models.TestConnectionRequest true "json body"
+// @Success 200  {object} shared.ApiBody "Success"
+// @Failure 400  {string} errcode.Error "Bad Request"
+// @Failure 500  {string} errcode.Error "Internel Error"
+// @Router /plugins/gitea/test [POST]
+func TestConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
+	var connection models.TestConnectionRequest
+	err := mapstructure.Decode(input.Body, &connection)
+	if err != nil {
+		return nil, err
+	}
+	err = vld.Struct(connection)
+	if err != nil {
+		return nil, err
+	}
+	// test connection
+	apiClient, err := helper.NewApiClient(
+		context.TODO(),
+		connection.Endpoint,
+		nil,
+		3*time.Second,
+		connection.Proxy,
+		basicRes,
+	)
+	if err != nil {
+		return nil, err
+	}
+
+	query := url.Values{}
+	query.Set("token", connection.Token)
+
+	res, err := apiClient.Get("user", query, nil)
+	if err != nil {
+		return nil, err
+	}
+	resBody := &models.ApiUserResponse{}
+	err = helper.UnmarshalResponse(res, resBody)
+	if err != nil {
+		return nil, err
+	}
+
+	if res.StatusCode != http.StatusOK {
+		return nil, fmt.Errorf("unexpected status code: %d", res.StatusCode)
+	}
+	return nil, nil
+}
+
+// @Summary create gitea connection
+// @Description Create gitea connection
+// @Tags plugins/gitea
+// @Param body body models.GithubConnection true "json body"
+// @Success 200  {object} models.GiteaConnection
+// @Failure 400  {string} errcode.Error "Bad Request"
+// @Failure 500  {string} errcode.Error "Internel Error"
+// @Router /plugins/gitea/connections [POST]
+func PostConnections(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
+	connection := &models.GiteaConnection{}
+	err := connectionHelper.Create(connection, input)
+	if err != nil {
+		return nil, err
+	}
+	return &core.ApiResourceOutput{Body: connection, Status: http.StatusOK}, nil
+}
+
+// @Summary patch gitea connection
+// @Description Patch gitea connection
+// @Tags plugins/gitea
+// @Param body body models.GithubConnection true "json body"
+// @Success 200  {object} models.GiteaConnection
+// @Failure 400  {string} errcode.Error "Bad Request"
+// @Failure 500  {string} errcode.Error "Internel Error"
+// @Router /plugins/gitea/connections/{connectionId} [PATCH]
+func PatchConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
+	connection := &models.GiteaConnection{}
+	err := connectionHelper.Patch(connection, input)
+	if err != nil {
+		return nil, err
+	}
+	return &core.ApiResourceOutput{Body: connection, Status: http.StatusOK}, nil
+}
+
+// @Summary delete a gitea connection
+// @Description Delete a gitea connection
+// @Tags plugins/gitea
+// @Success 200  {object} models.GiteaConnection
+// @Failure 400  {string} errcode.Error "Bad Request"
+// @Failure 500  {string} errcode.Error "Internel Error"
+// @Router /plugins/gitea/connections/{connectionId} [DELETE]
+func DeleteConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
+	connection := &models.GiteaConnection{}
+	err := connectionHelper.First(connection, input.Params)
+	if err != nil {
+		return nil, err
+	}
+	err = connectionHelper.Delete(connection)
+	return &core.ApiResourceOutput{Body: connection}, err
+}
+
+// @Summary get all gitea connections
+// @Description Get all gitea connections
+// @Tags plugins/gitea
+// @Success 200  {object} models.GiteaConnection
+// @Failure 400  {string} errcode.Error "Bad Request"
+// @Failure 500  {string} errcode.Error "Internel Error"
+// @Router /plugins/gitea/connections [GET]
+func ListConnections(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
+	var connections []models.GiteaConnection
+	err := connectionHelper.List(&connections)
+	if err != nil {
+		return nil, err
+	}
+
+	return &core.ApiResourceOutput{Body: connections}, nil
+}
+
+// @Summary get gitea connection detail
+// @Description Get gitea connection detail
+// @Tags plugins/gitea
+// @Success 200  {object} models.GiteaConnection
+// @Failure 400  {string} errcode.Error "Bad Request"
+// @Failure 500  {string} errcode.Error "Internel Error"
+// @Router /plugins/gitea/connections/{connectionId} [GET]
+func GetConnection(input *core.ApiResourceInput) (*core.ApiResourceOutput, error) {
+	connection := &models.GiteaConnection{}
+	err := connectionHelper.First(connection, input.Params)
+	return &core.ApiResourceOutput{Body: connection}, err
+}
diff --git a/plugins/gitea/api/init.go b/plugins/gitea/api/init.go
new file mode 100644
index 00000000..6774e148
--- /dev/null
+++ b/plugins/gitea/api/init.go
@@ -0,0 +1,39 @@
+/*
+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 (
+	"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 vld *validator.Validate
+var connectionHelper *helper.ConnectionApiHelper
+var basicRes core.BasicRes
+
+func Init(config *viper.Viper, logger core.Logger, database *gorm.DB) {
+	basicRes = helper.NewDefaultBasicRes(config, logger, database)
+	vld = validator.New()
+	connectionHelper = helper.NewConnectionHelper(
+		basicRes,
+		vld,
+	)
+}
diff --git a/plugins/gitea/api/swagger.go b/plugins/gitea/api/swagger.go
new file mode 100644
index 00000000..b497715e
--- /dev/null
+++ b/plugins/gitea/api/swagger.go
@@ -0,0 +1,49 @@
+/*
+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
+
+// @Summary pipelines plan for gitea
+// @Description pipelines plan for gitea
+// @Tags plugins/gitea
+// @Accept application/json
+// @Param pipeline body GiteaPipelinePlan true "json"
+// @Router /pipelines/gitea/pipeline-plan [post]
+func _() {}
+
+type CodeTransformationRules struct {
+	PrType               string `mapstructure:"prType" json:"prType"`
+	PrComponent          string `mapstructure:"prComponent" json:"prComponent"`
+	PrBodyClosePattern   string `mapstructure:"prBodyClosePattern" json:"prBodyClosePattern"`
+	IssueSeverity        string `mapstructure:"issueSeverity" json:"issueSeverity"`
+	IssuePriority        string `mapstructure:"issuePriority" json:"issuePriority"`
+	IssueComponent       string `mapstructure:"issueComponent" json:"issueComponent"`
+	IssueTypeBug         string `mapstructure:"issueTypeBug" json:"issueTypeBug"`
+	IssueTypeIncident    string `mapstructure:"issueTypeIncident" json:"issueTypeIncident"`
+	IssueTypeRequirement string `mapstructure:"issueTypeRequirement" json:"issueTypeRequirement"`
+}
+type GiteaPipelinePlan [][]struct {
+	Plugin   string   `json:"plugin"`
+	Subtasks []string `json:"subtasks"`
+	Options  struct {
+		ConnectionID   int    `json:"connectionId"`
+		Owner          string `json:"owner"`
+		Repo           string `json:"repo"`
+		Since          string
+		Transformation CodeTransformationRules `json:"transformation"`
+	} `json:"options"`
+}
diff --git a/plugins/gitea/gitea.go b/plugins/gitea/gitea.go
new file mode 100644
index 00000000..c6538590
--- /dev/null
+++ b/plugins/gitea/gitea.go
@@ -0,0 +1,46 @@
+/*
+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 main
+
+import (
+	"github.com/apache/incubator-devlake/plugins/gitea/impl"
+	"github.com/apache/incubator-devlake/runner"
+	"github.com/spf13/cobra"
+)
+
+var PluginEntry impl.Gitea //nolint
+
+// Gitea API Docs: https://docs.gitea.io/zh-cn/api-usage/
+func main() {
+	cmd := &cobra.Command{Use: "gitea"}
+	connectionId := cmd.Flags().Uint64P("connectionId", "c", 0, "gitea connection id")
+	owner := cmd.Flags().StringP("owner", "o", "", "gitea owner")
+	repo := cmd.Flags().StringP("repo", "r", "", "gitea repo")
+	_ = cmd.MarkFlagRequired("connectionId")
+	_ = cmd.MarkFlagRequired("owner")
+	_ = cmd.MarkFlagRequired("repo")
+
+	cmd.Run = func(cmd *cobra.Command, args []string) {
+		runner.DirectRun(cmd, args, PluginEntry, map[string]interface{}{
+			"connectionId": *connectionId,
+			"owner":        *owner,
+			"repo":         *repo,
+		})
+	}
+	runner.RunCmd(cmd)
+}
diff --git a/plugins/gitea/impl/impl.go b/plugins/gitea/impl/impl.go
new file mode 100644
index 00000000..ffc642a8
--- /dev/null
+++ b/plugins/gitea/impl/impl.go
@@ -0,0 +1,173 @@
+/*
+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 impl
+
+import (
+	"fmt"
+
+	"github.com/apache/incubator-devlake/migration"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/api"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/gitea/models/migrationscripts"
+	"github.com/apache/incubator-devlake/plugins/gitea/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/mitchellh/mapstructure"
+	"github.com/spf13/viper"
+	"gorm.io/gorm"
+)
+
+var _ core.PluginMeta = (*Gitea)(nil)
+var _ core.PluginInit = (*Gitea)(nil)
+var _ core.PluginTask = (*Gitea)(nil)
+var _ core.PluginApi = (*Gitea)(nil)
+var _ core.Migratable = (*Gitea)(nil)
+var _ core.CloseablePluginTask = (*Gitea)(nil)
+
+type Gitea string
+
+func (plugin Gitea) Init(config *viper.Viper, logger core.Logger, db *gorm.DB) error {
+	api.Init(config, logger, db)
+	return nil
+}
+
+func (plugin Gitea) GetTablesInfo() []core.Tabler {
+	return []core.Tabler{
+		&models.GiteaConnection{},
+		&models.GiteaAccount{},
+		&models.GiteaCommit{},
+		&models.GiteaCommitStat{},
+		&models.GiteaIssue{},
+		&models.GiteaIssueComment{},
+		&models.GiteaIssueLabel{},
+		&models.GiteaRepo{},
+		&models.GiteaRepoCommit{},
+		&models.GiteaResponse{},
+		&models.GiteaReviewer{},
+	}
+}
+
+func (plugin Gitea) Description() string {
+	return "To collect and enrich data from Gitea"
+}
+
+func (plugin Gitea) SubTaskMetas() []core.SubTaskMeta {
+	return []core.SubTaskMeta{
+		tasks.CollectApiRepoMeta,
+		tasks.ExtractApiRepoMeta,
+		tasks.CollectApiIssuesMeta,
+		tasks.ExtractApiIssuesMeta,
+		tasks.CollectCommitsMeta,
+		tasks.ExtractCommitsMeta,
+		tasks.CollectApiIssueCommentsMeta,
+		tasks.ExtractApiIssueCommentsMeta,
+		tasks.CollectApiCommitStatsMeta,
+		tasks.ExtractApiCommitStatsMeta,
+		tasks.ConvertRepoMeta,
+		tasks.ConvertIssuesMeta,
+		tasks.ConvertCommitsMeta,
+		tasks.ConvertIssueLabelsMeta,
+		tasks.ConvertAccountsMeta,
+		tasks.ConvertIssueCommentsMeta,
+	}
+}
+
+func (plugin Gitea) PrepareTaskData(taskCtx core.TaskContext, options map[string]interface{}) (interface{}, error) {
+	var op tasks.GiteaOptions
+	var err error
+	err = mapstructure.Decode(options, &op)
+	if err != nil {
+		return nil, err
+	}
+
+	if op.Owner == "" {
+		return nil, fmt.Errorf("owner is required for Gitea execution")
+	}
+
+	if op.Repo == "" {
+		return nil, fmt.Errorf("repo is required for Gitea execution")
+	}
+
+	if op.ConnectionId == 0 {
+		return nil, fmt.Errorf("connectionId is invalid")
+	}
+
+	connection := &models.GiteaConnection{}
+	connectionHelper := helper.NewConnectionHelper(
+		taskCtx,
+		nil,
+	)
+
+	if err != nil {
+		return nil, err
+	}
+
+	err = connectionHelper.FirstById(connection, op.ConnectionId)
+
+	if err != nil {
+		return nil, err
+	}
+	apiClient, err := tasks.NewGiteaApiClient(taskCtx, connection)
+
+	if err != nil {
+		return nil, err
+	}
+
+	return &tasks.GiteaTaskData{
+		Options:   &op,
+		ApiClient: apiClient,
+	}, nil
+}
+
+func (plugin Gitea) RootPkgPath() string {
+	return "github.com/apache/incubator-devlake/plugins/gitea"
+}
+
+func (plugin Gitea) MigrationScripts() []migration.Script {
+	return migrationscripts.All()
+}
+
+func (plugin Gitea) ApiResources() map[string]map[string]core.ApiResourceHandler {
+	return map[string]map[string]core.ApiResourceHandler{
+		"test": {
+			"POST": api.TestConnection,
+		},
+		"connections": {
+			"POST": api.PostConnections,
+			"GET":  api.ListConnections,
+		},
+		"connections/:connectionId": {
+			"GET":    api.GetConnection,
+			"PATCH":  api.PatchConnection,
+			"DELETE": api.DeleteConnection,
+		},
+	}
+}
+
+func (plugin Gitea) MakePipelinePlan(connectionId uint64, scope []*core.BlueprintScopeV100) (core.PipelinePlan, error) {
+	return api.MakePipelinePlan(plugin.SubTaskMetas(), connectionId, scope)
+}
+
+func (plugin Gitea) Close(taskCtx core.TaskContext) error {
+	data, ok := taskCtx.GetData().(*tasks.GiteaTaskData)
+	if !ok {
+		return fmt.Errorf("GetData failed when try to close %+v", taskCtx)
+	}
+	data.ApiClient.Release()
+	return nil
+}
diff --git a/plugins/gitea/models/account.go b/plugins/gitea/models/account.go
new file mode 100644
index 00000000..f3ffec73
--- /dev/null
+++ b/plugins/gitea/models/account.go
@@ -0,0 +1,42 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import "github.com/apache/incubator-devlake/models/common"
+
+type GiteaAccount struct {
+	ConnectionId      uint64 `gorm:"primaryKey"`
+	Id                int    `json:"id" gorm:"primaryKey;autoIncrement:false"`
+	Login             string `json:"login" gorm:"type:varchar(255)"`
+	FullName          string `json:"full_name" gorm:"type:varchar(255)"`
+	Email             string `gorm:"type:varchar(255)"`
+	Description       string `gorm:"type:varchar(255)"`
+	Language          string `gorm:"type:varchar(255)"`
+	Location          string `gorm:"type:varchar(255)"`
+	Website           string `gorm:"type:varchar(255)"`
+	AvatarUrl         string `gorm:"type:varchar(255)"`
+	IsAdmin           bool
+	FollowersCount    int
+	FollowingCount    int
+	StarredReposCount int
+	common.NoPKModel
+}
+
+func (GiteaAccount) TableName() string {
+	return "_tool_gitea_accounts"
+}
diff --git a/plugins/gitea/models/commit.go b/plugins/gitea/models/commit.go
new file mode 100644
index 00000000..9fc75200
--- /dev/null
+++ b/plugins/gitea/models/commit.go
@@ -0,0 +1,47 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaCommit struct {
+	Sha            string `gorm:"primaryKey;type:varchar(40)"`
+	CommentsUrl    string `gorm:"type:varchar(255)"`
+	Message        string
+	AuthorId       int
+	AuthorName     string `gorm:"type:varchar(255)"`
+	AuthorEmail    string `gorm:"type:varchar(255)"`
+	AuthoredDate   time.Time
+	CommitterId    int
+	CommitterName  string `gorm:"type:varchar(255)"`
+	CommitterEmail string `gorm:"type:varchar(255)"`
+	CommittedDate  time.Time
+	WebUrl         string `gorm:"type:varchar(255)"`
+	Additions      int    `gorm:"comment:Added lines of code"`
+	Deletions      int    `gorm:"comment:Deleted lines of code"`
+	Total          int    `gorm:"comment:Sum of added/deleted lines of code"`
+	common.NoPKModel
+}
+
+func (GiteaCommit) TableName() string {
+	return "_tool_gitea_commits"
+}
diff --git a/plugins/gitea/models/commit_stats.go b/plugins/gitea/models/commit_stats.go
new file mode 100644
index 00000000..c4ed80ff
--- /dev/null
+++ b/plugins/gitea/models/commit_stats.go
@@ -0,0 +1,37 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaCommitStat struct {
+	ConnectionId  uint64    `gorm:"primaryKey"`
+	Sha           string    `gorm:"primaryKey;type:varchar(40)"`
+	Additions     int       `gorm:"comment:Added lines of code"`
+	Deletions     int       `gorm:"comment:Deleted lines of code"`
+	CommittedDate time.Time `gorm:"index"`
+	common.NoPKModel
+}
+
+func (GiteaCommitStat) TableName() string {
+	return "_tool_gitea_commit_stats"
+}
diff --git a/plugins/gitea/models/connection.go b/plugins/gitea/models/connection.go
new file mode 100644
index 00000000..5ab3d4f2
--- /dev/null
+++ b/plugins/gitea/models/connection.go
@@ -0,0 +1,46 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import "github.com/apache/incubator-devlake/plugins/helper"
+
+type GiteaConnection struct {
+	helper.RestConnection `mapstructure:",squash"`
+	helper.AccessToken    `mapstructure:",squash"`
+}
+
+type GiteaResponse struct {
+	Name string `json:"name"`
+	ID   int    `json:"id"`
+	GiteaConnection
+}
+
+type ApiUserResponse struct {
+	Id   int
+	Name string `json:"name"`
+}
+
+type TestConnectionRequest struct {
+	Endpoint           string `json:"endpoint" validate:"required"`
+	Proxy              string `json:"proxy"`
+	helper.AccessToken `mapstructure:",squash"`
+}
+
+func (GiteaConnection) TableName() string {
+	return "_tool_gitea_connections"
+}
diff --git a/plugins/gitea/models/issue.go b/plugins/gitea/models/issue.go
new file mode 100644
index 00000000..2672a18d
--- /dev/null
+++ b/plugins/gitea/models/issue.go
@@ -0,0 +1,52 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaIssue struct {
+	ConnectionId    uint64 `gorm:"primaryKey"`
+	GiteaId         int    `gorm:"primaryKey"`
+	RepoId          int    `gorm:"index"`
+	Number          int    `gorm:"index;comment:Used in API requests ex. api/repo/1/issue/<THIS_NUMBER>"`
+	State           string `gorm:"type:varchar(255)"`
+	Title           string
+	Body            string
+	AuthorId        int
+	AuthorName      string `gorm:"type:varchar(255)"`
+	AssigneeId      int
+	AssigneeName    string `gorm:"type:varchar(255)"`
+	LeadTimeMinutes uint
+	Url             string `gorm:"type:varchar(255)"`
+	HtmlUrl         string `gorm:"type:varchar(255)"`
+	ClosedAt        *time.Time
+	GiteaCreatedAt  time.Time
+	GiteaUpdatedAt  time.Time `gorm:"index"`
+	DueDate         *time.Time
+	Comments        int
+	Ref             string `gorm:"type:varchar(255)"`
+	common.NoPKModel
+}
+
+func (GiteaIssue) TableName() string {
+	return "_tool_gitea_issues"
+}
diff --git a/plugins/gitea/models/issue_comment.go b/plugins/gitea/models/issue_comment.go
new file mode 100644
index 00000000..47017caf
--- /dev/null
+++ b/plugins/gitea/models/issue_comment.go
@@ -0,0 +1,43 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaIssueComment struct {
+	ConnectionId   uint64 `gorm:"primaryKey"`
+	GiteaId        int    `gorm:"primaryKey"`
+	IssueId        int    `gorm:"index;comment:References the Issue"`
+	Body           string
+	AuthorName     string `gorm:"type:varchar(255)"`
+	AuthorId       int
+	HtmlUrl        string `gorm:"type:varchar(255)"`
+	IssueUrl       string `gorm:"type:varchar(255)"`
+	PullRequestUrl string `gorm:"type:varchar(255)"`
+	GiteaCreatedAt time.Time
+	GiteaUpdatedAt time.Time `gorm:"index"`
+	common.NoPKModel
+}
+
+func (GiteaIssueComment) TableName() string {
+	return "_tool_gitea_issue_comments"
+}
diff --git a/plugins/gitea/models/issue_label.go b/plugins/gitea/models/issue_label.go
new file mode 100644
index 00000000..1ef91402
--- /dev/null
+++ b/plugins/gitea/models/issue_label.go
@@ -0,0 +1,39 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+// Please note that Issue Labels can also apply to Pull Requests.
+// Pull Requests are considered Issues in Gitea.
+
+type GiteaIssueLabel struct {
+	ConnectionId uint64 `gorm:"primaryKey"`
+	IssueId      int    `gorm:"primaryKey;autoIncrement:false"`
+	LabelName    string `gorm:"primaryKey;type:varchar(255)"`
+	LabelColor   string `gorm:"type:varchar(255)"`
+	LabelUrl     string `gorm:"type:varchar(255)"`
+	Description  string `gorm:"type:varchar(255)"`
+	common.NoPKModel
+}
+
+func (GiteaIssueLabel) TableName() string {
+	return "_tool_gitea_issue_labels"
+}
diff --git a/plugins/gitea/models/migrationscripts/20220830_add_init_tables.go b/plugins/gitea/models/migrationscripts/20220830_add_init_tables.go
new file mode 100644
index 00000000..1ce3d581
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/20220830_add_init_tables.go
@@ -0,0 +1,88 @@
+/*
+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"
+	"fmt"
+	"github.com/apache/incubator-devlake/plugins/gitea/models/migrationscripts/archived"
+
+	"gorm.io/gorm"
+)
+
+type addInitTables struct{}
+
+func (*addInitTables) Up(ctx context.Context, db *gorm.DB) error {
+	rawTableList := []string{
+		"_raw_gitea_api_commit",
+		"_raw_gitea_api_issues",
+		"_raw_gitea_api_repo",
+		"_raw_gitea_api_comments",
+		"_raw_gitea_api_commits",
+		"_raw_gitea_issue_comments",
+	}
+	for _, v := range rawTableList {
+		err := db.Exec(fmt.Sprintf("DROP TABLE IF EXISTS %s CASCADE", v)).Error
+		if err != nil {
+			return err
+		}
+	}
+
+	err := db.Migrator().DropTable(
+		&archived.GiteaRepo{},
+		&archived.GiteaCommit{},
+		&archived.GiteaRepoCommit{},
+		&archived.GiteaIssue{},
+		&archived.GiteaIssueComment{},
+		&archived.GiteaCommitStat{},
+		&archived.GiteaIssueLabel{},
+		&archived.GiteaReviewer{},
+		&archived.GiteaConnection{},
+	)
+
+	if err != nil {
+		return err
+	}
+
+	err = db.Migrator().AutoMigrate(
+		&archived.GiteaRepo{},
+		&archived.GiteaCommit{},
+		&archived.GiteaRepoCommit{},
+		&archived.GiteaAccount{},
+		&archived.GiteaIssue{},
+		&archived.GiteaIssueComment{},
+		&archived.GiteaCommitStat{},
+		&archived.GiteaIssueLabel{},
+		&archived.GiteaReviewer{},
+		&archived.GiteaConnection{},
+	)
+
+	if err != nil {
+		return err
+	}
+
+	return nil
+}
+
+func (*addInitTables) Version() uint64 {
+	return 20220830163407
+}
+
+func (*addInitTables) Name() string {
+	return "Gitea init schemas"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/account.go b/plugins/gitea/models/migrationscripts/archived/account.go
new file mode 100644
index 00000000..5d239f55
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/account.go
@@ -0,0 +1,37 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import "github.com/apache/incubator-devlake/models/common"
+
+type GiteaAccount struct {
+	ConnectionId uint64 `gorm:"primaryKey"`
+	Id           int    `json:"id" gorm:"primaryKey;autoIncrement:false"`
+	Login        string `json:"login" gorm:"type:varchar(255)"`
+	FullName     string `json:"full_name" gorm:"type:varchar(255)"`
+	Email        string `gorm:"type:varchar(255)"`
+	Description  string `gorm:"type:varchar(255)"`
+	Language     string `gorm:"type:varchar(255)"`
+	Location     string `gorm:"type:varchar(255)"`
+	Website      string `gorm:"type:varchar(255)"`
+	common.NoPKModel
+}
+
+func (GiteaAccount) TableName() string {
+	return "_tool_gitea_accounts"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/commit.go b/plugins/gitea/models/migrationscripts/archived/commit.go
new file mode 100644
index 00000000..8f0b830a
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/commit.go
@@ -0,0 +1,47 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaCommit struct {
+	Sha            string `gorm:"primaryKey;type:varchar(40)"`
+	CommentsUrl    string `gorm:"type:varchar(255)"`
+	Message        string
+	AuthorId       int
+	AuthorName     string `gorm:"type:varchar(255)"`
+	AuthorEmail    string `gorm:"type:varchar(255)"`
+	AuthoredDate   time.Time
+	CommitterId    int
+	CommitterName  string `gorm:"type:varchar(255)"`
+	CommitterEmail string `gorm:"type:varchar(255)"`
+	CommittedDate  time.Time
+	WebUrl         string `gorm:"type:varchar(255)"`
+	Additions      int    `gorm:"comment:Added lines of code"`
+	Deletions      int    `gorm:"comment:Deleted lines of code"`
+	Total          int    `gorm:"comment:Sum of added/deleted lines of code"`
+	common.NoPKModel
+}
+
+func (GiteaCommit) TableName() string {
+	return "_tool_gitea_commits"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/commit_stats.go b/plugins/gitea/models/migrationscripts/archived/commit_stats.go
new file mode 100644
index 00000000..fb05fd65
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/commit_stats.go
@@ -0,0 +1,37 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaCommitStat struct {
+	ConnectionId  uint64    `gorm:"primaryKey"`
+	Sha           string    `gorm:"primaryKey;type:varchar(40)"`
+	Additions     int       `gorm:"comment:Added lines of code"`
+	Deletions     int       `gorm:"comment:Deleted lines of code"`
+	CommittedDate time.Time `gorm:"index"`
+	common.NoPKModel
+}
+
+func (GiteaCommitStat) TableName() string {
+	return "_tool_gitea_commit_stats"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/connection.go b/plugins/gitea/models/migrationscripts/archived/connection.go
new file mode 100644
index 00000000..17b65ead
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/connection.go
@@ -0,0 +1,58 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import "github.com/apache/incubator-devlake/plugins/helper"
+
+type GiteaConnection struct {
+	helper.RestConnection `mapstructure:",squash"`
+	helper.AccessToken    `mapstructure:",squash"`
+}
+
+type GiteaResponse struct {
+	Name string `json:"name"`
+	ID   int    `json:"id"`
+	GiteaConnection
+}
+
+type ApiUserResponse struct {
+	Id   int
+	Name string `json:"name"`
+}
+
+type TestConnectionRequest struct {
+	Endpoint           string `json:"endpoint" validate:"required"`
+	Proxy              string `json:"proxy"`
+	helper.AccessToken `mapstructure:",squash"`
+}
+
+type TransformationRules struct {
+	PrType               string `mapstructure:"prType" env:"GITEE_PR_TYPE" json:"prType"`
+	PrComponent          string `mapstructure:"prComponent" env:"GITEE_PR_COMPONENT" json:"prComponent"`
+	PrBodyClosePattern   string `mapstructure:"prBodyClosePattern" json:"prBodyClosePattern"`
+	IssueSeverity        string `mapstructure:"issueSeverity" env:"GITEE_ISSUE_SEVERITY" json:"issueSeverity"`
+	IssuePriority        string `mapstructure:"issuePriority" env:"GITEE_ISSUE_PRIORITY" json:"issuePriority"`
+	IssueComponent       string `mapstructure:"issueComponent" env:"GITEE_ISSUE_COMPONENT" json:"issueComponent"`
+	IssueTypeBug         string `mapstructure:"issueTypeBug" env:"GITEE_ISSUE_TYPE_BUG" json:"issueTypeBug"`
+	IssueTypeIncident    string `mapstructure:"issueTypeIncident" env:"GITEE_ISSUE_TYPE_INCIDENT" json:"issueTypeIncident"`
+	IssueTypeRequirement string `mapstructure:"issueTypeRequirement" env:"GITEE_ISSUE_TYPE_REQUIREMENT" json:"issueTypeRequirement"`
+}
+
+func (GiteaConnection) TableName() string {
+	return "_tool_gitea_connections"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/issue.go b/plugins/gitea/models/migrationscripts/archived/issue.go
new file mode 100644
index 00000000..0e5d17d6
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/issue.go
@@ -0,0 +1,52 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaIssue struct {
+	ConnectionId    uint64 `gorm:"primaryKey"`
+	GiteaId         int    `gorm:"primaryKey"`
+	RepoId          int    `gorm:"index"`
+	Number          int    `gorm:"index;comment:Used in API requests ex. api/repo/1/issue/<THIS_NUMBER>"`
+	State           string `gorm:"type:varchar(255)"`
+	Title           string
+	Body            string
+	AuthorId        int
+	AuthorName      string `gorm:"type:varchar(255)"`
+	AssigneeId      int
+	AssigneeName    string `gorm:"type:varchar(255)"`
+	LeadTimeMinutes uint
+	Url             string `gorm:"type:varchar(255)"`
+	HtmlUrl         string `gorm:"type:varchar(255)"`
+	ClosedAt        *time.Time
+	GiteaCreatedAt  time.Time
+	GiteaUpdatedAt  time.Time `gorm:"index"`
+	DueDate         *time.Time
+	Comments        int
+	Ref             string `gorm:"type:varchar(255)"`
+	common.NoPKModel
+}
+
+func (GiteaIssue) TableName() string {
+	return "_tool_gitea_issues"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/issue_comment.go b/plugins/gitea/models/migrationscripts/archived/issue_comment.go
new file mode 100644
index 00000000..80e95b41
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/issue_comment.go
@@ -0,0 +1,43 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaIssueComment struct {
+	ConnectionId   uint64 `gorm:"primaryKey"`
+	GiteaId        int    `gorm:"primaryKey"`
+	IssueId        int    `gorm:"index;comment:References the Issue"`
+	Body           string
+	AuthorName     string `gorm:"type:varchar(255)"`
+	AuthorId       int
+	HtmlUrl        string `gorm:"type:varchar(255)"`
+	IssueUrl       string `gorm:"type:varchar(255)"`
+	PullRequestUrl string `gorm:"type:varchar(255)"`
+	GiteaCreatedAt time.Time
+	GiteaUpdatedAt time.Time `gorm:"index"`
+	common.NoPKModel
+}
+
+func (GiteaIssueComment) TableName() string {
+	return "_tool_gitea_issue_comments"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/issue_label.go b/plugins/gitea/models/migrationscripts/archived/issue_label.go
new file mode 100644
index 00000000..c9252eaf
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/issue_label.go
@@ -0,0 +1,39 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+// Please note that Issue Labels can also apply to Pull Requests.
+// Pull Requests are considered Issues in Gitea.
+
+type GiteaIssueLabel struct {
+	ConnectionId uint64 `gorm:"primaryKey"`
+	IssueId      int    `gorm:"primaryKey;autoIncrement:false"`
+	LabelName    string `gorm:"primaryKey;type:varchar(255)"`
+	LabelColor   string `gorm:"type:varchar(255)"`
+	LabelUrl     string `gorm:"type:varchar(255)"`
+	Description  string `gorm:"type:varchar(255)"`
+	common.NoPKModel
+}
+
+func (GiteaIssueLabel) TableName() string {
+	return "_tool_gitea_issue_labels"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/repo.go b/plugins/gitea/models/migrationscripts/archived/repo.go
new file mode 100644
index 00000000..584a5286
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/repo.go
@@ -0,0 +1,45 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaRepo struct {
+	ConnectionId  uint64 `gorm:"primaryKey"`
+	GiteaId       int    `gorm:"primaryKey"`
+	Name          string `gorm:"type:varchar(255)"`
+	FullName      string `gorm:"type:varchar(255)"`
+	HTMLUrl       string `gorm:"type:varchar(255)"`
+	Description   string
+	OwnerId       int        `json:"ownerId"`
+	OwnerName     string     `json:"ownerName" gorm:"type:varchar(255)"`
+	Language      string     `json:"language" gorm:"type:varchar(255)"`
+	ParentGiteaId int        `json:"parentId"`
+	ParentHTMLUrl string     `json:"parentHtmlUrl"`
+	CreatedDate   time.Time  `json:"createdDate"`
+	UpdatedDate   *time.Time `json:"updatedDate"`
+	common.NoPKModel
+}
+
+func (GiteaRepo) TableName() string {
+	return "_tool_gitea_repos"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/repo_commit.go b/plugins/gitea/models/migrationscripts/archived/repo_commit.go
new file mode 100644
index 00000000..f4e0b8d8
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/repo_commit.go
@@ -0,0 +1,33 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaRepoCommit struct {
+	ConnectionId uint64 `gorm:"primaryKey"`
+	RepoId       int    `gorm:"primaryKey"`
+	CommitSha    string `gorm:"primaryKey;type:varchar(40)"`
+	common.NoPKModel
+}
+
+func (GiteaRepoCommit) TableName() string {
+	return "_tool_gitea_repo_commits"
+}
diff --git a/plugins/gitea/models/migrationscripts/archived/reviewer.go b/plugins/gitea/models/migrationscripts/archived/reviewer.go
new file mode 100644
index 00000000..c6f6533b
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/archived/reviewer.go
@@ -0,0 +1,35 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package archived
+
+import (
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaReviewer struct {
+	ConnectionId  uint64 `gorm:"primaryKey"`
+	GiteaId       int    `gorm:"primaryKey"`
+	GiteaLogin    string `gorm:"type:varchar(255)"`
+	PullRequestId int    `gorm:"primaryKey"`
+
+	common.NoPKModel
+}
+
+func (GiteaReviewer) TableName() string {
+	return "_tool_gitea_reviewers"
+}
diff --git a/plugins/gitea/models/migrationscripts/register.go b/plugins/gitea/models/migrationscripts/register.go
new file mode 100644
index 00000000..c1365f7d
--- /dev/null
+++ b/plugins/gitea/models/migrationscripts/register.go
@@ -0,0 +1,29 @@
+/*
+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 (
+	"github.com/apache/incubator-devlake/migration"
+)
+
+// All return all the migration scripts
+func All() []migration.Script {
+	return []migration.Script{
+		new(addInitTables),
+	}
+}
diff --git a/plugins/gitea/models/repo.go b/plugins/gitea/models/repo.go
new file mode 100644
index 00000000..e4835b74
--- /dev/null
+++ b/plugins/gitea/models/repo.go
@@ -0,0 +1,45 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaRepo struct {
+	ConnectionId  uint64 `gorm:"primaryKey"`
+	GiteaId       int    `gorm:"primaryKey"`
+	Name          string `gorm:"type:varchar(255)"`
+	FullName      string `gorm:"type:varchar(255)"`
+	HTMLUrl       string `gorm:"type:varchar(255)"`
+	Description   string
+	OwnerId       int        `json:"ownerId"`
+	OwnerName     string     `json:"ownerName" gorm:"type:varchar(255)"`
+	Language      string     `json:"language" gorm:"type:varchar(255)"`
+	ParentGiteaId int        `json:"parentId"`
+	ParentHTMLUrl string     `json:"parentHtmlUrl"`
+	CreatedDate   time.Time  `json:"createdDate"`
+	UpdatedDate   *time.Time `json:"updatedDate"`
+	common.NoPKModel
+}
+
+func (GiteaRepo) TableName() string {
+	return "_tool_gitea_repos"
+}
diff --git a/plugins/gitea/models/repo_commit.go b/plugins/gitea/models/repo_commit.go
new file mode 100644
index 00000000..73da2652
--- /dev/null
+++ b/plugins/gitea/models/repo_commit.go
@@ -0,0 +1,33 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaRepoCommit struct {
+	ConnectionId uint64 `gorm:"primaryKey"`
+	RepoId       int    `gorm:"primaryKey"`
+	CommitSha    string `gorm:"primaryKey;type:varchar(40)"`
+	common.NoPKModel
+}
+
+func (GiteaRepoCommit) TableName() string {
+	return "_tool_gitea_repo_commits"
+}
diff --git a/plugins/gitea/models/reviewer.go b/plugins/gitea/models/reviewer.go
new file mode 100644
index 00000000..f0093967
--- /dev/null
+++ b/plugins/gitea/models/reviewer.go
@@ -0,0 +1,35 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GiteaReviewer struct {
+	ConnectionId  uint64 `gorm:"primaryKey"`
+	GiteaId       int    `gorm:"primaryKey"`
+	GiteaLogin    string `gorm:"type:varchar(255)"`
+	PullRequestId int    `gorm:"primaryKey"`
+
+	common.NoPKModel
+}
+
+func (GiteaReviewer) TableName() string {
+	return "_tool_gitea_reviewers"
+}
diff --git a/plugins/gitea/tasks/account_convertor.go b/plugins/gitea/tasks/account_convertor.go
new file mode 100644
index 00000000..abc06122
--- /dev/null
+++ b/plugins/gitea/tasks/account_convertor.go
@@ -0,0 +1,74 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"github.com/apache/incubator-devlake/models/domainlayer/crossdomain"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertAccountsMeta = core.SubTaskMeta{
+	Name:             "convertAccounts",
+	EntryPoint:       ConvertAccounts,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table gitea_accounts into  domain layer table accounts",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CROSS},
+}
+
+func ConvertAccounts(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_TABLE)
+
+	cursor, err := db.Cursor(dal.From(&models.GiteaAccount{}))
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	accountIdGen := didgen.NewDomainIdGenerator(&models.GiteaAccount{})
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		InputRowType:       reflect.TypeOf(models.GiteaAccount{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			giteaAccount := inputRow.(*models.GiteaAccount)
+			domainUser := &crossdomain.Account{
+				DomainEntity: domainlayer.DomainEntity{Id: accountIdGen.Generate(data.Options.ConnectionId, giteaAccount.Id)},
+				UserName:     giteaAccount.Login,
+				AvatarUrl:    giteaAccount.AvatarUrl,
+			}
+			return []interface{}{
+				domainUser,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/gitea/tasks/api_client.go b/plugins/gitea/tasks/api_client.go
new file mode 100644
index 00000000..09152bc6
--- /dev/null
+++ b/plugins/gitea/tasks/api_client.go
@@ -0,0 +1,51 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"net/http"
+	"time"
+)
+
+func NewGiteaApiClient(taskCtx core.TaskContext, connection *models.GiteaConnection) (*helper.ApiAsyncClient, error) {
+
+	apiClient, err := helper.NewApiClient(taskCtx.GetContext(), connection.Endpoint, nil, 0, connection.Proxy, taskCtx)
+	if err != nil {
+		return nil, err
+	}
+
+	apiClient.SetBeforeFunction(func(req *http.Request) error {
+		req.Header.Add("Authorization", fmt.Sprintf("token %s", connection.Token))
+		return nil
+	})
+
+	asyncApiClient, err := helper.CreateAsyncApiClient(
+		taskCtx,
+		apiClient,
+		nil,
+	)
+	apiClient.SetTimeout(30 * time.Second)
+	if err != nil {
+		return nil, err
+	}
+	return asyncApiClient, nil
+}
diff --git a/plugins/gitea/tasks/commit_collector.go b/plugins/gitea/tasks/commit_collector.go
new file mode 100644
index 00000000..d15d9d72
--- /dev/null
+++ b/plugins/gitea/tasks/commit_collector.go
@@ -0,0 +1,89 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"net/url"
+	"strconv"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+const RAW_COMMIT_TABLE = "gitea_api_commit"
+
+var CollectCommitsMeta = core.SubTaskMeta{
+	Name:             "collectApiCommits",
+	EntryPoint:       CollectApiCommits,
+	EnabledByDefault: true,
+	Description:      "Collect commit data from gitea api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE, core.DOMAIN_TYPE_CROSS},
+}
+
+func CollectApiCommits(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_TABLE)
+	since := data.Since
+	incremental := false
+	if since == nil {
+		latestUpdated := &models.GiteaCommit{}
+		err := db.All(
+			&latestUpdated,
+			dal.Join("left join _tool_gitea_repo_commits on _tool_gitea_commits.sha = _tool_gitea_repo_commits.commit_sha"),
+			dal.Join("left join _tool_gitea_repos on _tool_gitea_repo_commits.repo_id = _tool_gitea_repos.gitea_id"),
+			dal.Where("_tool_gitea_repo_commits.repo_id = ? AND _tool_gitea_repo_commits.connection_id = ?", data.Repo.GiteaId, data.Repo.ConnectionId),
+			dal.Orderby("committed_date DESC"),
+			dal.Limit(1),
+		)
+
+		if err != nil {
+			return fmt.Errorf("failed to get latest gitea commit record: %w", err)
+		}
+		if latestUpdated.Sha != "" {
+			since = &latestUpdated.CommittedDate
+			incremental = true
+		}
+	}
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		ApiClient:          data.ApiClient,
+		PageSize:           50,
+		Incremental:        incremental,
+		UrlTemplate:        "repos/{{ .Params.Owner }}/{{ .Params.Repo }}/commits",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			// page number of results to return (1-based)
+			query.Set("page", strconv.Itoa(reqData.Pager.Page))
+			// page size of results (ignored if used with 'path')
+			query.Set("limit", strconv.Itoa(reqData.Pager.Size))
+			return query, nil
+		},
+		ResponseParser: GetRawMessageFromResponse,
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
diff --git a/plugins/gitea/tasks/commit_convertor.go b/plugins/gitea/tasks/commit_convertor.go
new file mode 100644
index 00000000..860d71be
--- /dev/null
+++ b/plugins/gitea/tasks/commit_convertor.go
@@ -0,0 +1,103 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertCommitsMeta = core.SubTaskMeta{
+	Name:             "convertApiCommits",
+	EntryPoint:       ConvertCommits,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table gitea_commits into  domain layer table commits",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE, core.DOMAIN_TYPE_CROSS},
+}
+
+func ConvertCommits(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_TABLE)
+	db := taskCtx.GetDal()
+	repoId := data.Repo.GiteaId
+
+	// select all commits belongs to the project
+	cursor, err := db.Cursor(
+		dal.Select("gc.*"),
+		dal.From("_tool_gitea_commits gc"),
+		dal.Join(`left join _tool_gitea_repo_commits grc on (
+			grc.commit_sha = gc.sha
+		)`),
+		dal.Where("grc.repo_id = ? AND grc.connection_id = ?", repoId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	accountIdGen := didgen.NewDomainIdGenerator(&models.GiteaAccount{})
+	repoDidGen := didgen.NewDomainIdGenerator(&models.GiteaRepo{})
+	domainRepoId := repoDidGen.Generate(data.Options.ConnectionId, repoId)
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		InputRowType:       reflect.TypeOf(models.GiteaCommit{}),
+		Input:              cursor,
+
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			giteaCommit := inputRow.(*models.GiteaCommit)
+
+			// convert commit
+			commit := &code.Commit{}
+			commit.Sha = giteaCommit.Sha
+			commit.Message = giteaCommit.Message
+			commit.Additions = giteaCommit.Additions
+			commit.Deletions = giteaCommit.Deletions
+			commit.AuthorId = accountIdGen.Generate(data.Options.ConnectionId, giteaCommit.AuthorId)
+			commit.AuthorName = giteaCommit.AuthorName
+			commit.AuthorEmail = giteaCommit.AuthorEmail
+			commit.AuthoredDate = giteaCommit.AuthoredDate
+			commit.CommitterName = giteaCommit.CommitterName
+			commit.CommitterEmail = giteaCommit.CommitterEmail
+			commit.CommittedDate = giteaCommit.CommittedDate
+			commit.CommitterId = accountIdGen.Generate(data.Options.ConnectionId, giteaCommit.CommitterId)
+
+			// convert repo / commits relationship
+			repoCommit := &code.RepoCommit{
+				RepoId:    domainRepoId,
+				CommitSha: giteaCommit.Sha,
+			}
+
+			return []interface{}{
+				commit,
+				repoCommit,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/gitea/tasks/commit_extractor.go b/plugins/gitea/tasks/commit_extractor.go
new file mode 100644
index 00000000..ce431b21
--- /dev/null
+++ b/plugins/gitea/tasks/commit_extractor.go
@@ -0,0 +1,126 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractCommitsMeta = core.SubTaskMeta{
+	Name:             "extractApiCommits",
+	EntryPoint:       ExtractApiCommits,
+	EnabledByDefault: true,
+	Description:      "Extract raw commit data into tool layer table GiteaCommit,GiteaAccount and GiteaRepoCommit",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE, core.DOMAIN_TYPE_CROSS},
+}
+
+type GiteaCommit struct {
+	Author struct {
+		Date  helper.Iso8601Time `json:"date"`
+		Email string             `json:"email"`
+		Name  string             `json:"name"`
+	}
+	Committer struct {
+		Date  helper.Iso8601Time `json:"date"`
+		Email string             `json:"email"`
+		Name  string             `json:"name"`
+	}
+	Message string `json:"message"`
+}
+
+type GiteaApiCommitResponse struct {
+	Author    *models.GiteaAccount `json:"author"`
+	Commit    GiteaCommit          `json:"commit"`
+	Committer *models.GiteaAccount `json:"committer"`
+	HtmlUrl   string               `json:"html_url"`
+	Sha       string               `json:"sha"`
+	Url       string               `json:"url"`
+}
+
+func ExtractApiCommits(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_TABLE)
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			results := make([]interface{}, 0, 4)
+
+			commit := &GiteaApiCommitResponse{}
+
+			err := json.Unmarshal(row.Data, commit)
+
+			if err != nil {
+				return nil, err
+			}
+
+			if commit.Sha == "" {
+				return nil, nil
+			}
+
+			giteaCommit, err := ConvertCommit(commit)
+
+			if err != nil {
+				return nil, err
+			}
+
+			if commit.Author != nil {
+				giteaCommit.AuthorId = commit.Author.Id
+				results = append(results, commit.Author)
+			}
+			if commit.Committer != nil {
+				giteaCommit.CommitterId = commit.Committer.Id
+				results = append(results, commit.Committer)
+			}
+
+			giteaRepoCommit := &models.GiteaRepoCommit{
+				ConnectionId: data.Options.ConnectionId,
+				RepoId:       data.Repo.GiteaId,
+				CommitSha:    commit.Sha,
+			}
+			results = append(results, giteaCommit)
+			results = append(results, giteaRepoCommit)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
+
+// ConvertCommit Convert the API response to our DB model instance
+func ConvertCommit(commit *GiteaApiCommitResponse) (*models.GiteaCommit, error) {
+	giteaCommit := &models.GiteaCommit{
+		Sha:            commit.Sha,
+		Message:        commit.Commit.Message,
+		AuthorName:     commit.Commit.Author.Name,
+		AuthorEmail:    commit.Commit.Author.Email,
+		AuthoredDate:   commit.Commit.Author.Date.ToTime(),
+		CommitterName:  commit.Commit.Author.Name,
+		CommitterEmail: commit.Commit.Author.Email,
+		CommittedDate:  commit.Commit.Author.Date.ToTime(),
+		WebUrl:         commit.Url,
+	}
+	return giteaCommit, nil
+}
diff --git a/plugins/gitea/tasks/commit_stats_collector.go b/plugins/gitea/tasks/commit_stats_collector.go
new file mode 100644
index 00000000..11455ef9
--- /dev/null
+++ b/plugins/gitea/tasks/commit_stats_collector.go
@@ -0,0 +1,110 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"net/url"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/plugins/helper"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+)
+
+const RAW_COMMIT_STATS_TABLE = "gitea_api_commit_stats"
+
+var CollectApiCommitStatsMeta = core.SubTaskMeta{
+	Name:             "collectApiCommitStats",
+	EntryPoint:       CollectApiCommitStats,
+	EnabledByDefault: false,
+	Description:      "Collect commitStats data from Gitea api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE},
+}
+
+func CollectApiCommitStats(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_STATS_TABLE)
+
+	var latestUpdated models.GiteaCommitStat
+
+	err := db.First(
+		&latestUpdated,
+		dal.Join("left join _tool_gitea_repo_commits on _tool_gitea_commit_stats.sha = _tool_gitea_repo_commits.commit_sha"),
+		dal.Where("_tool_gitea_repo_commits.repo_id = ? and _tool_gitea_repo_commits.connection_id = ?", data.Repo.GiteaId, data.Repo.ConnectionId),
+		dal.Orderby("committed_date DESC"),
+		dal.Limit(1),
+	)
+
+	if err != nil {
+		return fmt.Errorf("failed to get latest gitea commit record: %w", err)
+	}
+
+	cursor, err := db.Cursor(
+		dal.Join("left join _tool_gitea_repo_commits on _tool_gitea_commits.sha = _tool_gitea_repo_commits.commit_sha"),
+		dal.From(models.GiteaCommit{}.TableName()),
+		dal.Where("_tool_gitea_repo_commits.repo_id = ? and _tool_gitea_repo_commits.connection_id = ? and _tool_gitea_commits.committed_date >= ?",
+			data.Repo.GiteaId, data.Repo.ConnectionId, latestUpdated.CommittedDate.String()),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(models.GiteaCommit{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		ApiClient:          data.ApiClient,
+		PageSize:           100,
+		Input:              iterator,
+		UrlTemplate:        "repos/{{ .Params.Owner }}/{{ .Params.Repo }}/commits/{{ .Input.Sha }}/status",
+		/*
+			(Optional) Return query string for request, or you can plug them into UrlTemplate directly
+		*/
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("limit", fmt.Sprintf("%v", reqData.Pager.Size))
+
+			return query, nil
+		},
+
+		ResponseParser: func(res *http.Response) ([]json.RawMessage, error) {
+			body, err := ioutil.ReadAll(res.Body)
+			res.Body.Close()
+			if err != nil {
+				return nil, err
+			}
+			return []json.RawMessage{body}, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
diff --git a/plugins/gitea/tasks/commit_stats_extractor.go b/plugins/gitea/tasks/commit_stats_extractor.go
new file mode 100644
index 00000000..d40693f1
--- /dev/null
+++ b/plugins/gitea/tasks/commit_stats_extractor.go
@@ -0,0 +1,102 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractApiCommitStatsMeta = core.SubTaskMeta{
+	Name:             "extractApiCommitStats",
+	EntryPoint:       ExtractApiCommitStats,
+	EnabledByDefault: false,
+	Description:      "Extract raw commit stats data into tool layer table gitea_commit_stats",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE},
+}
+
+type ApiSingleCommitResponse struct {
+	Sha   string
+	Stats struct {
+		id        string
+		Additions int
+		Deletions int
+		total     int
+	}
+	Commit struct {
+		Committer struct {
+			Name  string
+			Email string
+			Date  helper.Iso8601Time
+		}
+	}
+}
+
+func ExtractApiCommitStats(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMIT_STATS_TABLE)
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			body := &ApiSingleCommitResponse{}
+			err := json.Unmarshal(row.Data, body)
+			if err != nil {
+				return nil, err
+			}
+			if body.Sha == "" {
+				return nil, nil
+			}
+
+			db := taskCtx.GetDal()
+			commit := &models.GiteaCommit{}
+			err = db.First(commit, dal.Where("sha = ?", body.Sha), dal.Limit(1))
+			if err != nil {
+				return nil, err
+			}
+
+			commit.Additions = body.Stats.Additions
+			commit.Deletions = body.Stats.Deletions
+
+			commitStat := &models.GiteaCommitStat{
+				ConnectionId:  data.Options.ConnectionId,
+				Additions:     body.Stats.Additions,
+				Deletions:     body.Stats.Deletions,
+				CommittedDate: body.Commit.Committer.Date.ToTime(),
+				Sha:           body.Sha,
+			}
+
+			results := make([]interface{}, 0, 2)
+
+			results = append(results, commit)
+			results = append(results, commitStat)
+
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
diff --git a/plugins/gitea/tasks/issue_collector.go b/plugins/gitea/tasks/issue_collector.go
new file mode 100644
index 00000000..3db58684
--- /dev/null
+++ b/plugins/gitea/tasks/issue_collector.go
@@ -0,0 +1,102 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+	"fmt"
+	"net/http"
+	"net/url"
+	"time"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/plugins/helper"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+)
+
+const RAW_ISSUE_TABLE = "gitea_api_issues"
+
+var CollectApiIssuesMeta = core.SubTaskMeta{
+	Name:             "collectApiIssues",
+	EntryPoint:       CollectApiIssues,
+	EnabledByDefault: true,
+	Description:      "Collect issues data from Gitea api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func CollectApiIssues(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_TABLE)
+
+	since := data.Since
+	incremental := false
+	// user didn't specify a time range to sync, try load from database
+	if since == nil {
+		var latestUpdated models.GiteaIssue
+		err := db.All(
+			&latestUpdated,
+			dal.Where("repo_id = ? and connection_id = ?", data.Repo.GiteaId, data.Repo.ConnectionId),
+			dal.Orderby("gitea_updated_at DESC"),
+			dal.Limit(1),
+		)
+		if err != nil {
+			return fmt.Errorf("failed to get latest gitea issue record: %w", err)
+		}
+		if latestUpdated.GiteaId > 0 {
+			since = &latestUpdated.GiteaUpdatedAt
+			incremental = true
+		}
+	}
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		ApiClient:          data.ApiClient,
+		PageSize:           50,
+		Incremental:        incremental,
+		UrlTemplate:        "repos/{{ .Params.Owner }}/{{ .Params.Repo }}/issues",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			query.Set("state", "all")
+			if since != nil {
+				query.Set("since", since.Format(time.RFC3339))
+			}
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("limit", fmt.Sprintf("%v", reqData.Pager.Size))
+
+			return query, nil
+		},
+		GetTotalPages: GetTotalPagesFromResponse,
+		ResponseParser: func(res *http.Response) ([]json.RawMessage, error) {
+			var items []json.RawMessage
+			err := helper.UnmarshalResponse(res, &items)
+			if err != nil {
+				return nil, err
+			}
+			return items, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
diff --git a/plugins/gitea/tasks/issue_comment_collector.go b/plugins/gitea/tasks/issue_comment_collector.go
new file mode 100644
index 00000000..535fe7e8
--- /dev/null
+++ b/plugins/gitea/tasks/issue_comment_collector.go
@@ -0,0 +1,93 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"net/url"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+)
+
+const RAW_COMMENTS_TABLE = "gitea_issue_comments"
+
+var CollectApiIssueCommentsMeta = core.SubTaskMeta{
+	Name:             "collectApiIssueComments",
+	EntryPoint:       CollectApiIssueComments,
+	EnabledByDefault: true,
+	Description:      "Collect comments data from Gitea api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func CollectApiIssueComments(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMENTS_TABLE)
+
+	since := data.Since
+	incremental := false
+	// user didn't specify a time range to sync, try load from database
+	// actually, for gitea pull, since doesn't make any sense, gitea pull api doesn't support it
+	if since == nil {
+		var latestUpdatedIssueComment models.GiteaIssueComment
+		err := db.All(
+			&latestUpdatedIssueComment,
+			dal.Join("left join _tool_gitea_issues on _tool_gitea_issues.gitea_id = _tool_gitea_issue_comments.issue_id"),
+			dal.Where(
+				"_tool_gitea_issues.repo_id = ? AND _tool_gitea_issues.connection_id = ?", data.Repo.GiteaId, data.Repo.ConnectionId,
+			),
+			dal.Orderby("gitea_updated_at DESC"),
+			dal.Limit(1),
+		)
+		if err != nil {
+			return fmt.Errorf("failed to get latest gitea issue record: %w", err)
+		}
+
+	}
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		ApiClient:          data.ApiClient,
+		PageSize:           100,
+		Incremental:        incremental,
+
+		UrlTemplate: "repos/{{ .Params.Owner }}/{{ .Params.Repo }}/issues/comments",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+
+			if since != nil {
+				query.Set("since", since.String())
+			}
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("limit", fmt.Sprintf("%v", reqData.Pager.Size))
+
+			return query, nil
+		},
+		GetTotalPages:  GetTotalPagesFromResponse,
+		ResponseParser: GetRawMessageFromResponse,
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
diff --git a/plugins/gitea/tasks/issue_comment_convertor.go b/plugins/gitea/tasks/issue_comment_convertor.go
new file mode 100644
index 00000000..9ef26aed
--- /dev/null
+++ b/plugins/gitea/tasks/issue_comment_convertor.go
@@ -0,0 +1,85 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertIssueCommentsMeta = core.SubTaskMeta{
+	Name:             "convertIssueComments",
+	EntryPoint:       ConvertIssueComments,
+	EnabledByDefault: true,
+	Description:      "ConvertIssueComments data from Gitea api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func ConvertIssueComments(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_COMMENTS_TABLE)
+	repoId := data.Repo.GiteaId
+
+	cursor, err := db.Cursor(
+		dal.From(&models.GiteaIssueComment{}),
+		dal.Join("left join _tool_gitea_issues "+
+			"on _tool_gitea_issues.gitea_id = _tool_gitea_issue_comments.issue_id"),
+		dal.Where("repo_id = ? and _tool_gitea_issues.connection_id = ?", repoId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	issueIdGen := didgen.NewDomainIdGenerator(&models.GiteaIssue{})
+	accountIdGen := didgen.NewDomainIdGenerator(&models.GiteaAccount{})
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		InputRowType:       reflect.TypeOf(models.GiteaIssueComment{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			giteaIssueComment := inputRow.(*models.GiteaIssueComment)
+			domainIssueComment := &ticket.IssueComment{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: issueIdGen.Generate(data.Options.ConnectionId, giteaIssueComment.GiteaId),
+				},
+				IssueId:     issueIdGen.Generate(data.Options.ConnectionId, giteaIssueComment.IssueId),
+				Body:        giteaIssueComment.Body,
+				UserId:      accountIdGen.Generate(data.Options.ConnectionId, giteaIssueComment.AuthorId),
+				CreatedDate: giteaIssueComment.GiteaCreatedAt,
+			}
+			return []interface{}{
+				domainIssueComment,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/gitea/tasks/issue_comment_extractor.go b/plugins/gitea/tasks/issue_comment_extractor.go
new file mode 100644
index 00000000..5744a7a1
--- /dev/null
+++ b/plugins/gitea/tasks/issue_comment_extractor.go
@@ -0,0 +1,114 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractApiIssueCommentsMeta = core.SubTaskMeta{
+	Name:             "extractApiIssueComments",
+	EntryPoint:       ExtractApiIssueComments,
+	EnabledByDefault: true,
+	Description: "Extract raw comment data  into tool layer table gitea_pull_request_comments" +
+		"and gitea_issue_comments",
+	DomainTypes: []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type IssueComment struct {
+	GiteaId int `json:"id"`
+	Body    string
+
+	User struct {
+		Login string
+		Id    int
+	}
+
+	Target struct {
+		Issue struct {
+			Id     int    `json:"id"`
+			Title  string `json:"title"`
+			Number string `json:"number"`
+		}
+		PullRequest string `json:"pull_request"`
+	}
+
+	GiteaCreatedAt helper.Iso8601Time `json:"created_at"`
+	GiteaUpdatedAt helper.Iso8601Time `json:"updated_at"`
+}
+
+func ExtractApiIssueComments(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*GiteaTaskData)
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: GiteaApiParams{
+				Owner: data.Options.Owner,
+				Repo:  data.Options.Repo,
+			},
+			Table: RAW_COMMENTS_TABLE,
+		},
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			apiComment := &IssueComment{}
+			err := json.Unmarshal(row.Data, apiComment)
+			if err != nil {
+				return nil, err
+			}
+			// need to extract 2 kinds of entities here
+			results := make([]interface{}, 0, 2)
+			if apiComment.GiteaId == 0 {
+				return nil, nil
+			}
+			//If this is a pr, ignore
+			issueINumber := apiComment.Target.Issue.Number
+			if err != nil {
+				return nil, err
+			}
+			issue := &models.GiteaIssue{}
+			err = taskCtx.GetDal().All(issue, dal.Where("connection_id = ? and number = ? and repo_id = ?", data.Options.ConnectionId, issueINumber, data.Repo.GiteaId))
+			if err != nil {
+				return nil, err
+			}
+			giteaIssueComment := &models.GiteaIssueComment{
+				ConnectionId:   data.Options.ConnectionId,
+				GiteaId:        apiComment.GiteaId,
+				IssueId:        issue.GiteaId,
+				Body:           apiComment.Body,
+				AuthorName:     apiComment.User.Login,
+				AuthorId:       apiComment.User.Id,
+				GiteaCreatedAt: apiComment.GiteaCreatedAt.ToTime(),
+				GiteaUpdatedAt: apiComment.GiteaUpdatedAt.ToTime(),
+			}
+			results = append(results, giteaIssueComment)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
diff --git a/plugins/gitea/tasks/issue_convertor.go b/plugins/gitea/tasks/issue_convertor.go
new file mode 100644
index 00000000..59eceadd
--- /dev/null
+++ b/plugins/gitea/tasks/issue_convertor.go
@@ -0,0 +1,104 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+	"strconv"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	giteaModels "github.com/apache/incubator-devlake/plugins/gitea/models"
+)
+
+var ConvertIssuesMeta = core.SubTaskMeta{
+	Name:             "convertIssues",
+	EntryPoint:       ConvertIssues,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table gitea_issues into  domain layer table issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func ConvertIssues(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_TABLE)
+	repoId := data.Repo.GiteaId
+
+	issue := &giteaModels.GiteaIssue{}
+	cursor, err := db.Cursor(
+		dal.From(issue),
+		dal.Where("repo_id = ? and connection_id=?", repoId, data.Options.ConnectionId),
+	)
+
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	issueIdGen := didgen.NewDomainIdGenerator(&giteaModels.GiteaIssue{})
+	accountIdGen := didgen.NewDomainIdGenerator(&giteaModels.GiteaAccount{})
+	boardIdGen := didgen.NewDomainIdGenerator(&giteaModels.GiteaRepo{})
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		InputRowType:       reflect.TypeOf(giteaModels.GiteaIssue{}),
+		Input:              cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			issue := inputRow.(*giteaModels.GiteaIssue)
+			domainIssue := &ticket.Issue{
+				DomainEntity:    domainlayer.DomainEntity{Id: issueIdGen.Generate(data.Options.ConnectionId, issue.GiteaId)},
+				IssueKey:        strconv.Itoa(issue.Number),
+				Title:           issue.Title,
+				Description:     issue.Body,
+				AssigneeId:      accountIdGen.Generate(data.Options.ConnectionId, issue.AssigneeId),
+				AssigneeName:    issue.AssigneeName,
+				LeadTimeMinutes: issue.LeadTimeMinutes,
+				CreatorId:       accountIdGen.Generate(data.Options.ConnectionId, issue.AuthorId),
+				CreatorName:     issue.AuthorName,
+				Url:             issue.Url,
+				CreatedDate:     &issue.GiteaCreatedAt,
+				UpdatedDate:     &issue.GiteaUpdatedAt,
+				ResolutionDate:  issue.ClosedAt,
+			}
+			if issue.State == "closed" {
+				domainIssue.Status = ticket.DONE
+			} else {
+				domainIssue.Status = ticket.TODO
+			}
+			boardIssue := &ticket.BoardIssue{
+				BoardId: boardIdGen.Generate(data.Options.ConnectionId, repoId),
+				IssueId: domainIssue.Id,
+			}
+			return []interface{}{
+				domainIssue,
+				boardIssue,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/gitea/tasks/issue_extractor.go b/plugins/gitea/tasks/issue_extractor.go
new file mode 100644
index 00000000..91874b3a
--- /dev/null
+++ b/plugins/gitea/tasks/issue_extractor.go
@@ -0,0 +1,167 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractApiIssuesMeta = core.SubTaskMeta{
+	Name:             "extractApiIssues",
+	EntryPoint:       ExtractApiIssues,
+	EnabledByDefault: true,
+	Description:      "Extract raw Issues data into tool layer table gitea_issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type IssuesResponse struct {
+	GiteaId       int    `json:"id"`
+	Url           string `json:"url"`
+	RepositoryUrl string `json:"repository_url"`
+	Number        int    `json:"number"`
+	State         string `json:"state"`
+	Title         string
+	Body          string
+	HtmlUrl       string `json:"html_url"`
+	CommentsUrl   string `json:"comments_url"`
+	PullRequest   struct {
+		Url     string `json:"url"`
+		HtmlUrl string `json:"html_url"`
+	} `json:"pull_request"`
+	Labels []struct {
+		Id           int
+		RepositoryId int                `json:"repository_id"`
+		Name         string             `json:"name"`
+		CreatedAt    helper.Iso8601Time `json:"created_at"`
+		UpdatedAt    helper.Iso8601Time `json:"updated_at"`
+	} `json:"labels"`
+	Repository struct {
+		Id       int
+		FullName string `json:"full_name"`
+		Url      string `json:"url"`
+	} `json:"repository"`
+	Assignee *struct {
+		Login string
+		Id    int
+	}
+	User *struct {
+		Login string
+		Id    int
+		Name  string
+	}
+	Comments        int                 `json:"comments"`
+	Priority        int                 `json:"priority"`
+	IssueType       string              `json:"issue_type"`
+	SecurityHole    bool                `json:"security_hole"`
+	IssueState      string              `json:"issue_state"`
+	Branch          string              `json:"branch"`
+	FinishAt        *helper.Iso8601Time `json:"finished_at"`
+	GiteaCreatedAt  helper.Iso8601Time  `json:"created_at"`
+	GiteaUpdatedAt  helper.Iso8601Time  `json:"updated_at"`
+	IssueTypeDetail struct {
+		Id        int
+		Title     string
+		Ident     string
+		CreatedAt helper.Iso8601Time `json:"created_at"`
+		UpdatedAt helper.Iso8601Time `json:"updated_at"`
+	}
+	IssueStateDetail struct {
+		Id        int
+		Title     string
+		Serial    string
+		CreatedAt helper.Iso8601Time `json:"created_at"`
+		UpdatedAt helper.Iso8601Time `json:"updated_at"`
+	}
+}
+
+func ExtractApiIssues(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_TABLE)
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			body := &IssuesResponse{}
+			err := json.Unmarshal(row.Data, body)
+			if err != nil {
+				return nil, err
+			}
+			// need to extract 2 kinds of entities here
+			if body.GiteaId == 0 {
+				return nil, nil
+			}
+			//If this is a pr, ignore
+			if body.PullRequest.Url != "" {
+				return nil, nil
+			}
+			results := make([]interface{}, 0, 2)
+			giteaIssue, err := convertGiteaIssue(body, data.Options.ConnectionId, data.Repo.GiteaId)
+			if err != nil {
+				return nil, err
+			}
+			for _, label := range body.Labels {
+				results = append(results, &models.GiteaIssueLabel{
+					ConnectionId: data.Options.ConnectionId,
+					IssueId:      giteaIssue.GiteaId,
+					LabelName:    label.Name,
+				})
+
+			}
+			results = append(results, giteaIssue)
+
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
+func convertGiteaIssue(issue *IssuesResponse, connectionId uint64, repositoryId int) (*models.GiteaIssue, error) {
+	giteaIssue := &models.GiteaIssue{
+		ConnectionId:   connectionId,
+		GiteaId:        issue.GiteaId,
+		RepoId:         repositoryId,
+		Number:         issue.Number,
+		State:          issue.State,
+		Title:          issue.Title,
+		Body:           issue.Body,
+		Url:            issue.HtmlUrl,
+		ClosedAt:       helper.Iso8601TimeToTime(issue.FinishAt),
+		GiteaCreatedAt: issue.GiteaCreatedAt.ToTime(),
+		GiteaUpdatedAt: issue.GiteaUpdatedAt.ToTime(),
+	}
+
+	if issue.Assignee != nil {
+		giteaIssue.AssigneeId = issue.Assignee.Id
+		giteaIssue.AssigneeName = issue.Assignee.Login
+	}
+	if issue.User != nil {
+		giteaIssue.AuthorId = issue.User.Id
+		giteaIssue.AuthorName = issue.User.Login
+	}
+	if issue.FinishAt != nil {
+		giteaIssue.LeadTimeMinutes = uint(issue.FinishAt.ToTime().Sub(issue.GiteaCreatedAt.ToTime()).Minutes())
+	}
+
+	return giteaIssue, nil
+}
diff --git a/plugins/gitea/tasks/issue_label_convertor.go b/plugins/gitea/tasks/issue_label_convertor.go
new file mode 100644
index 00000000..395bc678
--- /dev/null
+++ b/plugins/gitea/tasks/issue_label_convertor.go
@@ -0,0 +1,78 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertIssueLabelsMeta = core.SubTaskMeta{
+	Name:             "convertIssueLabels",
+	EntryPoint:       ConvertIssueLabels,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table gitea_issue_labels into  domain layer table issue_labels",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+func ConvertIssueLabels(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_ISSUE_TABLE)
+	repoId := data.Repo.GiteaId
+
+	cursor, err := db.Cursor(
+		dal.From(&models.GiteaIssueLabel{}),
+		dal.Join(`left join _tool_gitea_issues on _tool_gitea_issues.gitea_id = _tool_gitea_issue_labels.issue_id`),
+		dal.Where("_tool_gitea_issues.repo_id = ? and _tool_gitea_issues.connection_id = ?", repoId, data.Options.ConnectionId),
+		dal.Orderby("issue_id ASC"),
+	)
+
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+	issueIdGen := didgen.NewDomainIdGenerator(&models.GiteaIssue{})
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		InputRowType:       reflect.TypeOf(models.GiteaIssueLabel{}),
+		Input:              cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			issueLabel := inputRow.(*models.GiteaIssueLabel)
+			domainIssueLabel := &ticket.IssueLabel{
+				IssueId:   issueIdGen.Generate(data.Options.ConnectionId, issueLabel.IssueId),
+				LabelName: issueLabel.LabelName,
+			}
+			return []interface{}{
+				domainIssueLabel,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/gitea/tasks/repo_collector.go b/plugins/gitea/tasks/repo_collector.go
new file mode 100644
index 00000000..f1f438c7
--- /dev/null
+++ b/plugins/gitea/tasks/repo_collector.go
@@ -0,0 +1,72 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"net/url"
+
+	"github.com/apache/incubator-devlake/plugins/helper"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+)
+
+const RAW_REPOSITORIES_TABLE = "gitea_api_repos"
+
+var CollectApiRepoMeta = core.SubTaskMeta{
+	Name:        "collectApiRepo",
+	EntryPoint:  CollectApiRepositories,
+	Required:    true,
+	Description: "Collect repositories data from Gitea api",
+	DomainTypes: []string{core.DOMAIN_TYPE_CODE},
+}
+
+func CollectApiRepositories(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_REPOSITORIES_TABLE)
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		ApiClient:          data.ApiClient,
+		PageSize:           50,
+		UrlTemplate:        "repos/{{ .Params.Owner }}/{{ .Params.Repo }}",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			query.Set("state", "all")
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("limit", fmt.Sprintf("%v", reqData.Pager.Size))
+			return query, nil
+		},
+		ResponseParser: func(res *http.Response) ([]json.RawMessage, error) {
+			body, err := ioutil.ReadAll(res.Body)
+			res.Body.Close()
+			if err != nil {
+				return nil, err
+			}
+			return []json.RawMessage{body}, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return collector.Execute()
+}
diff --git a/plugins/gitea/tasks/repo_convertor.go b/plugins/gitea/tasks/repo_convertor.go
new file mode 100644
index 00000000..2b80e8f8
--- /dev/null
+++ b/plugins/gitea/tasks/repo_convertor.go
@@ -0,0 +1,99 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/code"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ConvertRepoMeta = core.SubTaskMeta{
+	Name:             "convertRepo",
+	EntryPoint:       ConvertRepo,
+	EnabledByDefault: true,
+	Description:      "Convert tool layer table gitea_repos into  domain layer table repos and boards",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CODE},
+}
+
+func ConvertRepo(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_REPOSITORIES_TABLE)
+	db := taskCtx.GetDal()
+	repoId := data.Repo.GiteaId
+
+	cursor, err := db.Cursor(
+		dal.From(&models.GiteaRepo{}),
+		dal.Where("gitea_id = ?", repoId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	repoIdGen := didgen.NewDomainIdGenerator(&models.GiteaRepo{})
+
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		InputRowType:       reflect.TypeOf(models.GiteaRepo{}),
+		Input:              cursor,
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Convert: func(inputRow interface{}) ([]interface{}, error) {
+			repository := inputRow.(*models.GiteaRepo)
+			domainRepository := &code.Repo{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: repoIdGen.Generate(data.Options.ConnectionId, repository.GiteaId),
+				},
+				Name:        fmt.Sprintf("%s/%s", repository.OwnerName, repository.Name),
+				Url:         repository.HTMLUrl,
+				Description: repository.Description,
+				ForkedFrom:  repository.ParentHTMLUrl,
+				Language:    repository.Language,
+				CreatedDate: repository.CreatedDate,
+				UpdatedDate: repository.UpdatedDate,
+			}
+
+			domainBoard := &ticket.Board{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: repoIdGen.Generate(data.Options.ConnectionId, repository.GiteaId),
+				},
+				Name:        fmt.Sprintf("%s/%s", repository.OwnerName, repository.Name),
+				Url:         fmt.Sprintf("%s/%s", repository.HTMLUrl, "issues"),
+				Description: repository.Description,
+				CreatedDate: &repository.CreatedDate,
+			}
+
+			return []interface{}{
+				domainRepository,
+				domainBoard,
+			}, nil
+		},
+	})
+	if err != nil {
+		return err
+	}
+
+	return converter.Execute()
+}
diff --git a/plugins/gitea/tasks/repo_extractor.go b/plugins/gitea/tasks/repo_extractor.go
new file mode 100644
index 00000000..f7816596
--- /dev/null
+++ b/plugins/gitea/tasks/repo_extractor.go
@@ -0,0 +1,93 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+	"fmt"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractApiRepoMeta = core.SubTaskMeta{
+	Name:        "extractApiRepo",
+	EntryPoint:  ExtractApiRepositories,
+	Required:    true,
+	Description: "Extract raw Repositories data into tool layer table gitea_repos",
+	DomainTypes: []string{core.DOMAIN_TYPE_CODE},
+}
+
+type GiteaApiRepoResponse struct {
+	Name        string                `json:"name"`
+	GiteaId     int                   `json:"id"`
+	HTMLUrl     string                `json:"html_url"`
+	CloneUrl    string                `json:"clone_url"`
+	Language    string                `json:"language"`
+	Description string                `json:"description"`
+	Owner       models.GiteaAccount   `json:"owner"`
+	Parent      *GiteaApiRepoResponse `json:"parent"`
+	CreatedAt   helper.Iso8601Time    `json:"created_at"`
+	UpdatedAt   *helper.Iso8601Time   `json:"updated_at"`
+}
+
+func ExtractApiRepositories(taskCtx core.SubTaskContext) error {
+	rawDataSubTaskArgs, data := CreateRawDataSubTaskArgs(taskCtx, RAW_REPOSITORIES_TABLE)
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: *rawDataSubTaskArgs,
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			repo := &GiteaApiRepoResponse{}
+			err := json.Unmarshal(row.Data, repo)
+			if err != nil {
+				return nil, err
+			}
+			if repo.GiteaId == 0 {
+				return nil, fmt.Errorf("repo %s/%s not found", data.Options.Owner, data.Options.Repo)
+			}
+			results := make([]interface{}, 0, 1)
+			giteaRepository := &models.GiteaRepo{
+				ConnectionId: data.Options.ConnectionId,
+				GiteaId:      repo.GiteaId,
+				Name:         repo.Name,
+				HTMLUrl:      repo.HTMLUrl,
+				Description:  repo.Description,
+				OwnerId:      repo.Owner.Id,
+				OwnerName:    repo.Owner.Login,
+				Language:     repo.Language,
+				CreatedDate:  repo.CreatedAt.ToTime(),
+				UpdatedDate:  helper.Iso8601TimeToTime(repo.UpdatedAt),
+			}
+			data.Repo = giteaRepository
+
+			if repo.Parent != nil {
+				giteaRepository.ParentGiteaId = repo.Parent.GiteaId
+				giteaRepository.ParentHTMLUrl = repo.Parent.HTMLUrl
+			}
+			results = append(results, giteaRepository)
+			taskCtx.TaskContext().GetData().(*GiteaTaskData).Repo = giteaRepository
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
diff --git a/plugins/gitea/tasks/shared.go b/plugins/gitea/tasks/shared.go
new file mode 100644
index 00000000..2d2d17e0
--- /dev/null
+++ b/plugins/gitea/tasks/shared.go
@@ -0,0 +1,101 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"encoding/json"
+	"fmt"
+	"io/ioutil"
+	"net/http"
+	"strconv"
+	"time"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+type PagingInfo struct {
+	Next  int
+	Last  int
+	First int
+	Prev  int
+}
+
+type RateLimitInfo struct {
+	Date      time.Time
+	ResetTime time.Time
+	Remaining int
+}
+
+type GiteaApiParams struct {
+	ConnectionId uint64
+	Repo         string
+	Owner        string
+}
+
+type GiteaInput struct {
+	Repo  string
+	Owner string
+	Iid   int
+}
+
+func GetTotalPagesFromResponse(res *http.Response, args *helper.ApiCollectorArgs) (int, error) {
+	total := res.Header.Get("X-PageCount")
+	if total == "" {
+		return 0, nil
+	}
+	totalInt, err := strconv.Atoi(total)
+	if err != nil {
+		return 0, err
+	}
+	return totalInt, nil
+}
+
+func GetRawMessageFromResponse(res *http.Response) ([]json.RawMessage, error) {
+	var rawMessages []json.RawMessage
+
+	if res == nil {
+		return nil, fmt.Errorf("res is nil")
+	}
+	defer res.Body.Close()
+	resBody, err := ioutil.ReadAll(res.Body)
+	if err != nil {
+		return nil, fmt.Errorf("%w %s", err, res.Request.URL.String())
+	}
+
+	err = json.Unmarshal(resBody, &rawMessages)
+	if err != nil {
+		return nil, fmt.Errorf("%w %s %s", err, res.Request.URL.String(), string(resBody))
+	}
+
+	return rawMessages, nil
+}
+
+func CreateRawDataSubTaskArgs(taskCtx core.SubTaskContext, Table string) (*helper.RawDataSubTaskArgs, *GiteaTaskData) {
+	data := taskCtx.GetData().(*GiteaTaskData)
+	RawDataSubTaskArgs := &helper.RawDataSubTaskArgs{
+		Ctx: taskCtx,
+		Params: GiteaApiParams{
+			ConnectionId: data.Options.ConnectionId,
+			Repo:         data.Options.Repo,
+			Owner:        data.Options.Owner,
+		},
+		Table: Table,
+	}
+	return RawDataSubTaskArgs, data
+}
diff --git a/plugins/gitea/tasks/task_data.go b/plugins/gitea/tasks/task_data.go
new file mode 100644
index 00000000..e00d8840
--- /dev/null
+++ b/plugins/gitea/tasks/task_data.go
@@ -0,0 +1,60 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/mitchellh/mapstructure"
+	"time"
+
+	"github.com/apache/incubator-devlake/plugins/gitea/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+type GiteaOptions struct {
+	ConnectionId uint64 `json:"connectionId"`
+	Owner        string
+	Repo         string
+}
+
+type GiteaTaskData struct {
+	Options   *GiteaOptions
+	ApiClient *helper.ApiAsyncClient
+	Repo      *models.GiteaRepo
+	Since     *time.Time
+}
+
+func DecodeAndValidateTaskOptions(options map[string]interface{}) (*GiteaOptions, error) {
+	var op GiteaOptions
+	err := mapstructure.Decode(options, &op)
+	if err != nil {
+		return nil, err
+	}
+	if op.Owner == "" {
+		return nil, fmt.Errorf("owner is required for Gitea execution")
+	}
+	if op.Repo == "" {
+		return nil, fmt.Errorf("repo is required for Gitea execution")
+	}
+
+	// find the needed GitHub now
+	if op.ConnectionId == 0 {
+		return nil, fmt.Errorf("connectionId is invalid")
+	}
+	return &op, nil
+}