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/01 21:00:32 UTC

[GitHub] [incubator-devlake] keon94 opened a new pull request, #3641: [feat-3498]: Pagerduty plugin implementation

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

   ### ⚠️ Pre Checklist
   
   > Please complete _ALL_ items in this checklist, and remove before submitting
   
   - [ ] I have read through the [Contributing Documentation](https://devlake.apache.org/community/).
   - [ ] I have added relevant tests.
   - [ ] I have added relevant documentation.
   - [ ] I will add labels to the PR, such as `pr-type/bug-fix`, `pr-type/feature-development`, etc.
   
   
   
   # 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.
   -->
   
   ### Does this close any open issues?
   Closes xx
   
   ### Screenshots
   Include any relevant screenshots here.
   
   ### Other Information
   Any other information that is important to this PR.
   


-- 
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 merged pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


-- 
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 #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/models/service.go:
##########
@@ -0,0 +1,32 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import "github.com/apache/incubator-devlake/models/common"
+
+type Service struct {
+	common.NoPKModel
+	ConnectionId uint64 `gorm:"primaryKey"`
+	Url          string
+	Id           string `gorm:"primaryKey"`
+	Name         string
+}
+
+func (Service) TableName() string {
+	return "_tool_pagerduty_service"

Review Comment:
   use plural `_tool_pagerduty_services` as others



-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/tasks/incidents_converter.go:
##########
@@ -0,0 +1,159 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/common"
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/pagerduty/models"
+	"reflect"
+	"time"
+)
+
+var ConvertIncidentsMeta = core.SubTaskMeta{
+	Name:             "convertIncidents",
+	EntryPoint:       ConvertIncidents,
+	EnabledByDefault: true,
+	Description:      "Convert incidents into domain layer table issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type (
+	// IncidentWithUser struct that represents the joined query result
+	IncidentWithUser struct {
+		common.NoPKModel
+		*models.Incident
+		*models.User
+		AssignedAt time.Time
+	}
+)
+
+func ConvertIncidents(taskCtx core.SubTaskContext) errors.Error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*PagerDutyTaskData)
+	cursor, err := db.Cursor(
+		dal.Select("pi.*, pu.*, pa.assigned_at"),
+		dal.From("_tool_pagerduty_incident AS pi"),
+		dal.Join(`LEFT JOIN _tool_pagerduty_assignment AS pa ON pa.incident_number = pi.number`),
+		dal.Join(`LEFT JOIN _tool_pagerduty_user AS pu ON pa.user_id = pu.id`),
+		dal.Where("pi.connection_id = ?", data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	seenIncidents := map[int]*IncidentWithUser{}
+
+	idGen := didgen.NewDomainIdGenerator(&models.Incident{})
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: models.PagerDutyParams{
+				ConnectionId: data.Options.ConnectionId,
+				Stream:       models.IncidentStream,
+			},
+			Table: RAW_INCIDENTS_TABLE,
+		},
+		InputRowType: reflect.TypeOf(IncidentWithUser{}),
+		Input:        cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			combined := inputRow.(*IncidentWithUser)
+			incident := combined.Incident
+			user := combined.User
+			if seen, ok := seenIncidents[incident.Number]; ok {
+				if combined.AssignedAt.Before(seen.AssignedAt) {
+					// skip this one (it's an older assignee)
+					return nil, nil
+				}
+			}
+			status := getStatus(incident)
+			leadTime, resolutionDate := getTimes(incident)
+			domainIssue := &ticket.Issue{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: idGen.Generate(data.Options.ConnectionId, incident.Number),
+				},
+				Url:                     incident.Url,
+				IconURL:                 "",
+				IssueKey:                fmt.Sprintf("%d", incident.Number),
+				Title:                   "",
+				Description:             incident.Summary,
+				EpicKey:                 "",
+				Type:                    ticket.INCIDENT,
+				Status:                  status,
+				OriginalStatus:          string(incident.Status),
+				StoryPoint:              0,
+				ResolutionDate:          resolutionDate,
+				CreatedDate:             &incident.CreatedDate,
+				UpdatedDate:             &incident.UpdatedDate,
+				LeadTimeMinutes:         leadTime,
+				ParentIssueId:           "",
+				Priority:                string(incident.Urgency),
+				OriginalEstimateMinutes: 0,
+				TimeSpentMinutes:        0,
+				TimeRemainingMinutes:    0,
+				CreatorId:               "",
+				CreatorName:             "",
+				AssigneeId:              user.Id,
+				AssigneeName:            user.Name,
+				Severity:                "",
+				Component:               "",
+				DeploymentId:            incident.ServiceId,

Review Comment:
   Here're the fields on a service:
   ```json
   {
     "services": [
       {
         "id": "PIKL83L",
         "name": "DevService",
         "description": "Service for dev testing.",
         "created_at": "2022-11-03T01:21:07-05:00",
         "updated_at": "2022-11-03T01:22:15-05:00",
         "status": "critical",
         "teams": [],
         "alert_creation": "create_alerts_and_incidents",
         "addons": [],
         "scheduled_actions": [],
         "support_hours": null,
         "last_incident_timestamp": "2022-11-03T06:45:36Z",
         "escalation_policy": {
           "id": "PNJQLBU",
           "type": "escalation_policy_reference",
           "summary": "Default",
           "self": "https://api.pagerduty.com/escalation_policies/PNJQLBU",
           "html_url": "https://keon-test.pagerduty.com/escalation_policies/PNJQLBU"
         },
         "incident_urgency_rule": {
           "type": "constant",
           "urgency": "high"
         },
         "acknowledgement_timeout": null,
         "auto_resolve_timeout": null,
         "integrations": [],
         "type": "service",
         "summary": "DevService",
         "self": "https://api.pagerduty.com/services/PIKL83L",
         "html_url": "https://keon-test.pagerduty.com/service-directory/PIKL83L"
       }
     ],
     "limit": 25,
     "offset": 0,
     "total": null,
     "more": false
   }
   ```



-- 
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] keon94 commented on pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

