You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by GitBox <gi...@apache.org> on 2022/11/09 14:42:14 UTC

[GitHub] [incubator-devlake] likyh opened a new pull request, #3705: feat: add check_run collector in github graphql

likyh opened a new pull request, #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705

   # Summary
   
   <!--
   Thanks for submitting a pull request!
   
   We appreciate you spending the time to work on these changes.
   Please fill out as many sections below as possible.
   -->
   CheckSuite.CheckRuns have some columns which are similar to WorkflowJobs.
   
   So collect `CheckRuns` on each CheckSuite to replace WorkflowJobCollector.
   
   Now query 30 CheckSuite in one graphql request. This new collector costs about 40min to collect Devlake's runs & jobs on my computer, and its speed is about 1000/min (CheckSuite) or 2 requests per second.
   
   ### Does this close any open issues?
   Closes #3704
   
   ### Screenshots
   
   The result in `_tool_github_jobs`:
   
   <table border="1" style="border-collapse:collapse">
   <tr><th>created_at</th><th>updated_at</th><th>_raw_data_params</th><th>_raw_data_table</th><th>_raw_data_id</th><th>_raw_data_remark</th><th>connection_id</th><th>repo_id</th><th>id</th><th>run_id</th><th>run_url</th><th>node_id</th><th>head_sha</th><th>url</th><th>html_url</th><th>status</th><th>conclusion</th><th>started_at</th><th>completed_at</th><th>name</th><th>steps</th><th>check_run_url</th><th>labels</th><th>runner_id</th><th>runner_name</th><th>runner_group_id</th><th>type</th></tr>
   <tr><td>2022-11-09 14:08:19.001904 +00:00</td><td>2022-11-09 14:08:19.001904 +00:00</td><td>{&quot;ConnectionId&quot;:1,&quot;Owner&quot;:&quot;apache&quot;,&quot;Repo&quot;:&quot;incubator-devlake&quot;}</td><td>_raw_github_graphql_check_runs</td><td>280</td><td></td><td>1</td><td>384111310</td><td>9386708808</td><td>0</td><td></td><td>CR_kwDOFuUSzs8AAAACL33PSA</td><td></td><td></td><td>https://github.com/apache/incubator-devlake/actions/runs/3428125760/jobs/5711986448</td><td>COMPLETED</td><td>SUCCESS</td><td>2022-11-09 12:25:46.000000 +00:00</td><td>2022-11-09 12:28:36.000000 +00:00</td><td>test (ubuntu-latest)</td><td>[{&quot;Name&quot;: &quot;Set up job&quot;, &quot;Number&quot;: 1, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:25:46Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;498244a7-bb2d-4a89-b0e6-a02636e00f67&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:25:47Z&quot;, &quot;SecondsToCompl
 etion&quot;: 1}, {&quot;Name&quot;: &quot;Initialize containers&quot;, &quot;Number&quot;: 2, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:25:47Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;82ed782a-533a-443d-b9b3-08973755133f&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:26:03Z&quot;, &quot;SecondsToCompletion&quot;: 16}, {&quot;Name&quot;: &quot;Checkout code&quot;, &quot;Number&quot;: 3, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:26:03Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;dae6463b-ccc6-5762-2432-19367673d22a&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:26:08Z&quot;, &quot;SecondsToCompletion&quot;: 5}, {&quot;Name&quot;: &quot;Setup Golang env&quot;, &quot;Number&quot;: 4, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:26:08Z&quot;, &quot;Conclusion&quot;: &quot;S
 UCCESS&quot;, &quot;ExternalId&quot;: &quot;36275aed-210b-514e-91f2-23c7b347dfa3&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:26:09Z&quot;, &quot;SecondsToCompletion&quot;: 1}, {&quot;Name&quot;: &quot;Cache unit-test&quot;, &quot;Number&quot;: 5, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:26:09Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;f6f7da5e-8567-5f82-1a7e-cf9ca6c0ba8c&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:26:17Z&quot;, &quot;SecondsToCompletion&quot;: 8}, {&quot;Name&quot;: &quot;Unit tests&quot;, &quot;Number&quot;: 6, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:26:17Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;ff0bb0fc-e09e-55fc-4b83-ae5ebb72a959&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:28:32Z&quot;, &quot;SecondsToCompletion&quot;: 135}, {&quot;Name&quot;: &quot;Post Cache unit-tes
 t&quot;, &quot;Number&quot;: 9, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:28:32Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;9785e0d4-de7d-46ff-9d2a-148c27bdf52c&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:28:33Z&quot;, &quot;SecondsToCompletion&quot;: 1}, {&quot;Name&quot;: &quot;Post Setup Golang env&quot;, &quot;Number&quot;: 10, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:28:33Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;73c14928-e3cd-4574-9411-f324bf8d9424&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:28:33Z&quot;, &quot;SecondsToCompletion&quot;: 0}, {&quot;Name&quot;: &quot;Post Checkout code&quot;, &quot;Number&quot;: 11, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:28:33Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;ce92c31
 0-bd83-41cf-8bb8-2de5e9647bdc&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:28:33Z&quot;, &quot;SecondsToCompletion&quot;: 0}, {&quot;Name&quot;: &quot;Stop containers&quot;, &quot;Number&quot;: 12, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:28:34Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;990c54f2-653a-487c-9c26-1025a0b41a86&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:28:35Z&quot;, &quot;SecondsToCompletion&quot;: 1}, {&quot;Name&quot;: &quot;Complete job&quot;, &quot;Number&quot;: 13, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T12:28:35Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;bfea7717-cc32-4594-ae01-d1570b771e47&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T12:28:35Z&quot;, &quot;SecondsToCompletion&quot;: 0}]</td><td></td><td>NULL</td><td>0</td><td></td><td>0</td><td></td></tr>
   <tr><td>2022-11-09 14:08:19.001904 +00:00</td><td>2022-11-09 14:08:19.001904 +00:00</td><td>{&quot;ConnectionId&quot;:1,&quot;Owner&quot;:&quot;apache&quot;,&quot;Repo&quot;:&quot;incubator-devlake&quot;}</td><td>_raw_github_graphql_check_runs</td><td>280</td><td></td><td>1</td><td>384111310</td><td>9385401532</td><td>0</td><td></td><td>CR_kwDOFuUSzs8AAAACL2ncvA</td><td></td><td></td><td>https://github.com/apache/incubator-devlake/actions/runs/3427686139/jobs/5710992069</td><td>COMPLETED</td><td>SUCCESS</td><td>2022-11-09 11:18:09.000000 +00:00</td><td>2022-11-09 11:22:58.000000 +00:00</td><td>e2e-mysql</td><td>[{&quot;Name&quot;: &quot;Set up job&quot;, &quot;Number&quot;: 1, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:18:08Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;13d3eb72-233a-46e8-b033-6dc9db7520c1&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:18:10Z&quot;, &quot;SecondsToCompletion&quot;
 : 2}, {&quot;Name&quot;: &quot;Initialize containers&quot;, &quot;Number&quot;: 2, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:18:10Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;eac729a0-9092-4140-8b19-9398a560b028&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:18:44Z&quot;, &quot;SecondsToCompletion&quot;: 34}, {&quot;Name&quot;: &quot;Run actions/checkout@v3&quot;, &quot;Number&quot;: 3, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:18:44Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;70b9334f-e9db-5c1a-f7db-4250a47e57f2&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:18:48Z&quot;, &quot;SecondsToCompletion&quot;: 4}, {&quot;Name&quot;: &quot;Cache test-e2e&quot;, &quot;Number&quot;: 4, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:18:48Z&quot;, &quot;Conclusion&quot;: &quot;SUCC
 ESS&quot;, &quot;ExternalId&quot;: &quot;36b05d30-1014-5f7d-aad2-c4aee0d7ec30&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:19:02Z&quot;, &quot;SecondsToCompletion&quot;: 14}, {&quot;Name&quot;: &quot;Test&quot;, &quot;Number&quot;: 5, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:19:02Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;18a23b63-b968-5ac0-3668-34310a7a8e14&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:22:53Z&quot;, &quot;SecondsToCompletion&quot;: 231}, {&quot;Name&quot;: &quot;Post Cache test-e2e&quot;, &quot;Number&quot;: 8, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:22:53Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;b84ced8a-4679-4c36-95be-38b77b097724&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:22:54Z&quot;, &quot;SecondsToCompletion&quot;: 1}, {&quot;Name&quot;: &quot;Post Run actions/checko
 ut@v3&quot;, &quot;Number&quot;: 9, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:22:54Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;0741c6c7-c5fd-47c6-8963-beb995598b71&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:22:54Z&quot;, &quot;SecondsToCompletion&quot;: 0}, {&quot;Name&quot;: &quot;Stop containers&quot;, &quot;Number&quot;: 10, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:22:55Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;e08816f4-38a4-4af2-934d-3b55f3290812&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:22:57Z&quot;, &quot;SecondsToCompletion&quot;: 2}, {&quot;Name&quot;: &quot;Complete job&quot;, &quot;Number&quot;: 11, &quot;Status&quot;: &quot;COMPLETED&quot;, &quot;StartedAt&quot;: &quot;2022-11-09T11:22:57Z&quot;, &quot;Conclusion&quot;: &quot;SUCCESS&quot;, &quot;ExternalId&quot;: &quot;f549758f-8e87-4
 de5-b3bd-0e1ac661bb96&quot;, &quot;CompletedAt&quot;: &quot;2022-11-09T11:22:57Z&quot;, &quot;SecondsToCompletion&quot;: 0}]</td><td></td><td>NULL</td><td>0</td><td></td><td>0</td><td></td></tr>
   
   </table>
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018681607


