You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by ma...@apache.org on 2022/08/03 07:26:31 UTC

[incubator-devlake] branch main updated: feat: github add cicd feature (#2651)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new eddc87ce feat: github add cicd feature (#2651)
eddc87ce is described below

commit eddc87ce7eec06462b1e40518334c692699011fc
Author: abeizn <10...@users.noreply.github.com>
AuthorDate: Wed Aug 3 15:26:28 2022 +0800

    feat: github add cicd feature (#2651)
    
    * feat: github add cicd feature
    
    * feat: github add cicd feature and update fields
---
 plugins/github/impl/impl.go                        |   4 +
 plugins/github/models/job.go                       |  52 +++++++++++
 .../20220728_add_github_runs_table.go              |  83 +++++++++++++++++
 .../20220729_add_github_jobs_table.go              |  75 +++++++++++++++
 plugins/github/models/migrationscripts/register.go |   2 +
 plugins/github/models/run.go                       |  60 ++++++++++++
 plugins/github/tasks/cicd_job_collector.go         | 103 +++++++++++++++++++++
 plugins/github/tasks/cicd_job_extractor.go         |  88 ++++++++++++++++++
 plugins/github/tasks/cicd_run_collector.go         |  83 +++++++++++++++++
 plugins/github/tasks/cicd_run_extractor.go         |  98 ++++++++++++++++++++
 10 files changed, 648 insertions(+)

diff --git a/plugins/github/impl/impl.go b/plugins/github/impl/impl.go
index 56c0a16f..d57345f2 100644
--- a/plugins/github/impl/impl.go
+++ b/plugins/github/impl/impl.go
@@ -78,6 +78,10 @@ func (plugin Github) SubTaskMetas() []core.SubTaskMeta {
 		tasks.ExtractAccountsMeta,
 		tasks.CollectAccountOrgMeta,
 		tasks.ExtractAccountOrgMeta,
+		tasks.CollectRunsMeta,
+		tasks.ExtractRunsMeta,
+		tasks.CollectJobsMeta,
+		tasks.ExtractJobsMeta,
 		tasks.EnrichPullRequestIssuesMeta,
 		tasks.ConvertRepoMeta,
 		tasks.ConvertIssuesMeta,
diff --git a/plugins/github/models/job.go b/plugins/github/models/job.go
new file mode 100644
index 00000000..15acaa6b
--- /dev/null
+++ b/plugins/github/models/job.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"
+	"gorm.io/datatypes"
+)
+
+type GithubJob struct {
+	common.NoPKModel
+	ConnectionId  uint64         `gorm:"primaryKey"`
+	ID            int            `json:"id" gorm:"primaryKey;autoIncrement:false"`
+	RunID         int            `json:"run_id"`
+	RunURL        string         `json:"run_url" gorm:"type:varchar(255)"`
+	NodeID        string         `json:"node_id" gorm:"type:varchar(255)"`
+	HeadSha       string         `json:"head_sha" gorm:"type:varchar(255)"`
+	URL           string         `json:"url" gorm:"type:varchar(255)"`
+	HTMLURL       string         `json:"html_url" gorm:"type:varchar(255)"`
+	Status        string         `json:"status" gorm:"type:varchar(255)"`
+	Conclusion    string         `json:"conclusion" gorm:"type:varchar(255)"`
+	StartedAt     *time.Time     `json:"started_at"`
+	CompletedAt   *time.Time     `json:"completed_at"`
+	Name          string         `json:"name" gorm:"type:varchar(255)"`
+	Steps         datatypes.JSON `json:"steps"`
+	CheckRunURL   string         `json:"check_run_url" gorm:"type:varchar(255)"`
+	Labels        datatypes.JSON `json:"labels"`
+	RunnerID      int            `json:"runner_id"`
+	RunnerName    string         `json:"runner_name" gorm:"type:varchar(255)"`
+	RunnerGroupID int            `json:"runner_group_id"`
+}
+
+func (GithubJob) TableName() string {
+	return "_tool_github_jobs"
+}
diff --git a/plugins/github/models/migrationscripts/20220728_add_github_runs_table.go b/plugins/github/models/migrationscripts/20220728_add_github_runs_table.go
new file mode 100644
index 00000000..bc5b6307
--- /dev/null
+++ b/plugins/github/models/migrationscripts/20220728_add_github_runs_table.go
@@ -0,0 +1,83 @@
+/*
+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"
+	"time"
+
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
+	"gorm.io/gorm"
+)
+
+type GithubRun20220728 struct {
+	archived.NoPKModel
+	ConnectionId     uint64     `gorm:"primaryKey"`
+	GithubId         int        `gorm:"primaryKey"`
+	ID               int64      `json:"id" gorm:"primaryKey;autoIncrement:false"`
+	Name             string     `json:"name" gorm:"type:varchar(255)"`
+	NodeID           string     `json:"node_id" gorm:"type:varchar(255)"`
+	HeadBranch       string     `json:"head_branch" gorm:"type:varchar(255)"`
+	HeadSha          string     `json:"head_sha" gorm:"type:varchar(255)"`
+	Path             string     `json:"path" gorm:"type:varchar(255)"`
+	RunNumber        int        `json:"run_number"`
+	Event            string     `json:"event" gorm:"type:varchar(255)"`
+	Status           string     `json:"status" gorm:"type:varchar(255)"`
+	Conclusion       string     `json:"conclusion" gorm:"type:varchar(255)"`
+	WorkflowID       int        `json:"workflow_id"`
+	CheckSuiteID     int64      `json:"check_suite_id"`
+	CheckSuiteNodeID string     `json:"check_suite_node_id" gorm:"type:varchar(255)"`
+	URL              string     `json:"url" gorm:"type:varchar(255)"`
+	HTMLURL          string     `json:"html_url" gorm:"type:varchar(255)"`
+	GithubCreatedAt  *time.Time `json:"created_at"`
+	GithubUpdatedAt  *time.Time `json:"updated_at"`
+	RunAttempt       int        `json:"run_attempt"`
+	RunStartedAt     *time.Time `json:"run_started_at"`
+	JobsURL          string     `json:"jobs_url" gorm:"type:varchar(255)"`
+	LogsURL          string     `json:"logs_url" gorm:"type:varchar(255)"`
+	CheckSuiteURL    string     `json:"check_suite_url" gorm:"type:varchar(255)"`
+	ArtifactsURL     string     `json:"artifacts_url" gorm:"type:varchar(255)"`
+	CancelURL        string     `json:"cancel_url" gorm:"type:varchar(255)"`
+	RerunURL         string     `json:"rerun_url" gorm:"type:varchar(255)"`
+	WorkflowURL      string     `json:"workflow_url" gorm:"type:varchar(255)"`
+}
+
+func (GithubRun20220728) TableName() string {
+	return "_tool_github_runs"
+}
+
+type addGithubRunsTable struct{}
+
+func (u *addGithubRunsTable) Up(ctx context.Context, db *gorm.DB) error {
+	// create table
+	err := db.Migrator().CreateTable(GithubRun20220728{})
+	if err != nil {
+		return fmt.Errorf("create table _tool_github_runs error")
+	}
+	return nil
+
+}
+
+func (*addGithubRunsTable) Version() uint64 {
+	return 20220728000001
+}
+
+func (*addGithubRunsTable) Name() string {
+	return "Github add github_runs table"
+}
diff --git a/plugins/github/models/migrationscripts/20220729_add_github_jobs_table.go b/plugins/github/models/migrationscripts/20220729_add_github_jobs_table.go
new file mode 100644
index 00000000..981addd4
--- /dev/null
+++ b/plugins/github/models/migrationscripts/20220729_add_github_jobs_table.go
@@ -0,0 +1,75 @@
+/*
+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"
+	"time"
+
+	"github.com/apache/incubator-devlake/models/migrationscripts/archived"
+	"gorm.io/datatypes"
+	"gorm.io/gorm"
+)
+
+type GithubJob20220729 struct {
+	archived.NoPKModel
+	ConnectionId  uint64         `gorm:"primaryKey"`
+	ID            int            `json:"id" gorm:"primaryKey;autoIncrement:false"`
+	RunID         int            `json:"run_id"`
+	RunURL        string         `json:"run_url" gorm:"type:varchar(255)"`
+	NodeID        string         `json:"node_id" gorm:"type:varchar(255)"`
+	HeadSha       string         `json:"head_sha" gorm:"type:varchar(255)"`
+	URL           string         `json:"url" gorm:"type:varchar(255)"`
+	HTMLURL       string         `json:"html_url" gorm:"type:varchar(255)"`
+	Status        string         `json:"status" gorm:"type:varchar(255)"`
+	Conclusion    string         `json:"conclusion" gorm:"type:varchar(255)"`
+	StartedAt     *time.Time     `json:"started_at"`
+	CompletedAt   *time.Time     `json:"completed_at"`
+	Name          string         `json:"name" gorm:"type:varchar(255)"`
+	Steps         datatypes.JSON `json:"steps"`
+	CheckRunURL   string         `json:"check_run_url" gorm:"type:varchar(255)"`
+	Labels        datatypes.JSON `json:"labels"`
+	RunnerID      int            `json:"runner_id"`
+	RunnerName    string         `json:"runner_name" gorm:"type:varchar(255)"`
+	RunnerGroupID int            `json:"runner_group_id"`
+}
+
+func (GithubJob20220729) TableName() string {
+	return "_tool_github_jobs"
+}
+
+type addGithubJobsTable struct{}
+
+func (u *addGithubJobsTable) Up(ctx context.Context, db *gorm.DB) error {
+	// create table
+	err := db.Migrator().CreateTable(GithubJob20220729{})
+	if err != nil {
+		return fmt.Errorf("create table _tool_github_jobs error")
+	}
+	return nil
+
+}
+
+func (*addGithubJobsTable) Version() uint64 {
+	return 20220729000001
+}
+
+func (*addGithubJobsTable) Name() string {
+	return "Github add github_jobs table"
+}
diff --git a/plugins/github/models/migrationscripts/register.go b/plugins/github/models/migrationscripts/register.go
index c1365f7d..4f899e78 100644
--- a/plugins/github/models/migrationscripts/register.go
+++ b/plugins/github/models/migrationscripts/register.go
@@ -25,5 +25,7 @@ import (
 func All() []migration.Script {
 	return []migration.Script{
 		new(addInitTables),
+		new(addGithubRunsTable),
+		new(addGithubJobsTable),
 	}
 }
diff --git a/plugins/github/models/run.go b/plugins/github/models/run.go
new file mode 100644
index 00000000..d749f3a6
--- /dev/null
+++ b/plugins/github/models/run.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 models
+
+import (
+	"time"
+
+	"github.com/apache/incubator-devlake/models/common"
+)
+
+type GithubRun struct {
+	common.NoPKModel
+	ConnectionId     uint64     `gorm:"primaryKey"`
+	GithubId         int        `gorm:"primaryKey"`
+	ID               int64      `json:"id" gorm:"primaryKey;autoIncrement:false"`
+	Name             string     `json:"name" gorm:"type:varchar(255)"`
+	NodeID           string     `json:"node_id" gorm:"type:varchar(255)"`
+	HeadBranch       string     `json:"head_branch" gorm:"type:varchar(255)"`
+	HeadSha          string     `json:"head_sha" gorm:"type:varchar(255)"`
+	Path             string     `json:"path" gorm:"type:varchar(255)"`
+	RunNumber        int        `json:"run_number"`
+	Event            string     `json:"event" gorm:"type:varchar(255)"`
+	Status           string     `json:"status" gorm:"type:varchar(255)"`
+	Conclusion       string     `json:"conclusion" gorm:"type:varchar(255)"`
+	WorkflowID       int        `json:"workflow_id"`
+	CheckSuiteID     int64      `json:"check_suite_id"`
+	CheckSuiteNodeID string     `json:"check_suite_node_id" gorm:"type:varchar(255)"`
+	URL              string     `json:"url" gorm:"type:varchar(255)"`
+	HTMLURL          string     `json:"html_url" gorm:"type:varchar(255)"`
+	GithubCreatedAt  *time.Time `json:"created_at"`
+	GithubUpdatedAt  *time.Time `json:"updated_at"`
+	RunAttempt       int        `json:"run_attempt"`
+	RunStartedAt     *time.Time `json:"run_started_at"`
+	JobsURL          string     `json:"jobs_url" gorm:"type:varchar(255)"`
+	LogsURL          string     `json:"logs_url" gorm:"type:varchar(255)"`
+	CheckSuiteURL    string     `json:"check_suite_url" gorm:"type:varchar(255)"`
+	ArtifactsURL     string     `json:"artifacts_url" gorm:"type:varchar(255)"`
+	CancelURL        string     `json:"cancel_url" gorm:"type:varchar(255)"`
+	RerunURL         string     `json:"rerun_url" gorm:"type:varchar(255)"`
+	WorkflowURL      string     `json:"workflow_url" gorm:"type:varchar(255)"`
+}
+
+func (GithubRun) TableName() string {
+	return "_tool_github_runs"
+}
diff --git a/plugins/github/tasks/cicd_job_collector.go b/plugins/github/tasks/cicd_job_collector.go
new file mode 100644
index 00000000..d78034ad
--- /dev/null
+++ b/plugins/github/tasks/cicd_job_collector.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 (
+	"encoding/json"
+	"fmt"
+	"net/http"
+	"net/url"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+const RAW_JOB_TABLE = "github_api_jobs"
+
+var CollectJobsMeta = core.SubTaskMeta{
+	Name:             "collectJobs",
+	EntryPoint:       CollectJobs,
+	EnabledByDefault: true,
+	Description:      "Collect Jobs data from Github action api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CICD},
+}
+
+func CollectJobs(taskCtx core.SubTaskContext) error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*GithubTaskData)
+	cursor, err := db.Cursor(
+		dal.Select("id"),
+		dal.From(models.GithubRun{}.TableName()),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleGithubRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_JOB_TABLE,
+		},
+		ApiClient:   data.ApiClient,
+		PageSize:    100,
+		Input:       iterator,
+		Incremental: false,
+		UrlTemplate: "repos/{{ .Params.Owner }}/{{ .Params.Repo }}/actions/runs/{{ .Input.ID }}/jobs",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("per_page", fmt.Sprintf("%v", reqData.Pager.Size))
+			return query, nil
+		},
+		GetTotalPages: GetTotalPagesFromResponse,
+		ResponseParser: func(res *http.Response) ([]json.RawMessage, error) {
+			body := &GithubRawJobsResult{}
+			err := helper.UnmarshalResponse(res, body)
+			if err != nil {
+				return nil, err
+			}
+			return body.GithubWorkflowJobs, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+	return collector.Execute()
+}
+
+type SimpleGithubRun struct {
+	ID int64
+}
+
+type GithubRawJobsResult struct {
+	TotalCount         int64             `json:"total_count"`
+	GithubWorkflowJobs []json.RawMessage `json:"jobs"`
+}
diff --git a/plugins/github/tasks/cicd_job_extractor.go b/plugins/github/tasks/cicd_job_extractor.go
new file mode 100644
index 00000000..9cbf58db
--- /dev/null
+++ b/plugins/github/tasks/cicd_job_extractor.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 tasks
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractJobsMeta = core.SubTaskMeta{
+	Name:             "extractJobs",
+	EntryPoint:       ExtractJobs,
+	EnabledByDefault: true,
+	Description:      "Extract raw run data into tool layer table github_jobs",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CICD},
+}
+
+func ExtractJobs(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*GithubTaskData)
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_JOB_TABLE,
+		},
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			githubJob := &models.GithubJob{}
+			err := json.Unmarshal(row.Data, githubJob)
+			if err != nil {
+				return nil, err
+			}
+
+			results := make([]interface{}, 0, 1)
+			githubJobResult := &models.GithubJob{
+				ConnectionId:  data.Options.ConnectionId,
+				ID:            githubJob.ID,
+				RunID:         githubJob.RunID,
+				RunURL:        githubJob.RunURL,
+				NodeID:        githubJob.NodeID,
+				HeadSha:       githubJob.HeadSha,
+				URL:           githubJob.URL,
+				HTMLURL:       githubJob.HTMLURL,
+				Status:        githubJob.Status,
+				Conclusion:    githubJob.Conclusion,
+				StartedAt:     githubJob.StartedAt,
+				CompletedAt:   githubJob.CompletedAt,
+				Name:          githubJob.Name,
+				Steps:         githubJob.Steps,
+				CheckRunURL:   githubJob.CheckRunURL,
+				Labels:        githubJob.Labels,
+				RunnerID:      githubJob.RunID,
+				RunnerName:    githubJob.RunnerName,
+				RunnerGroupID: githubJob.RunnerGroupID,
+			}
+			results = append(results, githubJobResult)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}
diff --git a/plugins/github/tasks/cicd_run_collector.go b/plugins/github/tasks/cicd_run_collector.go
new file mode 100644
index 00000000..91068da3
--- /dev/null
+++ b/plugins/github/tasks/cicd_run_collector.go
@@ -0,0 +1,83 @@
+/*
+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"
+
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+const RAW_RUN_TABLE = "github_api_runs"
+
+var CollectRunsMeta = core.SubTaskMeta{
+	Name:             "collectRuns",
+	EntryPoint:       CollectRuns,
+	EnabledByDefault: true,
+	Description:      "Collect Runs data from Github action api",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CICD},
+}
+
+func CollectRuns(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*GithubTaskData)
+
+	collector, err := helper.NewApiCollector(helper.ApiCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_RUN_TABLE,
+		},
+		ApiClient:   data.ApiClient,
+		PageSize:    100,
+		Incremental: false,
+		UrlTemplate: "repos/{{ .Params.Owner }}/{{ .Params.Repo }}/actions/runs",
+		Query: func(reqData *helper.RequestData) (url.Values, error) {
+			query := url.Values{}
+			query.Set("page", fmt.Sprintf("%v", reqData.Pager.Page))
+			query.Set("per_page", fmt.Sprintf("%v", reqData.Pager.Size))
+			return query, nil
+		},
+		GetTotalPages: GetTotalPagesFromResponse,
+		ResponseParser: func(res *http.Response) ([]json.RawMessage, error) {
+			body := &GithubRawRunsResult{}
+			err := helper.UnmarshalResponse(res, body)
+			if err != nil {
+				return nil, err
+			}
+			return body.GithubWorkflowRuns, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+	return collector.Execute()
+}
+
+type GithubRawRunsResult struct {
+	TotalCount         int64             `json:"total_count"`
+	GithubWorkflowRuns []json.RawMessage `json:"workflow_runs"`
+}
diff --git a/plugins/github/tasks/cicd_run_extractor.go b/plugins/github/tasks/cicd_run_extractor.go
new file mode 100644
index 00000000..87d07b72
--- /dev/null
+++ b/plugins/github/tasks/cicd_run_extractor.go
@@ -0,0 +1,98 @@
+/*
+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/github/models"
+	"github.com/apache/incubator-devlake/plugins/helper"
+)
+
+var ExtractRunsMeta = core.SubTaskMeta{
+	Name:             "extractRuns",
+	EntryPoint:       ExtractRuns,
+	EnabledByDefault: true,
+	Description:      "Extract raw run data into tool layer table github_runs",
+	DomainTypes:      []string{core.DOMAIN_TYPE_CICD},
+}
+
+func ExtractRuns(taskCtx core.SubTaskContext) error {
+	data := taskCtx.GetData().(*GithubTaskData)
+	repoId := data.Repo.GithubId
+
+	extractor, err := helper.NewApiExtractor(helper.ApiExtractorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_RUN_TABLE,
+		},
+		Extract: func(row *helper.RawData) ([]interface{}, error) {
+			githubRun := &models.GithubRun{}
+			err := json.Unmarshal(row.Data, githubRun)
+			if err != nil {
+				return nil, err
+			}
+
+			results := make([]interface{}, 0, 1)
+			githubRunResult := &models.GithubRun{
+				ConnectionId:     data.Options.ConnectionId,
+				GithubId:         repoId,
+				ID:               githubRun.ID,
+				Name:             githubRun.Name,
+				NodeID:           githubRun.NodeID,
+				HeadBranch:       githubRun.HeadBranch,
+				HeadSha:          githubRun.HeadSha,
+				Path:             githubRun.Path,
+				RunNumber:        githubRun.RunNumber,
+				Event:            githubRun.Event,
+				Status:           githubRun.Status,
+				Conclusion:       githubRun.Conclusion,
+				WorkflowID:       githubRun.WorkflowID,
+				CheckSuiteID:     githubRun.CheckSuiteID,
+				CheckSuiteNodeID: githubRun.CheckSuiteNodeID,
+				URL:              githubRun.URL,
+				HTMLURL:          githubRun.HTMLURL,
+				GithubCreatedAt:  githubRun.GithubCreatedAt,
+				GithubUpdatedAt:  githubRun.GithubUpdatedAt,
+				RunAttempt:       githubRun.RunAttempt,
+				RunStartedAt:     githubRun.RunStartedAt,
+				JobsURL:          githubRun.JobsURL,
+				LogsURL:          githubRun.LogsURL,
+				CheckSuiteURL:    githubRun.CheckSuiteURL,
+				ArtifactsURL:     githubRun.ArtifactsURL,
+				CancelURL:        githubRun.CancelURL,
+				RerunURL:         githubRun.RerunURL,
+				WorkflowURL:      githubRun.WorkflowURL,
+			}
+			results = append(results, githubRunResult)
+			return results, nil
+		},
+	})
+
+	if err != nil {
+		return err
+	}
+
+	return extractor.Execute()
+}