Posted by GitBox <gi...@apache.org>.
keon94 commented on PR #3641:
URL: https://github.com/apache/incubator-devlake/pull/3641#issuecomment-1305114837

   ![image](https://user-images.githubusercontent.com/25063936/200235089-fe09e784-d314-4c43-bf2d-c30a0f9b11de.png)
   


-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/tasks/incidents_converter.go:
##########
@@ -0,0 +1,159 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/common"
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/pagerduty/models"
+	"reflect"
+	"time"
+)
+
+var ConvertIncidentsMeta = core.SubTaskMeta{
+	Name:             "convertIncidents",
+	EntryPoint:       ConvertIncidents,
+	EnabledByDefault: true,
+	Description:      "Convert incidents into domain layer table issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type (
+	// IncidentWithUser struct that represents the joined query result
+	IncidentWithUser struct {
+		common.NoPKModel
+		*models.Incident
+		*models.User
+		AssignedAt time.Time
+	}
+)
+
+func ConvertIncidents(taskCtx core.SubTaskContext) errors.Error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*PagerDutyTaskData)
+	cursor, err := db.Cursor(
+		dal.Select("pi.*, pu.*, pa.assigned_at"),
+		dal.From("_tool_pagerduty_incident AS pi"),
+		dal.Join(`LEFT JOIN _tool_pagerduty_assignment AS pa ON pa.incident_number = pi.number`),
+		dal.Join(`LEFT JOIN _tool_pagerduty_user AS pu ON pa.user_id = pu.id`),
+		dal.Where("pi.connection_id = ?", data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	seenIncidents := map[int]*IncidentWithUser{}
+
+	idGen := didgen.NewDomainIdGenerator(&models.Incident{})
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: models.PagerDutyParams{
+				ConnectionId: data.Options.ConnectionId,
+				Stream:       models.IncidentStream,
+			},
+			Table: RAW_INCIDENTS_TABLE,
+		},
+		InputRowType: reflect.TypeOf(IncidentWithUser{}),
+		Input:        cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			combined := inputRow.(*IncidentWithUser)
+			incident := combined.Incident
+			user := combined.User
+			if seen, ok := seenIncidents[incident.Number]; ok {
+				if combined.AssignedAt.Before(seen.AssignedAt) {
+					// skip this one (it's an older assignee)
+					return nil, nil
+				}
+			}
+			status := getStatus(incident)
+			leadTime, resolutionDate := getTimes(incident)
+			domainIssue := &ticket.Issue{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: idGen.Generate(data.Options.ConnectionId, incident.Number),
+				},
+				Url:                     incident.Url,
+				IconURL:                 "",
+				IssueKey:                fmt.Sprintf("%d", incident.Number),
+				Title:                   "",
+				Description:             incident.Summary,
+				EpicKey:                 "",
+				Type:                    ticket.INCIDENT,
+				Status:                  status,
+				OriginalStatus:          string(incident.Status),
+				StoryPoint:              0,
+				ResolutionDate:          resolutionDate,
+				CreatedDate:             &incident.CreatedDate,
+				UpdatedDate:             &incident.UpdatedDate,
+				LeadTimeMinutes:         leadTime,
+				ParentIssueId:           "",
+				Priority:                string(incident.Urgency),
+				OriginalEstimateMinutes: 0,
+				TimeSpentMinutes:        0,
+				TimeRemainingMinutes:    0,
+				CreatorId:               "",
+				CreatorName:             "",
+				AssigneeId:              user.Id,
+				AssigneeName:            user.Name,
+				Severity:                "",
+				Component:               "",
+				DeploymentId:            incident.ServiceId,

Review Comment:
   not sure if this is right. We're not creating any cicd tasks with pagerduty, so deployment-Id doesn't make sense in the usual sense... incident.ServiceId only refers to the pagerduty service that caused the incident, it's not a deployment of any sort. Let me know your thoughts



-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/tasks/incidents_converter.go:
##########
@@ -0,0 +1,159 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/common"
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/pagerduty/models"
+	"reflect"
+	"time"
+)
+
+var ConvertIncidentsMeta = core.SubTaskMeta{
+	Name:             "convertIncidents",
+	EntryPoint:       ConvertIncidents,
+	EnabledByDefault: true,
+	Description:      "Convert incidents into domain layer table issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type (
+	// IncidentWithUser struct that represents the joined query result
+	IncidentWithUser struct {
+		common.NoPKModel
+		*models.Incident
+		*models.User
+		AssignedAt time.Time
+	}
+)
+
+func ConvertIncidents(taskCtx core.SubTaskContext) errors.Error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*PagerDutyTaskData)
+	cursor, err := db.Cursor(
+		dal.Select("pi.*, pu.*, pa.assigned_at"),
+		dal.From("_tool_pagerduty_incident AS pi"),
+		dal.Join(`LEFT JOIN _tool_pagerduty_assignment AS pa ON pa.incident_number = pi.number`),
+		dal.Join(`LEFT JOIN _tool_pagerduty_user AS pu ON pa.user_id = pu.id`),
+		dal.Where("pi.connection_id = ?", data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	seenIncidents := map[int]*IncidentWithUser{}
+
+	idGen := didgen.NewDomainIdGenerator(&models.Incident{})
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: models.PagerDutyParams{
+				ConnectionId: data.Options.ConnectionId,
+				Stream:       models.IncidentStream,
+			},
+			Table: RAW_INCIDENTS_TABLE,
+		},
+		InputRowType: reflect.TypeOf(IncidentWithUser{}),
+		Input:        cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			combined := inputRow.(*IncidentWithUser)
+			incident := combined.Incident
+			user := combined.User
+			if seen, ok := seenIncidents[incident.Number]; ok {
+				if combined.AssignedAt.Before(seen.AssignedAt) {
+					// skip this one (it's an older assignee)
+					return nil, nil
+				}
+			}
+			status := getStatus(incident)
+			leadTime, resolutionDate := getTimes(incident)
+			domainIssue := &ticket.Issue{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: idGen.Generate(data.Options.ConnectionId, incident.Number),
+				},
+				Url:                     incident.Url,
+				IconURL:                 "",
+				IssueKey:                fmt.Sprintf("%d", incident.Number),
+				Title:                   "",
+				Description:             incident.Summary,
+				EpicKey:                 "",
+				Type:                    ticket.INCIDENT,
+				Status:                  status,
+				OriginalStatus:          string(incident.Status),
+				StoryPoint:              0,
+				ResolutionDate:          resolutionDate,
+				CreatedDate:             &incident.CreatedDate,
+				UpdatedDate:             &incident.UpdatedDate,
+				LeadTimeMinutes:         leadTime,
+				ParentIssueId:           "",
+				Priority:                string(incident.Urgency),
+				OriginalEstimateMinutes: 0,
+				TimeSpentMinutes:        0,
+				TimeRemainingMinutes:    0,
+				CreatorId:               "",
+				CreatorName:             "",
+				AssigneeId:              user.Id,
+				AssigneeName:            user.Name,
+				Severity:                "",
+				Component:               "",
+				DeploymentId:            incident.ServiceId,

Review Comment:
   I spoke to @hezyin today and we agreed the DeploymentId is not needed to be set here because the Dora plugin is the one that does that. I've made that change.



-- 
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 #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
config/tap/pagerduty.json:
##########
@@ -0,0 +1,1884 @@
+{

Review Comment:
   But maybe in the future, plugins can install and uninstall dynamic. putting all file in one path will be easy to manage.



-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/tasks/incidents_converter.go:
##########
@@ -0,0 +1,159 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/common"
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/pagerduty/models"
+	"reflect"
+	"time"
+)
+
+var ConvertIncidentsMeta = core.SubTaskMeta{
+	Name:             "convertIncidents",
+	EntryPoint:       ConvertIncidents,
+	EnabledByDefault: true,
+	Description:      "Convert incidents into domain layer table issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type (
+	// IncidentWithUser struct that represents the joined query result
+	IncidentWithUser struct {
+		common.NoPKModel
+		*models.Incident
+		*models.User
+		AssignedAt time.Time
+	}
+)
+
+func ConvertIncidents(taskCtx core.SubTaskContext) errors.Error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*PagerDutyTaskData)
+	cursor, err := db.Cursor(
+		dal.Select("pi.*, pu.*, pa.assigned_at"),
+		dal.From("_tool_pagerduty_incident AS pi"),
+		dal.Join(`LEFT JOIN _tool_pagerduty_assignment AS pa ON pa.incident_number = pi.number`),
+		dal.Join(`LEFT JOIN _tool_pagerduty_user AS pu ON pa.user_id = pu.id`),
+		dal.Where("pi.connection_id = ?", data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	seenIncidents := map[int]*IncidentWithUser{}
+
+	idGen := didgen.NewDomainIdGenerator(&models.Incident{})
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: models.PagerDutyParams{
+				ConnectionId: data.Options.ConnectionId,
+				Stream:       models.IncidentStream,
+			},
+			Table: RAW_INCIDENTS_TABLE,
+		},
+		InputRowType: reflect.TypeOf(IncidentWithUser{}),
+		Input:        cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			combined := inputRow.(*IncidentWithUser)
+			incident := combined.Incident
+			user := combined.User
+			if seen, ok := seenIncidents[incident.Number]; ok {
+				if combined.AssignedAt.Before(seen.AssignedAt) {
+					// skip this one (it's an older assignee)
+					return nil, nil
+				}
+			}
+			status := getStatus(incident)
+			leadTime, resolutionDate := getTimes(incident)
+			domainIssue := &ticket.Issue{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: idGen.Generate(data.Options.ConnectionId, incident.Number),
+				},
+				Url:                     incident.Url,
+				IconURL:                 "",
+				IssueKey:                fmt.Sprintf("%d", incident.Number),
+				Title:                   "",
+				Description:             incident.Summary,
+				EpicKey:                 "",
+				Type:                    ticket.INCIDENT,
+				Status:                  status,
+				OriginalStatus:          string(incident.Status),
+				StoryPoint:              0,
+				ResolutionDate:          resolutionDate,
+				CreatedDate:             &incident.CreatedDate,
+				UpdatedDate:             &incident.UpdatedDate,
+				LeadTimeMinutes:         leadTime,
+				ParentIssueId:           "",
+				Priority:                string(incident.Urgency),
+				OriginalEstimateMinutes: 0,
+				TimeSpentMinutes:        0,
+				TimeRemainingMinutes:    0,
+				CreatorId:               "",
+				CreatorName:             "",
+				AssigneeId:              user.Id,
+				AssigneeName:            user.Name,
+				Severity:                "",
+				Component:               "",
+				DeploymentId:            incident.ServiceId,

Review Comment:
   Here're the fields on a service [API doc](https://developer.pagerduty.com/api-reference/e960cca205c0f-list-services):
   ```json
   {
     "services": [
       {
         "id": "PIKL83L",
         "name": "DevService",
         "description": "Service for dev testing.",
         "created_at": "2022-11-03T01:21:07-05:00",
         "updated_at": "2022-11-03T01:22:15-05:00",
         "status": "critical",
         "teams": [],
         "alert_creation": "create_alerts_and_incidents",
         "addons": [],
         "scheduled_actions": [],
         "support_hours": null,
         "last_incident_timestamp": "2022-11-03T06:45:36Z",
         "escalation_policy": {
           "id": "PNJQLBU",
           "type": "escalation_policy_reference",
           "summary": "Default",
           "self": "https://api.pagerduty.com/escalation_policies/PNJQLBU",
           "html_url": "https://keon-test.pagerduty.com/escalation_policies/PNJQLBU"
         },
         "incident_urgency_rule": {
           "type": "constant",
           "urgency": "high"
         },
         "acknowledgement_timeout": null,
         "auto_resolve_timeout": null,
         "integrations": [],
         "type": "service",
         "summary": "DevService",
         "self": "https://api.pagerduty.com/services/PIKL83L",
         "html_url": "https://keon-test.pagerduty.com/service-directory/PIKL83L"
       }
     ],
     "limit": 25,
     "offset": 0,
     "total": null,
     "more": false
   }
   ```



-- 
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 #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
config/tap/pagerduty.json:
##########
@@ -0,0 +1,1884 @@
+{

Review Comment:
   should this file move into path `plugins/pagerduty`?



-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/models/service.go:
##########
@@ -0,0 +1,32 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import "github.com/apache/incubator-devlake/models/common"
+
+type Service struct {
+	common.NoPKModel
+	ConnectionId uint64 `gorm:"primaryKey"`
+	Url          string
+	Id           string `gorm:"primaryKey"`
+	Name         string
+}
+
+func (Service) TableName() string {
+	return "_tool_pagerduty_service"

Review Comment:
   done



-- 
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] keon94 commented on pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

Posted by GitBox <gi...@apache.org>.
keon94 commented on PR #3641:
URL: https://github.com/apache/incubator-devlake/pull/3641#issuecomment-1304397105

   Sample pipeline-run:
   ![image](https://user-images.githubusercontent.com/25063936/200100156-18477be0-b1c7-4751-aff7-895bc7480d72.png)
   


-- 
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


Re: [PR] [feat-3498][backend] Pagerduty plugin implementation [incubator-devlake]

Posted by "klesh (via GitHub)" <gi...@apache.org>.
klesh commented on PR #3641:
URL: https://github.com/apache/incubator-devlake/pull/3641#issuecomment-1931029211

   @barrykaplan I'm afraid so, you may copy the plugin folder and modify it according which should make your work a lot easier. The complicated parts (authentication, fetching data in parallel without exceeding rate limit, etc) are done by `helpers`, but the Model definitions, API detail, and e2e tests must be tailored according to the data source specification.


-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
config/tap/pagerduty.json:
##########
@@ -0,0 +1,1884 @@
+{

Review Comment:
   Well, it's kind of a config file, so putting it under config/ made sense to me. I'm thinking config/tap can be where we put all these sorts of config files so they're all in one place 



-- 
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] keon94 commented on a diff in pull request #3641: [feat-3498][backend] Pagerduty plugin implementation

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


##########
plugins/pagerduty/tasks/incidents_converter.go:
##########
@@ -0,0 +1,159 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package tasks
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/errors"
+	"github.com/apache/incubator-devlake/models/common"
+	"github.com/apache/incubator-devlake/models/domainlayer"
+	"github.com/apache/incubator-devlake/models/domainlayer/didgen"
+	"github.com/apache/incubator-devlake/models/domainlayer/ticket"
+	"github.com/apache/incubator-devlake/plugins/core"
+	"github.com/apache/incubator-devlake/plugins/core/dal"
+	"github.com/apache/incubator-devlake/plugins/helper"
+	"github.com/apache/incubator-devlake/plugins/pagerduty/models"
+	"reflect"
+	"time"
+)
+
+var ConvertIncidentsMeta = core.SubTaskMeta{
+	Name:             "convertIncidents",
+	EntryPoint:       ConvertIncidents,
+	EnabledByDefault: true,
+	Description:      "Convert incidents into domain layer table issues",
+	DomainTypes:      []string{core.DOMAIN_TYPE_TICKET},
+}
+
+type (
+	// IncidentWithUser struct that represents the joined query result
+	IncidentWithUser struct {
+		common.NoPKModel
+		*models.Incident
+		*models.User
+		AssignedAt time.Time
+	}
+)
+
+func ConvertIncidents(taskCtx core.SubTaskContext) errors.Error {
+	db := taskCtx.GetDal()
+	data := taskCtx.GetData().(*PagerDutyTaskData)
+	cursor, err := db.Cursor(
+		dal.Select("pi.*, pu.*, pa.assigned_at"),
+		dal.From("_tool_pagerduty_incident AS pi"),
+		dal.Join(`LEFT JOIN _tool_pagerduty_assignment AS pa ON pa.incident_number = pi.number`),
+		dal.Join(`LEFT JOIN _tool_pagerduty_user AS pu ON pa.user_id = pu.id`),
+		dal.Where("pi.connection_id = ?", data.Options.ConnectionId),
+	)
+	if err != nil {
+		return err
+	}
+	defer cursor.Close()
+
+	seenIncidents := map[int]*IncidentWithUser{}
+
+	idGen := didgen.NewDomainIdGenerator(&models.Incident{})
+	converter, err := helper.NewDataConverter(helper.DataConverterArgs{
+		RawDataSubTaskArgs: helper.RawDataSubTaskArgs{
+			Ctx: taskCtx,
+			Params: models.PagerDutyParams{
+				ConnectionId: data.Options.ConnectionId,
+				Stream:       models.IncidentStream,
+			},
+			Table: RAW_INCIDENTS_TABLE,
+		},
+		InputRowType: reflect.TypeOf(IncidentWithUser{}),
+		Input:        cursor,
+		Convert: func(inputRow interface{}) ([]interface{}, errors.Error) {
+			combined := inputRow.(*IncidentWithUser)
+			incident := combined.Incident
+			user := combined.User
+			if seen, ok := seenIncidents[incident.Number]; ok {
+				if combined.AssignedAt.Before(seen.AssignedAt) {
+					// skip this one (it's an older assignee)
+					return nil, nil
+				}
+			}
+			status := getStatus(incident)
+			leadTime, resolutionDate := getTimes(incident)
+			domainIssue := &ticket.Issue{
+				DomainEntity: domainlayer.DomainEntity{
+					Id: idGen.Generate(data.Options.ConnectionId, incident.Number),
+				},
+				Url:                     incident.Url,
+				IconURL:                 "",
+				IssueKey:                fmt.Sprintf("%d", incident.Number),
+				Title:                   "",
+				Description:             incident.Summary,
+				EpicKey:                 "",
+				Type:                    ticket.INCIDENT,
+				Status:                  status,
+				OriginalStatus:          string(incident.Status),
+				StoryPoint:              0,
+				ResolutionDate:          resolutionDate,
+				CreatedDate:             &incident.CreatedDate,
+				UpdatedDate:             &incident.UpdatedDate,
+				LeadTimeMinutes:         leadTime,
+				ParentIssueId:           "",
+				Priority:                string(incident.Urgency),
+				OriginalEstimateMinutes: 0,
+				TimeSpentMinutes:        0,
+				TimeRemainingMinutes:    0,
+				CreatorId:               "",
+				CreatorName:             "",
+				AssigneeId:              user.Id,
+				AssigneeName:            user.Name,
+				Severity:                "",
+				Component:               "",
+				DeploymentId:            incident.ServiceId,

Review Comment:
   Should I try to map this to a CICDTask somehow? cc @Startrekzky 



-- 
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


Re: [PR] [feat-3498][backend] Pagerduty plugin implementation [incubator-devlake]

Posted by "barrykaplan (via GitHub)" <gi...@apache.org>.
barrykaplan commented on PR #3641:
URL: https://github.com/apache/incubator-devlake/pull/3641#issuecomment-1930372750

   We would like to replicate for grafana cloud incidents. Looking the PR there were a lot of changes (47 files?). Would I expect it to a 
   similar amount of work for adding grafana, or was a lot of that base capability. (I'll look, but I'm new to the devlake so it will take some work to orient.)


-- 
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