##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,

Review Comment:
   how about 60?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] likyh commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
likyh commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018692543


##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] likyh commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
likyh commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018690860


##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})

Review Comment:
   fixed



##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})
+			query := &GraphqlQueryCheckRunWrapper{}
+			checkSuiteIds := []map[string]interface{}{}
+			for _, iAccount := range accounts {

Review Comment:
   fixed



##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})
+			query := &GraphqlQueryCheckRunWrapper{}
+			checkSuiteIds := []map[string]interface{}{}
+			for _, iAccount := range accounts {
+				workflowRun := iAccount.(*SimpleWorkflowRun)
+				checkSuiteIds = append(checkSuiteIds, map[string]interface{}{
+					`id`: graphql.ID(workflowRun.CheckSuiteNodeID),
+				})
+			}
+			variables := map[string]interface{}{
+				"node": checkSuiteIds,
+			}
+			return query, variables, nil
+		},
+		ResponseParserWithDataErrors: func(iQuery interface{}, variables map[string]interface{}, dataErrors []graphql.DataError) ([]interface{}, error) {
+			for _, dataError := range dataErrors {
+				// log and ignore
+				taskCtx.GetLogger().Warn(dataError, `query check run get error but ignore`)
+			}
+			query := iQuery.(*GraphqlQueryCheckRunWrapper)
+			nodes := query.Node
+
+			results := make([]interface{}, 0, 1)
+			for _, node := range nodes {
+				for _, checkRun := range node.CheckSuite.CheckRuns.Nodes {
+
+					paramsBytes, err := json.Marshal(checkRun.Steps.Nodes)
+					if err != nil {
+						logger.Error(err, `Marshal checkRun.Steps.Nodes fail and ignore`)
+					}
+					githubJob := &models.GithubJob{
+						ConnectionId: data.Options.ConnectionId,
+						RepoId:       data.Repo.GithubId,
+						ID:           checkRun.DatabaseId,
+						NodeID:       checkRun.Id,
+						HTMLURL:      checkRun.DetailsUrl,
+						Status:       checkRun.Status,
+						Conclusion:   checkRun.Conclusion,
+						StartedAt:    checkRun.StartedAt,
+						CompletedAt:  checkRun.CompletedAt,
+						Name:         checkRun.Name,
+						Steps:        paramsBytes,
+						// these columns can not fill by graphql
+						//RunID:         ``,  // use _tool_github_runs

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 merged pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
warren830 merged PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] likyh commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
likyh commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018692281


##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})
+			query := &GraphqlQueryCheckRunWrapper{}
+			checkSuiteIds := []map[string]interface{}{}
+			for _, iAccount := range accounts {
+				workflowRun := iAccount.(*SimpleWorkflowRun)
+				checkSuiteIds = append(checkSuiteIds, map[string]interface{}{
+					`id`: graphql.ID(workflowRun.CheckSuiteNodeID),
+				})
+			}
+			variables := map[string]interface{}{
+				"node": checkSuiteIds,
+			}
+			return query, variables, nil
+		},
+		ResponseParserWithDataErrors: func(iQuery interface{}, variables map[string]interface{}, dataErrors []graphql.DataError) ([]interface{}, error) {
+			for _, dataError := range dataErrors {
+				// log and ignore
+				taskCtx.GetLogger().Warn(dataError, `query check run get error but ignore`)
+			}
+			query := iQuery.(*GraphqlQueryCheckRunWrapper)
+			nodes := query.Node
+
+			results := make([]interface{}, 0, 1)
+			for _, node := range nodes {
+				for _, checkRun := range node.CheckSuite.CheckRuns.Nodes {
+
+					paramsBytes, err := json.Marshal(checkRun.Steps.Nodes)
+					if err != nil {
+						logger.Error(err, `Marshal checkRun.Steps.Nodes fail and ignore`)
+					}
+					githubJob := &models.GithubJob{
+						ConnectionId: data.Options.ConnectionId,
+						RepoId:       data.Repo.GithubId,
+						ID:           checkRun.DatabaseId,
+						NodeID:       checkRun.Id,
+						HTMLURL:      checkRun.DetailsUrl,
+						Status:       checkRun.Status,
+						Conclusion:   checkRun.Conclusion,
+						StartedAt:    checkRun.StartedAt,
+						CompletedAt:  checkRun.CompletedAt,
+						Name:         checkRun.Name,
+						Steps:        paramsBytes,
+						// these columns can not fill by graphql
+						//RunID:         ``,  // use _tool_github_runs
+						//HeadSha:       ``,  // use _tool_github_runs
+						//RunURL:        ``,
+						//CheckRunURL:   ``,
+						//Labels:        ``, // not on use
+						//RunnerID:      ``, // not on use
+						//RunnerName:    ``, // not on use
+						//RunnerGroupID: ``, // not on use

Review Comment:
   that is to tell other developers where we can fill runner info if they want.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018680203


##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})
+			query := &GraphqlQueryCheckRunWrapper{}
+			checkSuiteIds := []map[string]interface{}{}
+			for _, iAccount := range accounts {
+				workflowRun := iAccount.(*SimpleWorkflowRun)
+				checkSuiteIds = append(checkSuiteIds, map[string]interface{}{
+					`id`: graphql.ID(workflowRun.CheckSuiteNodeID),
+				})
+			}
+			variables := map[string]interface{}{
+				"node": checkSuiteIds,
+			}
+			return query, variables, nil
+		},
+		ResponseParserWithDataErrors: func(iQuery interface{}, variables map[string]interface{}, dataErrors []graphql.DataError) ([]interface{}, error) {
+			for _, dataError := range dataErrors {
+				// log and ignore
+				taskCtx.GetLogger().Warn(dataError, `query check run get error but ignore`)
+			}
+			query := iQuery.(*GraphqlQueryCheckRunWrapper)
+			nodes := query.Node
+
+			results := make([]interface{}, 0, 1)
+			for _, node := range nodes {
+				for _, checkRun := range node.CheckSuite.CheckRuns.Nodes {
+
+					paramsBytes, err := json.Marshal(checkRun.Steps.Nodes)
+					if err != nil {
+						logger.Error(err, `Marshal checkRun.Steps.Nodes fail and ignore`)
+					}
+					githubJob := &models.GithubJob{
+						ConnectionId: data.Options.ConnectionId,
+						RepoId:       data.Repo.GithubId,
+						ID:           checkRun.DatabaseId,
+						NodeID:       checkRun.Id,
+						HTMLURL:      checkRun.DetailsUrl,
+						Status:       checkRun.Status,
+						Conclusion:   checkRun.Conclusion,
+						StartedAt:    checkRun.StartedAt,
+						CompletedAt:  checkRun.CompletedAt,
+						Name:         checkRun.Name,
+						Steps:        paramsBytes,
+						// these columns can not fill by graphql
+						//RunID:         ``,  // use _tool_github_runs
+						//HeadSha:       ``,  // use _tool_github_runs
+						//RunURL:        ``,
+						//CheckRunURL:   ``,
+						//Labels:        ``, // not on use
+						//RunnerID:      ``, // not on use
+						//RunnerName:    ``, // not on use
+						//RunnerGroupID: ``, // not on use

Review Comment:
   please delete these unused code



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018679935


##########
plugins/github_graphql/plugin_main.go:
##########
@@ -75,6 +80,7 @@ func (plugin GithubGraphql) SubTaskMetas() []core.SubTaskMeta {
 
 		tasks.CollectAccountMeta,
 

Review Comment:
   please also delete these blank lines



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] warren830 commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
warren830 commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018677666


##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})

Review Comment:
   should this be workflowRun?



##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})
+			query := &GraphqlQueryCheckRunWrapper{}
+			checkSuiteIds := []map[string]interface{}{}
+			for _, iAccount := range accounts {
+				workflowRun := iAccount.(*SimpleWorkflowRun)
+				checkSuiteIds = append(checkSuiteIds, map[string]interface{}{
+					`id`: graphql.ID(workflowRun.CheckSuiteNodeID),
+				})
+			}
+			variables := map[string]interface{}{
+				"node": checkSuiteIds,
+			}
+			return query, variables, nil
+		},
+		ResponseParserWithDataErrors: func(iQuery interface{}, variables map[string]interface{}, dataErrors []graphql.DataError) ([]interface{}, error) {
+			for _, dataError := range dataErrors {
+				// log and ignore
+				taskCtx.GetLogger().Warn(dataError, `query check run get error but ignore`)
+			}
+			query := iQuery.(*GraphqlQueryCheckRunWrapper)
+			nodes := query.Node
+
+			results := make([]interface{}, 0, 1)
+			for _, node := range nodes {
+				for _, checkRun := range node.CheckSuite.CheckRuns.Nodes {
+
+					paramsBytes, err := json.Marshal(checkRun.Steps.Nodes)
+					if err != nil {
+						logger.Error(err, `Marshal checkRun.Steps.Nodes fail and ignore`)
+					}
+					githubJob := &models.GithubJob{
+						ConnectionId: data.Options.ConnectionId,
+						RepoId:       data.Repo.GithubId,
+						ID:           checkRun.DatabaseId,
+						NodeID:       checkRun.Id,
+						HTMLURL:      checkRun.DetailsUrl,
+						Status:       checkRun.Status,
+						Conclusion:   checkRun.Conclusion,
+						StartedAt:    checkRun.StartedAt,
+						CompletedAt:  checkRun.CompletedAt,
+						Name:         checkRun.Name,
+						Steps:        paramsBytes,
+						// these columns can not fill by graphql
+						//RunID:         ``,  // use _tool_github_runs

Review Comment:
   I think we also need to set RunId as this will be used to connect to pipelines



##########
plugins/github_graphql/tasks/check_run_collector.go:
##########
@@ -0,0 +1,190 @@
+/*
+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/errors"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/github/models"
+	githubTasks "github.com/apache/incubator-devlake/plugins/github/tasks"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/merico-dev/graphql"
+	"reflect"
+	"time"
+)
+
+const RAW_CHECK_RUNS_TABLE = "github_graphql_check_runs"
+
+type GraphqlQueryCheckRunWrapper struct {
+	RateLimit struct {
+		Cost int
+	}
+	Node []GraphqlQueryCheckSuite `graphql:"node(id: $id)" graphql-extend:"true"`
+}
+
+type GraphqlQueryCheckSuite struct {
+	Id         string
+	Typename   string `graphql:"__typename"`
+	CheckSuite struct {
+		CheckRuns struct {
+			TotalCount int
+			Nodes      []struct {
+				Id          string
+				Name        string
+				DetailsUrl  string
+				DatabaseId  int
+				Status      string
+				StartedAt   *time.Time
+				Conclusion  string
+				CompletedAt *time.Time
+				//ExternalId   string
+				//Url          string
+				//Title        interface{}
+				//Text         interface{}
+				//Summary      interface{}
+
+				Steps struct {
+					TotalCount int
+					Nodes      []struct {
+						CompletedAt         *time.Time
+						Conclusion          string
+						ExternalId          string
+						Name                string
+						Number              int
+						SecondsToCompletion int
+						StartedAt           *time.Time
+						Status              string
+					}
+				} `graphql:"steps(first: 50)"`
+			}
+		} `graphql:"checkRuns(first: 50)"`
+	} `graphql:"... on CheckSuite"`
+}
+
+type SimpleWorkflowRun struct {
+	CheckSuiteNodeID string
+}
+
+var CollectCheckRunMeta = core.SubTaskMeta{
+	Name:             "CollectCheckRun",
+	EntryPoint:       CollectCheckRun,
+	EnabledByDefault: true,
+	Description:      "Collect CheckRun data from GithubGraphql api",
+}
+
+var _ core.SubTaskEntryPoint = CollectAccount
+
+func CollectCheckRun(taskCtx core.SubTaskContext) errors.Error {
+	logger := taskCtx.GetLogger()
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*githubTasks.GithubTaskData)
+
+	cursor, err := db.Cursor(
+		dal.Select("check_suite_node_id"),
+		dal.From(models.GithubRun{}.TableName()),
+		dal.Where("repo_id = ? and connection_id=?", data.Repo.GithubId, data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	iterator, err := helper.NewDalCursorIterator(db, cursor, reflect.TypeOf(SimpleWorkflowRun{}))
+	if err != nil {
+		return err
+	}
+
+	collector, err := helper.NewGraphqlCollector(helper.GraphqlCollectorArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: githubTasks.GithubApiParams{
+				ConnectionId: data.Options.ConnectionId,
+				Owner:        data.Options.Owner,
+				Repo:         data.Options.Repo,
+			},
+			Table: RAW_CHECK_RUNS_TABLE,
+		},
+		Input:         iterator,
+		InputStep:     30,
+		GraphqlClient: data.GraphqlClient,
+		BuildQuery: func(reqData *helper.GraphqlRequestData) (interface{}, map[string]interface{}, error) {
+			accounts := reqData.Input.([]interface{})
+			query := &GraphqlQueryCheckRunWrapper{}
+			checkSuiteIds := []map[string]interface{}{}
+			for _, iAccount := range accounts {

Review Comment:
   same as above



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] likyh commented on a diff in pull request #3705: feat: add check_run collector in github graphql

Posted by GitBox <gi...@apache.org>.
likyh commented on code in PR #3705:
URL: https://github.com/apache/incubator-devlake/pull/3705#discussion_r1018691628


##########
plugins/github_graphql/plugin_main.go:
##########
@@ -75,6 +80,7 @@ func (plugin GithubGraphql) SubTaskMetas() []core.SubTaskMeta {
 
 		tasks.CollectAccountMeta,
 

Review Comment:
   add some comments here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org