feat: multi-workflow runs listener on a single endpoint

* new api-contract for workflow run events

* feat: initial implementation for new subscribe listener

* fix: sync issues and send workflow runs immediately

* refactor: add context to all engine db queries, fix deadlocking query

* fix: use new ctx for deleting dispatcher and ticker

* add cancellation reasons

* fix: docs linting

---------

Co-authored-by: gabriel ruttner <gabriel.ruttner@gmail.com>
This commit is contained in:
abelanger5
2024-04-18 20:55:11 -04:00
committed by GitHub
parent 5216a6c31b
commit 4ce1dd8632
67 changed files with 1864 additions and 908 deletions

View File

@@ -18,6 +18,8 @@ service Dispatcher {
rpc SubscribeToWorkflowEvents(SubscribeToWorkflowEventsRequest) returns (stream WorkflowEvent) {}
rpc SubscribeToWorkflowRuns(stream SubscribeToWorkflowRunsRequest) returns (stream WorkflowRunEvent) {}
rpc SendStepActionEvent(StepActionEvent) returns (ActionEventResponse) {}
rpc SendGroupKeyActionEvent(GroupKeyActionEvent) returns (ActionEventResponse) {}
@@ -190,6 +192,11 @@ message SubscribeToWorkflowEventsRequest {
string workflowRunId = 1;
}
message SubscribeToWorkflowRunsRequest {
// the id of the workflow run
string workflowRunId = 1;
}
enum ResourceType {
RESOURCE_TYPE_UNKNOWN = 0;
RESOURCE_TYPE_STEP_RUN = 1;
@@ -232,6 +239,33 @@ message WorkflowEvent {
optional int32 retryCount = 9;
}
enum WorkflowRunEventType {
WORKFLOW_RUN_EVENT_TYPE_FINISHED = 0;
}
message WorkflowRunEvent {
// the id of the workflow run
string workflowRunId = 1;
WorkflowRunEventType eventType = 2;
google.protobuf.Timestamp eventTimestamp = 3;
repeated StepRunResult results = 4;
}
message StepRunResult {
string stepRunId = 1;
string stepReadableId = 2;
string jobRunId = 3;
optional string error = 4;
optional string output = 5;
}
message OverridesData {
// the step run id
string stepRunId = 1;

View File

@@ -177,7 +177,7 @@ func (a *AuthN) handleBearerAuth(c echo.Context) error {
}
// Validate the token.
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(token)
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(c.Request().Context(), token)
if err != nil {
a.l.Debug().Err(err).Msg("error validating tenant token")

View File

@@ -17,7 +17,7 @@ func (a *APITokenService) ApiTokenCreate(ctx echo.Context, request gen.ApiTokenC
return gen.ApiTokenCreate400JSONResponse(*apiErrors), nil
}
token, err := a.config.Auth.JWTManager.GenerateTenantToken(tenant.ID, request.Body.Name)
token, err := a.config.Auth.JWTManager.GenerateTenantToken(ctx.Request().Context(), tenant.ID, request.Body.Name)
if err != nil {
return nil, err

View File

@@ -19,7 +19,7 @@ func (t *EventService) EventUpdateReplay(ctx echo.Context, request gen.EventUpda
eventIds[i] = request.Body.EventIds[i].String()
}
events, err := t.config.EngineRepository.Event().ListEventsByIds(tenant.ID, eventIds)
events, err := t.config.EngineRepository.Event().ListEventsByIds(ctx.Request().Context(), tenant.ID, eventIds)
if err != nil {
return nil, err

View File

@@ -27,7 +27,7 @@ func (t *StepRunService) StepRunUpdateCancel(ctx echo.Context, request gen.StepR
), nil
}
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(tenant.ID, stepRun.ID)
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(ctx.Request().Context(), tenant.ID, stepRun.ID)
if err != nil {
return nil, fmt.Errorf("could not get step run for engine: %w", err)

View File

@@ -70,7 +70,7 @@ func (t *StepRunService) StepRunUpdateRerun(ctx echo.Context, request gen.StepRu
return nil, err
}
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(tenant.ID, stepRun.ID)
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(ctx.Request().Context(), tenant.ID, stepRun.ID)
if err != nil {
return nil, fmt.Errorf("could not get step run for engine: %w", err)

View File

@@ -36,7 +36,7 @@ func (t *WorkflowService) WorkflowRunCreate(ctx echo.Context, request gen.Workfl
workflowVersionId = versions[0].ID
}
workflowVersion, err := t.config.EngineRepository.Workflow().GetWorkflowVersionById(tenant.ID, workflowVersionId)
workflowVersion, err := t.config.EngineRepository.Workflow().GetWorkflowVersionById(ctx.Request().Context(), tenant.ID, workflowVersionId)
if err != nil {
if errors.Is(err, db.ErrNotFound) {

View File

@@ -1,6 +1,7 @@
package cli
import (
"context"
"errors"
"fmt"
"log"
@@ -123,14 +124,14 @@ func runSeed(cf *loader.ConfigLoader) error {
}
func seedDev(repo repository.EngineRepository, tenantId string) error {
_, err := repo.Workflow().GetWorkflowByName(tenantId, "test-workflow")
_, err := repo.Workflow().GetWorkflowByName(context.Background(), tenantId, "test-workflow")
if err != nil {
if !errors.Is(err, pgx.ErrNoRows) {
return err
}
wf, err := repo.Workflow().CreateNewWorkflow(tenantId, &repository.CreateWorkflowVersionOpts{
wf, err := repo.Workflow().CreateNewWorkflow(context.Background(), tenantId, &repository.CreateWorkflowVersionOpts{
Name: "test-workflow",
Description: repository.StringPtr("This is a test workflow."),
Version: repository.StringPtr("v0.1.0"),

View File

@@ -1,6 +1,7 @@
package cli
import (
"context"
"fmt"
"log"
"os"
@@ -68,7 +69,7 @@ func runCreateAPIToken() error {
defer serverConf.Disconnect() // nolint:errcheck
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(tokenTenantId, tokenName)
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(context.Background(), tokenTenantId, tokenName)
if err != nil {
return err

View File

@@ -58,7 +58,7 @@ func run(ch <-chan interface{}, events chan<- string) error {
interruptCtx, cancel := cmdutils.InterruptContextFromChan(ch)
defer cancel()
err = c.Subscribe().On(interruptCtx, workflowRunId, func(event client.RunEvent) error {
err = c.Subscribe().On(interruptCtx, workflowRunId, func(event client.WorkflowEvent) error {
fmt.Println(event.EventPayload)
return nil

View File

@@ -2,6 +2,7 @@ package main
import (
"fmt"
"sync"
"time"
"github.com/joho/godotenv"
@@ -95,7 +96,8 @@ func run(events chan<- string) (func() error, error) {
eg.SetLimit(NUM_CHILDREN)
childOutputs := make([]int, NUM_CHILDREN)
childOutputs := make([]int, 0)
childOutputsMu := sync.Mutex{}
for i, childWorkflow := range childWorkflows {
eg.Go(func(i int, childWorkflow *worker.ChildWorkflow) func() error {
@@ -114,7 +116,9 @@ func run(events chan<- string) (func() error, error) {
return err
}
childOutputs[i] = childOutput.Index
childOutputsMu.Lock()
childOutputs = append(childOutputs, childOutput.Index)
childOutputsMu.Unlock()
events <- fmt.Sprintf("child-%d-completed", childOutput.Index)
@@ -124,10 +128,38 @@ func run(events chan<- string) (func() error, error) {
}(i, childWorkflow))
}
err = eg.Wait()
finishedCh := make(chan struct{})
if err != nil {
return nil, err
go func() {
defer close(finishedCh)
err = eg.Wait()
}()
timer := time.NewTimer(60 * time.Second)
select {
case <-finishedCh:
if err != nil {
return nil, err
}
case <-timer.C:
incomplete := make([]int, 0)
// print non-complete children
for i := range childWorkflows {
completed := false
for _, childOutput := range childOutputs {
if childOutput == i {
completed = true
break
}
}
if !completed {
incomplete = append(incomplete, i)
}
}
return nil, fmt.Errorf("timed out waiting for the following child workflows to complete: %v", incomplete)
}
sum := 0
@@ -140,7 +172,7 @@ func run(events chan<- string) (func() error, error) {
ChildSum: sum,
}, nil
},
),
).SetTimeout("10m"),
},
},
)

View File

@@ -283,7 +283,7 @@ Which gives us much more promising output:
This works — and you can modify this logic to be more distributed by maintaining a lease when a worker starts for a set amount of time — as long as the polling interval is below the query duration time (or more specifically, `pollingTime / numWorkers` is below the query duration time). But what happens when our queue starts to fill up? Let's add 10,000 enqueued tasks and run an `EXPLAIN ANALYZE` for this query to take a look at performance:
```sql
QUERY PLAN
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------------------------
Update on tasks (cost=259.44..514.23 rows=1 width=78) (actual time=132.717..154.337 rows=100 loops=1)
-> Hash Join (cost=259.44..514.23 rows=1 width=78) (actual time=125.423..141.271 rows=100 loops=1)

View File

@@ -1,6 +1,7 @@
package token
import (
"context"
"fmt"
"time"
@@ -12,8 +13,8 @@ import (
)
type JWTManager interface {
GenerateTenantToken(tenantId, name string) (string, error)
ValidateTenantToken(token string) (string, error)
GenerateTenantToken(ctx context.Context, tenantId, name string) (string, error)
ValidateTenantToken(ctx context.Context, token string) (string, error)
}
type TokenOpts struct {
@@ -45,7 +46,7 @@ func NewJWTManager(encryptionSvc encryption.EncryptionService, tokenRepo reposit
}, nil
}
func (j *jwtManagerImpl) GenerateTenantToken(tenantId, name string) (string, error) {
func (j *jwtManagerImpl) GenerateTenantToken(ctx context.Context, tenantId, name string) (string, error) {
// Retrieve the JWT Signer primitive from privateKeysetHandle.
signer, err := jwt.NewSigner(j.encryption.GetPrivateJWTHandle())
@@ -68,7 +69,7 @@ func (j *jwtManagerImpl) GenerateTenantToken(tenantId, name string) (string, err
}
// write the token to the database
_, err = j.tokenRepo.CreateAPIToken(&repository.CreateAPITokenOpts{
_, err = j.tokenRepo.CreateAPIToken(ctx, &repository.CreateAPITokenOpts{
ID: tokenId,
ExpiresAt: expiresAt,
TenantId: &tenantId,
@@ -82,7 +83,7 @@ func (j *jwtManagerImpl) GenerateTenantToken(tenantId, name string) (string, err
return token, nil
}
func (j *jwtManagerImpl) ValidateTenantToken(token string) (tenantId string, err error) {
func (j *jwtManagerImpl) ValidateTenantToken(ctx context.Context, token string) (tenantId string, err error) {
// Verify the signed token.
audience := j.opts.Audience
@@ -140,7 +141,7 @@ func (j *jwtManagerImpl) ValidateTenantToken(token string) (tenantId string, err
}
// read the token from the database
dbToken, err := j.tokenRepo.GetAPITokenById(tokenId)
dbToken, err := j.tokenRepo.GetAPITokenById(ctx, tokenId)
if err != nil {
return "", fmt.Errorf("failed to read token from database: %v", err)

View File

@@ -3,6 +3,7 @@
package token_test
import (
"context"
"fmt"
"os"
"testing"
@@ -40,14 +41,14 @@ func TestCreateTenantToken(t *testing.T) { // make sure no cache is used for tes
t.Fatal(err.Error())
}
token, err := jwtManager.GenerateTenantToken(tenantId, "test token")
token, err := jwtManager.GenerateTenantToken(context.Background(), tenantId, "test token")
if err != nil {
t.Fatal(err.Error())
}
// validate the token
newTenantId, err := jwtManager.ValidateTenantToken(token)
newTenantId, err := jwtManager.ValidateTenantToken(context.Background(), token)
assert.NoError(t, err)
assert.Equal(t, tenantId, newTenantId)
@@ -81,14 +82,14 @@ func TestRevokeTenantToken(t *testing.T) {
t.Fatal(err.Error())
}
token, err := jwtManager.GenerateTenantToken(tenantId, "test token")
token, err := jwtManager.GenerateTenantToken(context.Background(), tenantId, "test token")
if err != nil {
t.Fatal(err.Error())
}
// validate the token
_, err = jwtManager.ValidateTenantToken(token)
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
assert.NoError(t, err)
@@ -107,7 +108,7 @@ func TestRevokeTenantToken(t *testing.T) {
}
// validate the token again
_, err = jwtManager.ValidateTenantToken(token)
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
// error as the token was revoked
assert.Error(t, err)
@@ -141,14 +142,14 @@ func TestRevokeTenantTokenCache(t *testing.T) {
t.Fatal(err.Error())
}
token, err := jwtManager.GenerateTenantToken(tenantId, "test token")
token, err := jwtManager.GenerateTenantToken(context.Background(), tenantId, "test token")
if err != nil {
t.Fatal(err.Error())
}
// validate the token
_, err = jwtManager.ValidateTenantToken(token)
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
assert.NoError(t, err)
@@ -167,7 +168,7 @@ func TestRevokeTenantTokenCache(t *testing.T) {
}
// validate the token again
_, err = jwtManager.ValidateTenantToken(token)
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
// no error as it is cached
assert.NoError(t, err)

View File

@@ -336,7 +336,7 @@ func GetServerConfigFromConfigfile(dc *database.Config, cf *server.ServerConfigF
}
// generate a token for the internal client
token, err := auth.JWTManager.GenerateTenantToken(internalTenant.ID, fmt.Sprintf("internal-%s", tokenSuffix))
token, err := auth.JWTManager.GenerateTenantToken(context.Background(), internalTenant.ID, fmt.Sprintf("internal-%s", tokenSuffix))
if err != nil {
return nil, nil, fmt.Errorf("could not generate internal token: %w", err)

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
@@ -28,6 +29,6 @@ type APITokenRepository interface {
}
type EngineTokenRepository interface {
CreateAPIToken(opts *CreateAPITokenOpts) (*dbsqlc.APIToken, error)
GetAPITokenById(id string) (*dbsqlc.APIToken, error)
CreateAPIToken(ctx context.Context, opts *CreateAPITokenOpts) (*dbsqlc.APIToken, error)
GetAPITokenById(ctx context.Context, id string) (*dbsqlc.APIToken, error)
}

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
@@ -16,12 +17,12 @@ type UpdateDispatcherOpts struct {
type DispatcherEngineRepository interface {
// CreateNewDispatcher creates a new dispatcher for a given tenant.
CreateNewDispatcher(opts *CreateDispatcherOpts) (*dbsqlc.Dispatcher, error)
CreateNewDispatcher(ctx context.Context, opts *CreateDispatcherOpts) (*dbsqlc.Dispatcher, error)
// UpdateDispatcher updates a dispatcher for a given tenant.
UpdateDispatcher(dispatcherId string, opts *UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error)
UpdateDispatcher(ctx context.Context, dispatcherId string, opts *UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error)
Delete(dispatcherId string) error
Delete(ctx context.Context, dispatcherId string) error
UpdateStaleDispatchers(onStale func(dispatcherId string, getValidDispatcherId func() string) error) error
UpdateStaleDispatchers(ctx context.Context, onStale func(dispatcherId string, getValidDispatcherId func() string) error) error
}

View File

@@ -74,7 +74,7 @@ type EventEngineRepository interface {
CreateEvent(ctx context.Context, opts *CreateEventOpts) (*dbsqlc.Event, error)
// GetEventForEngine returns an event for the engine by id.
GetEventForEngine(tenantId, id string) (*dbsqlc.Event, error)
GetEventForEngine(ctx context.Context, tenantId, id string) (*dbsqlc.Event, error)
ListEventsByIds(tenantId string, ids []string) ([]*dbsqlc.Event, error)
ListEventsByIds(ctx context.Context, tenantId string, ids []string) ([]*dbsqlc.Event, error)
}

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
@@ -35,14 +36,14 @@ type UpdateGetGroupKeyRunOpts struct {
type GetGroupKeyRunEngineRepository interface {
// ListStepRunsToRequeue returns a list of step runs which are in a requeueable state.
ListGetGroupKeyRunsToRequeue(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
ListGetGroupKeyRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
ListGetGroupKeyRunsToReassign(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
ListGetGroupKeyRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
AssignGetGroupKeyRunToWorker(tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error)
AssignGetGroupKeyRunToTicker(tenantId, getGroupKeyRunId string) (tickerId string, err error)
AssignGetGroupKeyRunToWorker(ctx context.Context, tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error)
AssignGetGroupKeyRunToTicker(ctx context.Context, tenantId, getGroupKeyRunId string) (tickerId string, err error)
UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId string, opts *UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
UpdateGetGroupKeyRun(ctx context.Context, tenantId, getGroupKeyRunId string, opts *UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
GetGroupKeyRunForEngine(tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
GetGroupKeyRunForEngine(ctx context.Context, tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
}

View File

@@ -1,6 +1,8 @@
package repository
import (
"context"
"github.com/jackc/pgx/v5/pgtype"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
@@ -32,7 +34,7 @@ type JobRunAPIRepository interface {
type JobRunEngineRepository interface {
// SetJobRunStatusRunning resets the status of a job run to a RUNNING status. This is useful if a step
// run is being manually replayed, but shouldn't be used by most callers.
SetJobRunStatusRunning(tenantId, jobRunId string) error
SetJobRunStatusRunning(ctx context.Context, tenantId, jobRunId string) error
ListJobRunsForWorkflowRun(tenantId, workflowRunId string) ([]pgtype.UUID, error)
ListJobRunsForWorkflowRun(ctx context.Context, tenantId, workflowRunId string) ([]pgtype.UUID, error)
}

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
@@ -58,5 +59,5 @@ type LogsAPIRepository interface {
type LogsEngineRepository interface {
// PutLog creates a new log line.
PutLog(tenantId string, opts *CreateLogLineOpts) (*dbsqlc.LogLine, error)
PutLog(ctx context.Context, tenantId string, opts *CreateLogLineOpts) (*dbsqlc.LogLine, error)
}

View File

@@ -100,7 +100,7 @@ func NewEngineTokenRepository(pool *pgxpool.Pool, v validator.Validator, l *zero
}
}
func (a *engineTokenRepository) CreateAPIToken(opts *repository.CreateAPITokenOpts) (*dbsqlc.APIToken, error) {
func (a *engineTokenRepository) CreateAPIToken(ctx context.Context, opts *repository.CreateAPITokenOpts) (*dbsqlc.APIToken, error) {
if err := a.v.Validate(opts); err != nil {
return nil, err
}
@@ -118,11 +118,11 @@ func (a *engineTokenRepository) CreateAPIToken(opts *repository.CreateAPITokenOp
createParams.Name = sqlchelpers.TextFromStr(*opts.Name)
}
return a.queries.CreateAPIToken(context.Background(), a.pool, createParams)
return a.queries.CreateAPIToken(ctx, a.pool, createParams)
}
func (a *engineTokenRepository) GetAPITokenById(id string) (*dbsqlc.APIToken, error) {
func (a *engineTokenRepository) GetAPITokenById(ctx context.Context, id string) (*dbsqlc.APIToken, error) {
return cache.MakeCacheable[dbsqlc.APIToken](a.cache, id, func() (*dbsqlc.APIToken, error) {
return a.queries.GetAPITokenById(context.Background(), a.pool, sqlchelpers.UUIDFromStr(id))
return a.queries.GetAPITokenById(ctx, a.pool, sqlchelpers.UUIDFromStr(id))
})
}

View File

@@ -96,14 +96,17 @@ FROM
JOIN
"JobRun" ON "StepRun"."jobRunId" = "JobRun"."id"
WHERE
"StepRun"."tenantId" = @tenantId::uuid
(
sqlc.narg('tenantId')::uuid IS NULL OR
"StepRun"."tenantId" = sqlc.narg('tenantId')::uuid
)
AND (
sqlc.narg('status')::"StepRunStatus" IS NULL OR
"StepRun"."status" = sqlc.narg('status')::"StepRunStatus"
)
AND (
sqlc.narg('workflowRunId')::uuid IS NULL OR
"JobRun"."workflowRunId" = sqlc.narg('workflowRunId')::uuid
sqlc.narg('workflowRunIds')::uuid[] IS NULL OR
"JobRun"."workflowRunId" = ANY(sqlc.narg('workflowRunIds')::uuid[])
)
AND (
sqlc.narg('jobRunId')::uuid IS NULL OR

View File

@@ -432,14 +432,17 @@ FROM
JOIN
"JobRun" ON "StepRun"."jobRunId" = "JobRun"."id"
WHERE
"StepRun"."tenantId" = $1::uuid
(
$1::uuid IS NULL OR
"StepRun"."tenantId" = $1::uuid
)
AND (
$2::"StepRunStatus" IS NULL OR
"StepRun"."status" = $2::"StepRunStatus"
)
AND (
$3::uuid IS NULL OR
"JobRun"."workflowRunId" = $3::uuid
$3::uuid[] IS NULL OR
"JobRun"."workflowRunId" = ANY($3::uuid[])
)
AND (
$4::uuid IS NULL OR
@@ -452,18 +455,18 @@ WHERE
`
type ListStepRunsParams struct {
Tenantid pgtype.UUID `json:"tenantid"`
Status NullStepRunStatus `json:"status"`
WorkflowRunId pgtype.UUID `json:"workflowRunId"`
JobRunId pgtype.UUID `json:"jobRunId"`
TickerId pgtype.UUID `json:"tickerId"`
TenantId pgtype.UUID `json:"tenantId"`
Status NullStepRunStatus `json:"status"`
WorkflowRunIds []pgtype.UUID `json:"workflowRunIds"`
JobRunId pgtype.UUID `json:"jobRunId"`
TickerId pgtype.UUID `json:"tickerId"`
}
func (q *Queries) ListStepRuns(ctx context.Context, db DBTX, arg ListStepRunsParams) ([]pgtype.UUID, error) {
rows, err := db.Query(ctx, listStepRuns,
arg.Tenantid,
arg.TenantId,
arg.Status,
arg.WorkflowRunId,
arg.WorkflowRunIds,
arg.JobRunId,
arg.TickerId,
)

View File

@@ -21,6 +21,10 @@ WHERE
sqlc.narg('workflowId')::uuid IS NULL OR
workflow."id" = sqlc.narg('workflowId')::uuid
) AND
(
sqlc.narg('ids')::uuid[] IS NULL OR
runs."id" = ANY(sqlc.narg('ids')::uuid[])
) AND
(
sqlc.narg('parentId')::uuid IS NULL OR
runs."parentId" = sqlc.narg('parentId')::uuid
@@ -70,6 +74,10 @@ WHERE
sqlc.narg('workflowId')::uuid IS NULL OR
workflow."id" = sqlc.narg('workflowId')::uuid
) AND
(
sqlc.narg('ids')::uuid[] IS NULL OR
runs."id" = ANY(sqlc.narg('ids')::uuid[])
) AND
(
sqlc.narg('parentId')::uuid IS NULL OR
runs."parentId" = sqlc.narg('parentId')::uuid

View File

@@ -35,36 +35,41 @@ WHERE
workflow."id" = $3::uuid
) AND
(
$4::uuid IS NULL OR
runs."parentId" = $4::uuid
$4::uuid[] IS NULL OR
runs."id" = ANY($4::uuid[])
) AND
(
$5::uuid IS NULL OR
runs."parentStepRunId" = $5::uuid
runs."parentId" = $5::uuid
) AND
(
$6::uuid IS NULL OR
events."id" = $6::uuid
runs."parentStepRunId" = $6::uuid
) AND
(
$7::text IS NULL OR
runs."concurrencyGroupId" = $7::text
$7::uuid IS NULL OR
events."id" = $7::uuid
) AND
(
$8::text[] IS NULL OR
"status" = ANY(cast($8::text[] as "WorkflowRunStatus"[]))
$8::text IS NULL OR
runs."concurrencyGroupId" = $8::text
) AND
(
$9::text[] IS NULL OR
"status" = ANY(cast($9::text[] as "WorkflowRunStatus"[]))
)
`
type CountWorkflowRunsParams struct {
TenantId pgtype.UUID `json:"tenantId"`
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
WorkflowId pgtype.UUID `json:"workflowId"`
ParentId pgtype.UUID `json:"parentId"`
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
EventId pgtype.UUID `json:"eventId"`
GroupKey pgtype.Text `json:"groupKey"`
Statuses []string `json:"statuses"`
TenantId pgtype.UUID `json:"tenantId"`
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
WorkflowId pgtype.UUID `json:"workflowId"`
Ids []pgtype.UUID `json:"ids"`
ParentId pgtype.UUID `json:"parentId"`
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
EventId pgtype.UUID `json:"eventId"`
GroupKey pgtype.Text `json:"groupKey"`
Statuses []string `json:"statuses"`
}
func (q *Queries) CountWorkflowRuns(ctx context.Context, db DBTX, arg CountWorkflowRunsParams) (int64, error) {
@@ -72,6 +77,7 @@ func (q *Queries) CountWorkflowRuns(ctx context.Context, db DBTX, arg CountWorkf
arg.TenantId,
arg.WorkflowVersionId,
arg.WorkflowId,
arg.Ids,
arg.ParentId,
arg.ParentStepRunId,
arg.EventId,
@@ -709,46 +715,51 @@ WHERE
workflow."id" = $3::uuid
) AND
(
$4::uuid IS NULL OR
runs."parentId" = $4::uuid
$4::uuid[] IS NULL OR
runs."id" = ANY($4::uuid[])
) AND
(
$5::uuid IS NULL OR
runs."parentStepRunId" = $5::uuid
runs."parentId" = $5::uuid
) AND
(
$6::uuid IS NULL OR
events."id" = $6::uuid
runs."parentStepRunId" = $6::uuid
) AND
(
$7::text IS NULL OR
runs."concurrencyGroupId" = $7::text
$7::uuid IS NULL OR
events."id" = $7::uuid
) AND
(
$8::text[] IS NULL OR
"status" = ANY(cast($8::text[] as "WorkflowRunStatus"[]))
$8::text IS NULL OR
runs."concurrencyGroupId" = $8::text
) AND
(
$9::text[] IS NULL OR
"status" = ANY(cast($9::text[] as "WorkflowRunStatus"[]))
)
ORDER BY
case when $9 = 'createdAt ASC' THEN runs."createdAt" END ASC ,
case when $9 = 'createdAt DESC' then runs."createdAt" END DESC
case when $10 = 'createdAt ASC' THEN runs."createdAt" END ASC ,
case when $10 = 'createdAt DESC' then runs."createdAt" END DESC
OFFSET
COALESCE($10, 0)
COALESCE($11, 0)
LIMIT
COALESCE($11, 50)
COALESCE($12, 50)
`
type ListWorkflowRunsParams struct {
TenantId pgtype.UUID `json:"tenantId"`
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
WorkflowId pgtype.UUID `json:"workflowId"`
ParentId pgtype.UUID `json:"parentId"`
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
EventId pgtype.UUID `json:"eventId"`
GroupKey pgtype.Text `json:"groupKey"`
Statuses []string `json:"statuses"`
Orderby interface{} `json:"orderby"`
Offset interface{} `json:"offset"`
Limit interface{} `json:"limit"`
TenantId pgtype.UUID `json:"tenantId"`
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
WorkflowId pgtype.UUID `json:"workflowId"`
Ids []pgtype.UUID `json:"ids"`
ParentId pgtype.UUID `json:"parentId"`
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
EventId pgtype.UUID `json:"eventId"`
GroupKey pgtype.Text `json:"groupKey"`
Statuses []string `json:"statuses"`
Orderby interface{} `json:"orderby"`
Offset interface{} `json:"offset"`
Limit interface{} `json:"limit"`
}
type ListWorkflowRunsRow struct {
@@ -767,6 +778,7 @@ func (q *Queries) ListWorkflowRuns(ctx context.Context, db DBTX, arg ListWorkflo
arg.TenantId,
arg.WorkflowVersionId,
arg.WorkflowId,
arg.Ids,
arg.ParentId,
arg.ParentStepRunId,
arg.EventId,

View File

@@ -31,33 +31,33 @@ func NewDispatcherRepository(pool *pgxpool.Pool, v validator.Validator, l *zerol
}
}
func (d *dispatcherRepository) CreateNewDispatcher(opts *repository.CreateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
func (d *dispatcherRepository) CreateNewDispatcher(ctx context.Context, opts *repository.CreateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
if err := d.v.Validate(opts); err != nil {
return nil, err
}
return d.queries.CreateDispatcher(context.Background(), d.pool, sqlchelpers.UUIDFromStr(opts.ID))
return d.queries.CreateDispatcher(ctx, d.pool, sqlchelpers.UUIDFromStr(opts.ID))
}
func (d *dispatcherRepository) UpdateDispatcher(dispatcherId string, opts *repository.UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
func (d *dispatcherRepository) UpdateDispatcher(ctx context.Context, dispatcherId string, opts *repository.UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
if err := d.v.Validate(opts); err != nil {
return nil, err
}
return d.queries.UpdateDispatcher(context.Background(), d.pool, dbsqlc.UpdateDispatcherParams{
return d.queries.UpdateDispatcher(ctx, d.pool, dbsqlc.UpdateDispatcherParams{
ID: sqlchelpers.UUIDFromStr(dispatcherId),
LastHeartbeatAt: sqlchelpers.TimestampFromTime(opts.LastHeartbeatAt.UTC()),
})
}
func (d *dispatcherRepository) Delete(dispatcherId string) error {
_, err := d.queries.DeleteDispatcher(context.Background(), d.pool, sqlchelpers.UUIDFromStr(dispatcherId))
func (d *dispatcherRepository) Delete(ctx context.Context, dispatcherId string) error {
_, err := d.queries.DeleteDispatcher(ctx, d.pool, sqlchelpers.UUIDFromStr(dispatcherId))
return err
}
func (d *dispatcherRepository) UpdateStaleDispatchers(onStale func(dispatcherId string, getValidDispatcherId func() string) error) error {
tx, err := d.pool.Begin(context.Background())
func (d *dispatcherRepository) UpdateStaleDispatchers(ctx context.Context, onStale func(dispatcherId string, getValidDispatcherId func() string) error) error {
tx, err := d.pool.Begin(ctx)
if err != nil {
return err

View File

@@ -206,8 +206,8 @@ func NewEventEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zero
}
}
func (r *eventEngineRepository) GetEventForEngine(tenantId, id string) (*dbsqlc.Event, error) {
return r.queries.GetEventForEngine(context.Background(), r.pool, sqlchelpers.UUIDFromStr(id))
func (r *eventEngineRepository) GetEventForEngine(ctx context.Context, tenantId, id string) (*dbsqlc.Event, error) {
return r.queries.GetEventForEngine(ctx, r.pool, sqlchelpers.UUIDFromStr(id))
}
func (r *eventEngineRepository) CreateEvent(ctx context.Context, opts *repository.CreateEventOpts) (*dbsqlc.Event, error) {
@@ -242,7 +242,7 @@ func (r *eventEngineRepository) CreateEvent(ctx context.Context, opts *repositor
return e, nil
}
func (r *eventEngineRepository) ListEventsByIds(tenantId string, ids []string) ([]*dbsqlc.Event, error) {
func (r *eventEngineRepository) ListEventsByIds(ctx context.Context, tenantId string, ids []string) ([]*dbsqlc.Event, error) {
pgIds := make([]pgtype.UUID, len(ids))
for i, id := range ids {
@@ -253,7 +253,7 @@ func (r *eventEngineRepository) ListEventsByIds(tenantId string, ids []string) (
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
return r.queries.ListEventsByIDs(context.Background(), r.pool, dbsqlc.ListEventsByIDsParams{
return r.queries.ListEventsByIDs(ctx, r.pool, dbsqlc.ListEventsByIDsParams{
Tenantid: pgTenantId,
Ids: pgIds,
})

View File

@@ -34,20 +34,20 @@ func NewGetGroupKeyRunRepository(pool *pgxpool.Pool, v validator.Validator, l *z
}
}
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToRequeue(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
return s.queries.ListGetGroupKeyRunsToRequeue(context.Background(), s.pool, sqlchelpers.UUIDFromStr(tenantId))
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
return s.queries.ListGetGroupKeyRunsToRequeue(ctx, s.pool, sqlchelpers.UUIDFromStr(tenantId))
}
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToReassign(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
return s.queries.ListGetGroupKeyRunsToReassign(context.Background(), s.pool, sqlchelpers.UUIDFromStr(tenantId))
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
return s.queries.ListGetGroupKeyRunsToReassign(ctx, s.pool, sqlchelpers.UUIDFromStr(tenantId))
}
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToWorker(tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error) {
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToWorker(ctx context.Context, tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error) {
// var assigned
var assigned *dbsqlc.AssignGetGroupKeyRunToWorkerRow
err = retrier(s.l, func() (err error) {
assigned, err = s.queries.AssignGetGroupKeyRunToWorker(context.Background(), s.pool, dbsqlc.AssignGetGroupKeyRunToWorkerParams{
assigned, err = s.queries.AssignGetGroupKeyRunToWorker(ctx, s.pool, dbsqlc.AssignGetGroupKeyRunToWorkerParams{
Getgroupkeyrunid: sqlchelpers.UUIDFromStr(getGroupKeyRunId),
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
})
@@ -70,12 +70,12 @@ func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToWorker(tenantId, getGro
return sqlchelpers.UUIDToStr(assigned.WorkerId), sqlchelpers.UUIDToStr(assigned.DispatcherId), nil
}
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToTicker(tenantId, getGroupKeyRunId string) (tickerId string, err error) {
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToTicker(ctx context.Context, tenantId, getGroupKeyRunId string) (tickerId string, err error) {
// var assigned
var assigned *dbsqlc.AssignGetGroupKeyRunToTickerRow
err = retrier(s.l, func() (err error) {
assigned, err = s.queries.AssignGetGroupKeyRunToTicker(context.Background(), s.pool, dbsqlc.AssignGetGroupKeyRunToTickerParams{
assigned, err = s.queries.AssignGetGroupKeyRunToTicker(ctx, s.pool, dbsqlc.AssignGetGroupKeyRunToTickerParams{
Getgroupkeyrunid: sqlchelpers.UUIDFromStr(getGroupKeyRunId),
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
})
@@ -98,7 +98,7 @@ func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToTicker(tenantId, getGro
return sqlchelpers.UUIDToStr(assigned.TickerId), nil
}
func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId string, opts *repository.UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(ctx context.Context, tenantId, getGroupKeyRunId string, opts *repository.UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
if err := s.v.Validate(opts); err != nil {
return nil, err
}
@@ -158,15 +158,15 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
updateParams.ScheduleTimeoutAt = sqlchelpers.TimestampFromTime(*opts.ScheduleTimeoutAt)
}
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
res1, err := s.queries.UpdateGetGroupKeyRun(context.Background(), tx, updateParams)
res1, err := s.queries.UpdateGetGroupKeyRun(ctx, tx, updateParams)
if err != nil {
return nil, fmt.Errorf("could not update get group key run: %w", err)
@@ -174,14 +174,14 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
// only update workflow run if status or output has changed
if opts.Status != nil || opts.Output != nil {
_, err = s.queries.UpdateWorkflowRunGroupKey(context.Background(), tx, updateWorkflowRunParams)
_, err = s.queries.UpdateWorkflowRunGroupKey(ctx, tx, updateWorkflowRunParams)
if err != nil {
return nil, fmt.Errorf("could not resolve workflow run status from get group key run: %w", err)
}
}
getGroupKeyRuns, err := s.queries.GetGroupKeyRunForEngine(context.Background(), tx, dbsqlc.GetGroupKeyRunForEngineParams{
getGroupKeyRuns, err := s.queries.GetGroupKeyRunForEngine(ctx, tx, dbsqlc.GetGroupKeyRunForEngineParams{
Ids: []pgtype.UUID{res1.ID},
Tenantid: pgTenantId,
})
@@ -194,7 +194,7 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
return nil, fmt.Errorf("could not find get group key run for engine")
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -203,8 +203,8 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
return getGroupKeyRuns[0], nil
}
func (s *getGroupKeyRunRepository) GetGroupKeyRunForEngine(tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
res, err := s.queries.GetGroupKeyRunForEngine(context.Background(), s.pool, dbsqlc.GetGroupKeyRunForEngineParams{
func (s *getGroupKeyRunRepository) GetGroupKeyRunForEngine(ctx context.Context, tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
res, err := s.queries.GetGroupKeyRunForEngine(ctx, s.pool, dbsqlc.GetGroupKeyRunForEngineParams{
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(getGroupKeyRunId)},
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
})

View File

@@ -56,16 +56,16 @@ func NewJobRunEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zer
}
}
func (j *jobRunEngineRepository) SetJobRunStatusRunning(tenantId, jobRunId string) error {
return setJobRunStatusRunning(context.Background(), j.pool, j.queries, j.l, tenantId, jobRunId)
func (j *jobRunEngineRepository) SetJobRunStatusRunning(ctx context.Context, tenantId, jobRunId string) error {
return setJobRunStatusRunning(ctx, j.pool, j.queries, j.l, tenantId, jobRunId)
}
func (j *jobRunEngineRepository) ListJobRunsForWorkflowRun(tenantId, workflowRunId string) ([]pgtype.UUID, error) {
return j.queries.ListJobRunsForWorkflowRun(context.Background(), j.pool, sqlchelpers.UUIDFromStr(workflowRunId))
func (j *jobRunEngineRepository) ListJobRunsForWorkflowRun(ctx context.Context, tenantId, workflowRunId string) ([]pgtype.UUID, error) {
return j.queries.ListJobRunsForWorkflowRun(ctx, j.pool, sqlchelpers.UUIDFromStr(workflowRunId))
}
func setJobRunStatusRunning(ctx context.Context, pool *pgxpool.Pool, queries *dbsqlc.Queries, l *zerolog.Logger, tenantId, jobRunId string) error {
tx, err := pool.Begin(context.Background())
tx, err := pool.Begin(ctx)
if err != nil {
return err

View File

@@ -150,7 +150,7 @@ func NewLogEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zerolo
}
}
func (r *logEngineRepository) PutLog(tenantId string, opts *repository.CreateLogLineOpts) (*dbsqlc.LogLine, error) {
func (r *logEngineRepository) PutLog(ctx context.Context, tenantId string, opts *repository.CreateLogLineOpts) (*dbsqlc.LogLine, error) {
if err := r.v.Validate(opts); err != nil {
return nil, err
}
@@ -177,16 +177,16 @@ func (r *logEngineRepository) PutLog(tenantId string, opts *repository.CreateLog
createParams.Metadata = opts.Metadata
}
tx, err := r.pool.Begin(context.Background())
tx, err := r.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), r.l, tx.Rollback)
defer deferRollback(ctx, r.l, tx.Rollback)
logLine, err := r.queries.CreateLogLine(
context.Background(),
ctx,
tx,
createParams,
)
@@ -195,7 +195,7 @@ func (r *logEngineRepository) PutLog(tenantId string, opts *repository.CreateLog
return nil, fmt.Errorf("could not create log line: %w", err)
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, fmt.Errorf("could not commit transaction: %w", err)

View File

@@ -31,7 +31,7 @@ func NewRateLimitEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *
}
}
func (r *rateLimitEngineRepository) UpsertRateLimit(tenantId string, key string, opts *repository.UpsertRateLimitOpts) (*dbsqlc.RateLimit, error) {
func (r *rateLimitEngineRepository) UpsertRateLimit(ctx context.Context, tenantId string, key string, opts *repository.UpsertRateLimitOpts) (*dbsqlc.RateLimit, error) {
if err := r.v.Validate(opts); err != nil {
return nil, err
}
@@ -46,7 +46,7 @@ func (r *rateLimitEngineRepository) UpsertRateLimit(tenantId string, key string,
upsertParams.Window = sqlchelpers.TextFromStr(fmt.Sprintf("1 %s", *opts.Duration))
}
rateLimit, err := r.queries.UpsertRateLimit(context.Background(), r.pool, upsertParams)
rateLimit, err := r.queries.UpsertRateLimit(ctx, r.pool, upsertParams)
if err != nil {
return nil, fmt.Errorf("could not upsert rate limit: %w", err)

View File

@@ -91,18 +91,19 @@ func NewStepRunEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *ze
}
}
func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(tickerId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
tx, err := s.pool.Begin(context.Background())
func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(ctx context.Context, tickerId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
srs, err := s.queries.ListStepRuns(context.Background(), s.pool, dbsqlc.ListStepRunsParams{
srs, err := s.queries.ListStepRuns(ctx, tx, dbsqlc.ListStepRunsParams{
Status: dbsqlc.NullStepRunStatus{
StepRunStatus: dbsqlc.StepRunStatusRUNNING,
Valid: true,
},
TickerId: sqlchelpers.UUIDFromStr(tickerId),
})
@@ -111,7 +112,7 @@ func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(tickerId string)
return nil, err
}
res, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
res, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
Ids: srs,
})
@@ -119,32 +120,50 @@ func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(tickerId string)
return nil, err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
return res, err
}
func (s *stepRunEngineRepository) ListRunningStepRunsForWorkflowRun(tenantId, workflowRunId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
tx, err := s.pool.Begin(context.Background())
func (s *stepRunEngineRepository) ListStepRuns(ctx context.Context, tenantId string, opts *repository.ListStepRunsOpts) ([]*dbsqlc.GetStepRunForEngineRow, error) {
if err := s.v.Validate(opts); err != nil {
return nil, err
}
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
srs, err := s.queries.ListStepRuns(context.Background(), s.pool, dbsqlc.ListStepRunsParams{
Status: dbsqlc.NullStepRunStatus{
StepRunStatus: dbsqlc.StepRunStatusRUNNING,
},
WorkflowRunId: sqlchelpers.UUIDFromStr(workflowRunId),
})
listOpts := dbsqlc.ListStepRunsParams{
TenantId: sqlchelpers.UUIDFromStr(tenantId),
}
if opts.Status != nil {
listOpts.Status = dbsqlc.NullStepRunStatus{
StepRunStatus: *opts.Status,
Valid: true,
}
}
if opts.WorkflowRunIds != nil {
listOpts.WorkflowRunIds = make([]pgtype.UUID, len(opts.WorkflowRunIds))
for i, id := range opts.WorkflowRunIds {
listOpts.WorkflowRunIds[i] = sqlchelpers.UUIDFromStr(id)
}
}
srs, err := s.queries.ListStepRuns(ctx, tx, listOpts)
if err != nil {
return nil, err
}
res, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
res, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
Ids: srs,
})
@@ -152,30 +171,30 @@ func (s *stepRunEngineRepository) ListRunningStepRunsForWorkflowRun(tenantId, wo
return nil, err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
return res, err
}
func (s *stepRunEngineRepository) ListStepRunsToRequeue(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
func (s *stepRunEngineRepository) ListStepRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
// get the step run and make sure it's still in pending
stepRunIds, err := s.queries.ListStepRunsToRequeue(context.Background(), tx, pgTenantId)
stepRunIds, err := s.queries.ListStepRunsToRequeue(ctx, tx, pgTenantId)
if err != nil {
return nil, err
}
stepRuns, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
stepRuns, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
Ids: stepRunIds,
TenantId: pgTenantId,
})
@@ -184,7 +203,7 @@ func (s *stepRunEngineRepository) ListStepRunsToRequeue(tenantId string) ([]*dbs
return nil, err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -193,25 +212,25 @@ func (s *stepRunEngineRepository) ListStepRunsToRequeue(tenantId string) ([]*dbs
return stepRuns, nil
}
func (s *stepRunEngineRepository) ListStepRunsToReassign(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
func (s *stepRunEngineRepository) ListStepRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
// get the step run and make sure it's still in pending
stepRunIds, err := s.queries.ListStepRunsToReassign(context.Background(), tx, pgTenantId)
stepRunIds, err := s.queries.ListStepRunsToReassign(ctx, tx, pgTenantId)
if err != nil {
return nil, err
}
stepRuns, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
stepRuns, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
Ids: stepRunIds,
TenantId: pgTenantId,
})
@@ -220,7 +239,7 @@ func (s *stepRunEngineRepository) ListStepRunsToReassign(tenantId string) ([]*db
return nil, err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -266,13 +285,13 @@ var retrier = func(l *zerolog.Logger, f func() error) error {
}
func (s *stepRunEngineRepository) AssignStepRunToWorker(ctx context.Context, stepRun *dbsqlc.GetStepRunForEngineRow) (string, string, error) {
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return "", "", err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
// Update the old worker semaphore. This will only increment if the step run was already assigned to a worker,
// which means the step run is being retried or rerun.
@@ -336,7 +355,7 @@ func (s *stepRunEngineRepository) AssignStepRunToWorker(ctx context.Context, ste
}
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return "", "", err
@@ -364,13 +383,13 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
var stepRun *dbsqlc.GetStepRunForEngineRow
err = retrier(s.l, func() error {
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
stepRun, err = s.updateStepRunCore(ctx, tx, tenantId, updateParams, updateJobRunLookupDataParams)
@@ -378,7 +397,7 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
return err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
return err
})
@@ -388,13 +407,13 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
}
err = retrier(s.l, func() error {
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
updateInfo, err = s.updateStepRunExtra(ctx, tx, tenantId, resolveJobRunParams, resolveLaterStepRunsParams)
@@ -402,7 +421,7 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
return err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
return err
})
@@ -414,18 +433,18 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
return stepRun, updateInfo, nil
}
func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId string, opts *repository.UpdateStepRunOverridesDataOpts) ([]byte, error) {
func (s *stepRunEngineRepository) UpdateStepRunOverridesData(ctx context.Context, tenantId, stepRunId string, opts *repository.UpdateStepRunOverridesDataOpts) ([]byte, error) {
if err := s.v.Validate(opts); err != nil {
return nil, err
}
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
if err != nil {
return nil, err
@@ -441,7 +460,7 @@ func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId
}
input, err := s.queries.UpdateStepRunOverridesData(
context.Background(),
ctx,
tx,
dbsqlc.UpdateStepRunOverridesDataParams{
Steprunid: pgStepRunId,
@@ -462,7 +481,7 @@ func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId
return nil, fmt.Errorf("could not update step run overrides data: %w", err)
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -471,20 +490,20 @@ func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId
return input, nil
}
func (s *stepRunEngineRepository) UpdateStepRunInputSchema(tenantId, stepRunId string, schema []byte) ([]byte, error) {
tx, err := s.pool.Begin(context.Background())
func (s *stepRunEngineRepository) UpdateStepRunInputSchema(ctx context.Context, tenantId, stepRunId string, schema []byte) ([]byte, error) {
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
pgStepRunId := sqlchelpers.UUIDFromStr(stepRunId)
inputSchema, err := s.queries.UpdateStepRunInputSchema(
context.Background(),
ctx,
tx,
dbsqlc.UpdateStepRunInputSchemaParams{
Steprunid: pgStepRunId,
@@ -497,7 +516,7 @@ func (s *stepRunEngineRepository) UpdateStepRunInputSchema(tenantId, stepRunId s
return nil, fmt.Errorf("could not update step run input schema: %w", err)
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -524,16 +543,16 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
var isNotPending bool
retrierErr := retrier(s.l, func() error {
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
// get the step run and make sure it's still in pending
innerStepRun, err := s.queries.GetStepRun(context.Background(), tx, dbsqlc.GetStepRunParams{
innerStepRun, err := s.queries.GetStepRun(ctx, tx, dbsqlc.GetStepRunParams{
ID: sqlchelpers.UUIDFromStr(stepRunId),
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
})
@@ -560,7 +579,7 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
return err
}
if err := tx.Commit(context.Background()); err != nil {
if err := tx.Commit(ctx); err != nil {
return err
}
@@ -576,13 +595,13 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
}
retrierExtraErr := retrier(s.l, func() error {
tx, err := s.pool.Begin(context.Background())
tx, err := s.pool.Begin(ctx)
if err != nil {
return err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
_, err = s.updateStepRunExtra(ctx, tx, tenantId, resolveJobRunParams, resolveLaterStepRunsParams)
@@ -590,7 +609,7 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
return err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
return err
})
@@ -763,13 +782,13 @@ func (s *stepRunEngineRepository) updateStepRunExtra(
ctx, span := telemetry.NewSpan(ctx, "update-step-run-extra") // nolint:ineffassign
defer span.End()
_, err := s.queries.ResolveLaterStepRuns(context.Background(), tx, resolveLaterStepRunsParams)
_, err := s.queries.ResolveLaterStepRuns(ctx, tx, resolveLaterStepRunsParams)
if err != nil {
return nil, fmt.Errorf("could not resolve later step runs: %w", err)
}
jobRun, err := s.queries.ResolveJobRunStatus(context.Background(), tx, resolveJobRunParams)
jobRun, err := s.queries.ResolveJobRunStatus(ctx, tx, resolveJobRunParams)
if err != nil {
return nil, fmt.Errorf("could not resolve job run status: %w", err)
@@ -780,7 +799,7 @@ func (s *stepRunEngineRepository) updateStepRunExtra(
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
}
workflowRun, err := s.queries.ResolveWorkflowRunStatus(context.Background(), tx, resolveWorkflowRunParams)
workflowRun, err := s.queries.ResolveWorkflowRunStatus(ctx, tx, resolveWorkflowRunParams)
if err != nil {
return nil, fmt.Errorf("could not resolve workflow run status: %w", err)
@@ -810,8 +829,8 @@ func isFinalWorkflowRunStatus(status dbsqlc.WorkflowRunStatus) bool {
}
// performant query for step run id, only returns what the engine needs
func (s *stepRunEngineRepository) GetStepRunForEngine(tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error) {
res, err := s.queries.GetStepRunForEngine(context.Background(), s.pool, dbsqlc.GetStepRunForEngineParams{
func (s *stepRunEngineRepository) GetStepRunForEngine(ctx context.Context, tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error) {
res, err := s.queries.GetStepRunForEngine(ctx, s.pool, dbsqlc.GetStepRunForEngineParams{
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(stepRunId)},
TenantId: sqlchelpers.UUIDFromStr(tenantId),
})
@@ -827,14 +846,14 @@ func (s *stepRunEngineRepository) GetStepRunForEngine(tenantId, stepRunId string
return res[0], nil
}
func (s *stepRunEngineRepository) ListStartableStepRuns(tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
tx, err := s.pool.Begin(context.Background())
func (s *stepRunEngineRepository) ListStartableStepRuns(ctx context.Context, tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
tx, err := s.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), s.l, tx.Rollback)
defer deferRollback(ctx, s.l, tx.Rollback)
params := dbsqlc.ListStartableStepRunsParams{
Jobrunid: sqlchelpers.UUIDFromStr(jobRunId),
@@ -844,13 +863,13 @@ func (s *stepRunEngineRepository) ListStartableStepRuns(tenantId, jobRunId strin
params.ParentStepRunId = sqlchelpers.UUIDFromStr(*parentStepRunId)
}
srs, err := s.queries.ListStartableStepRuns(context.Background(), tx, params)
srs, err := s.queries.ListStartableStepRuns(ctx, tx, params)
if err != nil {
return nil, err
}
res, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
res, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
Ids: srs,
TenantId: sqlchelpers.UUIDFromStr(tenantId),
})
@@ -859,13 +878,13 @@ func (s *stepRunEngineRepository) ListStartableStepRuns(tenantId, jobRunId strin
return nil, err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
return res, err
}
func (s *stepRunEngineRepository) ArchiveStepRunResult(tenantId, stepRunId string) error {
_, err := s.queries.ArchiveStepRunResultFromStepRun(context.Background(), s.pool, dbsqlc.ArchiveStepRunResultFromStepRunParams{
func (s *stepRunEngineRepository) ArchiveStepRunResult(ctx context.Context, tenantId, stepRunId string) error {
_, err := s.queries.ArchiveStepRunResultFromStepRun(ctx, s.pool, dbsqlc.ArchiveStepRunResultFromStepRunParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Steprunid: sqlchelpers.UUIDFromStr(stepRunId),
})

View File

@@ -33,7 +33,7 @@ func NewStreamEventsEngineRepository(pool *pgxpool.Pool, v validator.Validator,
}
}
func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repository.CreateStreamEventOpts) (*dbsqlc.StreamEvent, error) {
func (r *streamEventEngineRepository) PutStreamEvent(ctx context.Context, tenantId string, opts *repository.CreateStreamEventOpts) (*dbsqlc.StreamEvent, error) {
if err := r.v.Validate(opts); err != nil {
return nil, err
}
@@ -53,16 +53,16 @@ func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repo
createParams.Metadata = opts.Metadata
}
tx, err := r.pool.Begin(context.Background())
tx, err := r.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), r.l, tx.Rollback)
defer deferRollback(ctx, r.l, tx.Rollback)
streamEvent, err := r.queries.CreateStreamEvent(
context.Background(),
ctx,
tx,
createParams,
)
@@ -71,7 +71,7 @@ func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repo
return nil, fmt.Errorf("could not create stream event: %w", err)
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, fmt.Errorf("could not commit transaction: %w", err)
@@ -80,16 +80,16 @@ func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repo
return streamEvent, nil
}
func (r *streamEventEngineRepository) GetStreamEvent(tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error) {
func (r *streamEventEngineRepository) GetStreamEvent(ctx context.Context, tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error) {
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
tx, err := r.pool.Begin(context.Background())
tx, err := r.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), r.l, tx.Rollback)
defer deferRollback(ctx, r.l, tx.Rollback)
streamEvent, err := r.queries.GetStreamEvent(context.Background(), tx, dbsqlc.GetStreamEventParams{
streamEvent, err := r.queries.GetStreamEvent(ctx, tx, dbsqlc.GetStreamEventParams{
ID: streamEventId,
Tenantid: pgTenantId,
})
@@ -101,7 +101,7 @@ func (r *streamEventEngineRepository) GetStreamEvent(tenantId string, streamEven
return nil, fmt.Errorf("could not get stream event: %w", err)
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, fmt.Errorf("could not commit transaction: %w", err)
@@ -110,15 +110,15 @@ func (r *streamEventEngineRepository) GetStreamEvent(tenantId string, streamEven
return streamEvent, nil
}
func (r *streamEventEngineRepository) CleanupStreamEvents() error {
tx, err := r.pool.Begin(context.Background())
func (r *streamEventEngineRepository) CleanupStreamEvents(ctx context.Context) error {
tx, err := r.pool.Begin(ctx)
if err != nil {
return err
}
defer deferRollback(context.Background(), r.l, tx.Rollback)
defer deferRollback(ctx, r.l, tx.Rollback)
err = r.queries.CleanupStreamEvents(context.Background(), r.pool)
err = r.queries.CleanupStreamEvents(ctx, r.pool)
if err != nil {
return fmt.Errorf("could not cleanup stream events: %w", err)

View File

@@ -151,12 +151,12 @@ func NewTenantEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zer
}
}
func (r *tenantEngineRepository) ListTenants() ([]*dbsqlc.Tenant, error) {
return r.queries.ListTenants(context.Background(), r.pool)
func (r *tenantEngineRepository) ListTenants(ctx context.Context) ([]*dbsqlc.Tenant, error) {
return r.queries.ListTenants(ctx, r.pool)
}
func (r *tenantEngineRepository) GetTenantByID(tenantId string) (*dbsqlc.Tenant, error) {
func (r *tenantEngineRepository) GetTenantByID(ctx context.Context, tenantId string) (*dbsqlc.Tenant, error) {
return cache.MakeCacheable[dbsqlc.Tenant](r.cache, tenantId, func() (*dbsqlc.Tenant, error) {
return r.queries.GetTenantByID(context.Background(), r.pool, sqlchelpers.UUIDFromStr(tenantId))
return r.queries.GetTenantByID(ctx, r.pool, sqlchelpers.UUIDFromStr(tenantId))
})
}

View File

@@ -30,21 +30,21 @@ func NewTickerRepository(pool *pgxpool.Pool, v validator.Validator, l *zerolog.L
}
}
func (t *tickerRepository) CreateNewTicker(opts *repository.CreateTickerOpts) (*dbsqlc.Ticker, error) {
func (t *tickerRepository) CreateNewTicker(ctx context.Context, opts *repository.CreateTickerOpts) (*dbsqlc.Ticker, error) {
if err := t.v.Validate(opts); err != nil {
return nil, err
}
return t.queries.CreateTicker(context.Background(), t.pool, sqlchelpers.UUIDFromStr(opts.ID))
return t.queries.CreateTicker(ctx, t.pool, sqlchelpers.UUIDFromStr(opts.ID))
}
func (t *tickerRepository) UpdateTicker(tickerId string, opts *repository.UpdateTickerOpts) (*dbsqlc.Ticker, error) {
func (t *tickerRepository) UpdateTicker(ctx context.Context, tickerId string, opts *repository.UpdateTickerOpts) (*dbsqlc.Ticker, error) {
if err := t.v.Validate(opts); err != nil {
return nil, err
}
return t.queries.UpdateTicker(
context.Background(),
ctx,
t.pool,
dbsqlc.UpdateTickerParams{
ID: sqlchelpers.UUIDFromStr(tickerId),
@@ -53,7 +53,7 @@ func (t *tickerRepository) UpdateTicker(tickerId string, opts *repository.Update
)
}
func (t *tickerRepository) ListTickers(opts *repository.ListTickerOpts) ([]*dbsqlc.Ticker, error) {
func (t *tickerRepository) ListTickers(ctx context.Context, opts *repository.ListTickerOpts) ([]*dbsqlc.Ticker, error) {
if err := t.v.Validate(opts); err != nil {
return nil, err
}
@@ -69,15 +69,15 @@ func (t *tickerRepository) ListTickers(opts *repository.ListTickerOpts) ([]*dbsq
}
return t.queries.ListTickers(
context.Background(),
ctx,
t.pool,
params,
)
}
func (t *tickerRepository) Delete(tickerId string) error {
func (t *tickerRepository) Delete(ctx context.Context, tickerId string) error {
_, err := t.queries.DeleteTicker(
context.Background(),
ctx,
t.pool,
sqlchelpers.UUIDFromStr(tickerId),
)
@@ -85,163 +85,18 @@ func (t *tickerRepository) Delete(tickerId string) error {
return err
}
func (t *tickerRepository) PollStepRuns(tickerId string) ([]*dbsqlc.StepRun, error) {
return t.queries.PollStepRuns(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
func (t *tickerRepository) PollStepRuns(ctx context.Context, tickerId string) ([]*dbsqlc.StepRun, error) {
return t.queries.PollStepRuns(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
}
func (t *tickerRepository) PollGetGroupKeyRuns(tickerId string) ([]*dbsqlc.GetGroupKeyRun, error) {
return t.queries.PollGetGroupKeyRuns(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
func (t *tickerRepository) PollGetGroupKeyRuns(ctx context.Context, tickerId string) ([]*dbsqlc.GetGroupKeyRun, error) {
return t.queries.PollGetGroupKeyRuns(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
}
func (t *tickerRepository) PollCronSchedules(tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error) {
return t.queries.PollCronSchedules(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
func (t *tickerRepository) PollCronSchedules(ctx context.Context, tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error) {
return t.queries.PollCronSchedules(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
}
func (t *tickerRepository) PollScheduledWorkflows(tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error) {
return t.queries.PollScheduledWorkflows(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
func (t *tickerRepository) PollScheduledWorkflows(ctx context.Context, tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error) {
return t.queries.PollScheduledWorkflows(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
}
// func (t *tickerRepository) AddJobRun(tickerId string, jobRun *db.JobRunModel) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.JobRuns.Link(
// db.JobRun.ID.Equals(jobRun.ID),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) AddStepRun(tickerId, stepRunId string) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.StepRuns.Link(
// db.StepRun.ID.Equals(stepRunId),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) AddGetGroupKeyRun(tickerId, getGroupKeyRunId string) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.GroupKeyRuns.Link(
// db.GetGroupKeyRun.ID.Equals(getGroupKeyRunId),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) AddCron(tickerId string, cron *db.WorkflowTriggerCronRefModel) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.Crons.Link(
// db.WorkflowTriggerCronRef.ParentIDCron(
// db.WorkflowTriggerCronRef.ParentID.Equals(cron.ParentID),
// db.WorkflowTriggerCronRef.Cron.Equals(cron.Cron),
// ),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) RemoveCron(tickerId string, cron *db.WorkflowTriggerCronRefModel) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.Crons.Unlink(
// db.WorkflowTriggerCronRef.ParentIDCron(
// db.WorkflowTriggerCronRef.ParentID.Equals(cron.ParentID),
// db.WorkflowTriggerCronRef.Cron.Equals(cron.Cron),
// ),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) AddScheduledWorkflow(tickerId string, schedule *db.WorkflowTriggerScheduledRefModel) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.Scheduled.Link(
// db.WorkflowTriggerScheduledRef.ID.Equals(schedule.ID),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) RemoveScheduledWorkflow(tickerId string, schedule *db.WorkflowTriggerScheduledRefModel) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).Update(
// db.Ticker.Scheduled.Unlink(
// db.WorkflowTriggerScheduledRef.ID.Equals(schedule.ID),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) GetTickerById(tickerId string) (*db.TickerModel, error) {
// return t.client.Ticker.FindUnique(
// db.Ticker.ID.Equals(tickerId),
// ).With(
// db.Ticker.Crons.Fetch().With(
// db.WorkflowTriggerCronRef.Parent.Fetch().With(
// db.WorkflowTriggers.Workflow.Fetch().With(
// db.WorkflowVersion.Workflow.Fetch(),
// ),
// ),
// ),
// db.Ticker.Scheduled.Fetch().With(
// db.WorkflowTriggerScheduledRef.Parent.Fetch().With(
// db.WorkflowVersion.Workflow.Fetch(),
// ),
// ),
// ).Exec(context.Background())
// }
// func (t *tickerRepository) UpdateStaleTickers(onStale func(tickerId string, getValidTickerId func() string) error) error {
// tx, err := t.pool.Begin(context.Background())
// if err != nil {
// return err
// }
// defer deferRollback(context.Background(), t.l, tx.Rollback)
// staleTickers, err := t.queries.ListNewlyStaleTickers(context.Background(), tx)
// if err != nil {
// return err
// }
// activeTickers, err := t.queries.ListActiveTickers(context.Background(), tx)
// if err != nil {
// return err
// }
// // if there are no active tickers, we can't reassign the stale tickers
// if len(activeTickers) == 0 {
// return nil
// }
// tickersToDelete := make([]pgtype.UUID, 0)
// for i, ticker := range staleTickers {
// err := onStale(sqlchelpers.UUIDToStr(ticker.Ticker.ID), func() string {
// // assign tickers in round-robin fashion
// return sqlchelpers.UUIDToStr(activeTickers[i%len(activeTickers)].Ticker.ID)
// })
// if err != nil {
// return err
// }
// tickersToDelete = append(tickersToDelete, ticker.Ticker.ID)
// }
// _, err = t.queries.SetTickersInactive(context.Background(), tx, tickersToDelete)
// if err != nil {
// return err
// }
// return tx.Commit(context.Background())
// }

View File

@@ -138,25 +138,25 @@ func NewWorkerEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zer
}
}
func (w *workerEngineRepository) GetWorkerForEngine(tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error) {
return w.queries.GetWorkerForEngine(context.Background(), w.pool, dbsqlc.GetWorkerForEngineParams{
func (w *workerEngineRepository) GetWorkerForEngine(ctx context.Context, tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error) {
return w.queries.GetWorkerForEngine(ctx, w.pool, dbsqlc.GetWorkerForEngineParams{
ID: sqlchelpers.UUIDFromStr(workerId),
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
})
}
func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *repository.CreateWorkerOpts) (*dbsqlc.Worker, error) {
func (w *workerEngineRepository) CreateNewWorker(ctx context.Context, tenantId string, opts *repository.CreateWorkerOpts) (*dbsqlc.Worker, error) {
if err := w.v.Validate(opts); err != nil {
return nil, err
}
tx, err := w.pool.Begin(context.Background())
tx, err := w.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), w.l, tx.Rollback)
defer deferRollback(ctx, w.l, tx.Rollback)
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
@@ -180,7 +180,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
}
}
worker, err := w.queries.CreateWorker(context.Background(), tx, createParams)
worker, err := w.queries.CreateWorker(ctx, tx, createParams)
if err != nil {
return nil, fmt.Errorf("could not create worker: %w", err)
@@ -188,7 +188,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
createSemParams.Workerid = worker.ID
_, err = w.queries.CreateWorkerSemaphore(context.Background(), tx, createSemParams)
_, err = w.queries.CreateWorkerSemaphore(ctx, tx, createSemParams)
if err != nil {
return nil, fmt.Errorf("could not create worker semaphore: %w", err)
@@ -197,7 +197,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
svcUUIDs := make([]pgtype.UUID, len(opts.Services))
for i, svc := range opts.Services {
dbSvc, err := w.queries.UpsertService(context.Background(), tx, dbsqlc.UpsertServiceParams{
dbSvc, err := w.queries.UpsertService(ctx, tx, dbsqlc.UpsertServiceParams{
Name: svc,
Tenantid: pgTenantId,
})
@@ -209,7 +209,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
svcUUIDs[i] = dbSvc.ID
}
err = w.queries.LinkServicesToWorker(context.Background(), tx, dbsqlc.LinkServicesToWorkerParams{
err = w.queries.LinkServicesToWorker(ctx, tx, dbsqlc.LinkServicesToWorkerParams{
Services: svcUUIDs,
Workerid: worker.ID,
})
@@ -221,7 +221,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
actionUUIDs := make([]pgtype.UUID, len(opts.Actions))
for i, action := range opts.Actions {
dbAction, err := w.queries.UpsertAction(context.Background(), tx, dbsqlc.UpsertActionParams{
dbAction, err := w.queries.UpsertAction(ctx, tx, dbsqlc.UpsertActionParams{
Action: action,
Tenantid: pgTenantId,
})
@@ -233,7 +233,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
actionUUIDs[i] = dbAction.ID
}
err = w.queries.LinkActionsToWorker(context.Background(), tx, dbsqlc.LinkActionsToWorkerParams{
err = w.queries.LinkActionsToWorker(ctx, tx, dbsqlc.LinkActionsToWorkerParams{
Actionids: actionUUIDs,
Workerid: worker.ID,
})
@@ -242,7 +242,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
return nil, fmt.Errorf("could not link actions to worker: %w", err)
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, fmt.Errorf("could not commit transaction: %w", err)
@@ -251,18 +251,18 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
return worker, nil
}
func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *repository.UpdateWorkerOpts) (*dbsqlc.Worker, error) {
func (w *workerEngineRepository) UpdateWorker(ctx context.Context, tenantId, workerId string, opts *repository.UpdateWorkerOpts) (*dbsqlc.Worker, error) {
if err := w.v.Validate(opts); err != nil {
return nil, err
}
tx, err := w.pool.Begin(context.Background())
tx, err := w.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), w.l, tx.Rollback)
defer deferRollback(ctx, w.l, tx.Rollback)
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
@@ -285,7 +285,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
updateParams.DispatcherId = sqlchelpers.UUIDFromStr(*opts.DispatcherId)
}
worker, err := w.queries.UpdateWorker(context.Background(), tx, updateParams)
worker, err := w.queries.UpdateWorker(ctx, tx, updateParams)
if err != nil {
return nil, fmt.Errorf("could not update worker: %w", err)
@@ -295,7 +295,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
actionUUIDs := make([]pgtype.UUID, len(opts.Actions))
for i, action := range opts.Actions {
dbAction, err := w.queries.UpsertAction(context.Background(), tx, dbsqlc.UpsertActionParams{
dbAction, err := w.queries.UpsertAction(ctx, tx, dbsqlc.UpsertActionParams{
Action: action,
Tenantid: pgTenantId,
})
@@ -307,7 +307,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
actionUUIDs[i] = dbAction.ID
}
err = w.queries.LinkActionsToWorker(context.Background(), tx, dbsqlc.LinkActionsToWorkerParams{
err = w.queries.LinkActionsToWorker(ctx, tx, dbsqlc.LinkActionsToWorkerParams{
Actionids: actionUUIDs,
Workerid: sqlchelpers.UUIDFromStr(workerId),
})
@@ -317,7 +317,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
}
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, fmt.Errorf("could not commit transaction: %w", err)
@@ -326,8 +326,8 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
return worker, nil
}
func (w *workerEngineRepository) DeleteWorker(tenantId, workerId string) error {
_, err := w.queries.DeleteWorker(context.Background(), w.pool, sqlchelpers.UUIDFromStr(workerId))
func (w *workerEngineRepository) DeleteWorker(ctx context.Context, tenantId, workerId string) error {
_, err := w.queries.DeleteWorker(ctx, w.pool, sqlchelpers.UUIDFromStr(workerId))
return err
}

View File

@@ -298,7 +298,7 @@ func NewWorkflowEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *z
}
}
func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
func (r *workflowEngineRepository) CreateNewWorkflow(ctx context.Context, tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
if err := r.v.Validate(opts); err != nil {
return nil, err
}
@@ -313,7 +313,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
}
// preflight check to ensure the workflow doesn't already exist
workflow, err := r.queries.GetWorkflowByName(context.Background(), r.pool, dbsqlc.GetWorkflowByNameParams{
workflow, err := r.queries.GetWorkflowByName(ctx, r.pool, dbsqlc.GetWorkflowByNameParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Name: opts.Name,
})
@@ -329,20 +329,20 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
)
}
tx, err := r.pool.Begin(context.Background())
tx, err := r.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), r.l, tx.Rollback)
defer deferRollback(ctx, r.l, tx.Rollback)
workflowId := sqlchelpers.UUIDFromStr(uuid.New().String())
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
// create a workflow
_, err = r.queries.CreateWorkflow(
context.Background(),
ctx,
tx,
dbsqlc.CreateWorkflowParams{
ID: workflowId,
@@ -366,7 +366,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
}
err = r.queries.UpsertWorkflowTag(
context.Background(),
ctx,
tx,
dbsqlc.UpsertWorkflowTagParams{
Tenantid: pgTenantId,
@@ -381,13 +381,13 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
}
}
workflowVersionId, err := r.createWorkflowVersionTxs(context.Background(), tx, pgTenantId, workflowId, opts)
workflowVersionId, err := r.createWorkflowVersionTxs(ctx, tx, pgTenantId, workflowId, opts)
if err != nil {
return nil, err
}
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(context.Background(), tx, dbsqlc.GetWorkflowVersionForEngineParams{
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(ctx, tx, dbsqlc.GetWorkflowVersionForEngineParams{
Tenantid: pgTenantId,
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(workflowVersionId)},
})
@@ -400,7 +400,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
return nil, fmt.Errorf("expected 1 workflow version when creating new, got %d", len(workflowVersion))
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -409,7 +409,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
return workflowVersion[0], nil
}
func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
func (r *workflowEngineRepository) CreateWorkflowVersion(ctx context.Context, tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
if err := r.v.Validate(opts); err != nil {
return nil, err
}
@@ -424,7 +424,7 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
}
// preflight check to ensure the workflow already exists
workflow, err := r.queries.GetWorkflowByName(context.Background(), r.pool, dbsqlc.GetWorkflowByNameParams{
workflow, err := r.queries.GetWorkflowByName(ctx, r.pool, dbsqlc.GetWorkflowByNameParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Name: opts.Name,
})
@@ -440,23 +440,23 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
)
}
tx, err := r.pool.Begin(context.Background())
tx, err := r.pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), r.l, tx.Rollback)
defer deferRollback(ctx, r.l, tx.Rollback)
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
workflowVersionId, err := r.createWorkflowVersionTxs(context.Background(), tx, pgTenantId, workflow.ID, opts)
workflowVersionId, err := r.createWorkflowVersionTxs(ctx, tx, pgTenantId, workflow.ID, opts)
if err != nil {
return nil, err
}
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(context.Background(), tx, dbsqlc.GetWorkflowVersionForEngineParams{
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(ctx, tx, dbsqlc.GetWorkflowVersionForEngineParams{
Tenantid: pgTenantId,
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(workflowVersionId)},
})
@@ -469,7 +469,7 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
return nil, fmt.Errorf("expected 1 workflow version when creating version, got %d", len(workflowVersion))
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -479,6 +479,7 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
}
func (r *workflowEngineRepository) CreateSchedules(
ctx context.Context,
tenantId, workflowVersionId string,
opts *repository.CreateWorkflowSchedulesOpts,
) ([]*dbsqlc.WorkflowTriggerScheduledRef, error) {
@@ -496,17 +497,17 @@ func (r *workflowEngineRepository) CreateSchedules(
createParams.Triggertimes[i] = sqlchelpers.TimestampFromTime(scheduledTrigger)
}
return r.queries.CreateSchedules(context.Background(), r.pool, createParams)
return r.queries.CreateSchedules(ctx, r.pool, createParams)
}
func (r *workflowEngineRepository) GetLatestWorkflowVersion(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
versionId, err := r.queries.GetWorkflowLatestVersion(context.Background(), r.pool, sqlchelpers.UUIDFromStr(workflowId))
func (r *workflowEngineRepository) GetLatestWorkflowVersion(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
versionId, err := r.queries.GetWorkflowLatestVersion(ctx, r.pool, sqlchelpers.UUIDFromStr(workflowId))
if err != nil {
return nil, fmt.Errorf("failed to fetch latest version: %w", err)
}
versions, err := r.queries.GetWorkflowVersionForEngine(context.Background(), r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
versions, err := r.queries.GetWorkflowVersionForEngine(ctx, r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Ids: []pgtype.UUID{versionId},
})
@@ -522,15 +523,15 @@ func (r *workflowEngineRepository) GetLatestWorkflowVersion(tenantId, workflowId
return versions[0], nil
}
func (r *workflowEngineRepository) GetWorkflowByName(tenantId, workflowName string) (*dbsqlc.Workflow, error) {
return r.queries.GetWorkflowByName(context.Background(), r.pool, dbsqlc.GetWorkflowByNameParams{
func (r *workflowEngineRepository) GetWorkflowByName(ctx context.Context, tenantId, workflowName string) (*dbsqlc.Workflow, error) {
return r.queries.GetWorkflowByName(ctx, r.pool, dbsqlc.GetWorkflowByNameParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Name: workflowName,
})
}
func (r *workflowEngineRepository) GetWorkflowVersionById(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
versions, err := r.queries.GetWorkflowVersionForEngine(context.Background(), r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
func (r *workflowEngineRepository) GetWorkflowVersionById(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
versions, err := r.queries.GetWorkflowVersionForEngine(ctx, r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(workflowId)},
})
@@ -571,7 +572,7 @@ func (r *workflowEngineRepository) ListWorkflowsForEvent(ctx context.Context, te
ctx, span3 := telemetry.NewSpan(ctx, "db-get-workflow-versions-for-engine") // nolint: ineffassign
defer span3.End()
workflows, err := r.queries.GetWorkflowVersionForEngine(context.Background(), r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
workflows, err := r.queries.GetWorkflowVersionForEngine(ctx, r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Ids: workflowVersionIds,
})
@@ -610,7 +611,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
}
sqlcWorkflowVersion, err := r.queries.CreateWorkflowVersion(
context.Background(),
ctx,
tx,
createParams,
)
@@ -623,7 +624,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
if opts.Concurrency != nil {
// upsert the action
action, err := r.queries.UpsertAction(
context.Background(),
ctx,
tx,
dbsqlc.UpsertActionParams{
Action: opts.Concurrency.Action,
@@ -659,7 +660,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
}
_, err = r.queries.CreateWorkflowConcurrency(
context.Background(),
ctx,
tx,
params,
)
@@ -682,7 +683,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
}
sqlcJob, err := r.queries.CreateJob(
context.Background(),
ctx,
tx,
dbsqlc.CreateJobParams{
ID: sqlchelpers.UUIDFromStr(jobId),
@@ -724,7 +725,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
// upsert the action
_, err := r.queries.UpsertAction(
context.Background(),
ctx,
tx,
dbsqlc.UpsertActionParams{
Action: stepOpts.Action,
@@ -752,7 +753,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
}
_, err = r.queries.CreateStep(
context.Background(),
ctx,
tx,
createStepParams,
)
@@ -763,7 +764,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
if len(stepOpts.Parents) > 0 {
err := r.queries.AddStepParents(
context.Background(),
ctx,
tx,
dbsqlc.AddStepParentsParams{
ID: sqlchelpers.UUIDFromStr(stepId),
@@ -780,7 +781,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
if len(stepOpts.RateLimits) > 0 {
for _, rateLimit := range stepOpts.RateLimits {
_, err := r.queries.CreateStepRateLimit(
context.Background(),
ctx,
tx,
dbsqlc.CreateStepRateLimitParams{
Stepid: sqlchelpers.UUIDFromStr(stepId),
@@ -802,7 +803,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
workflowTriggersId := uuid.New().String()
sqlcWorkflowTriggers, err := r.queries.CreateWorkflowTriggers(
context.Background(),
ctx,
tx,
dbsqlc.CreateWorkflowTriggersParams{
ID: sqlchelpers.UUIDFromStr(workflowTriggersId),
@@ -817,7 +818,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
for _, eventTrigger := range opts.EventTriggers {
_, err := r.queries.CreateWorkflowTriggerEventRef(
context.Background(),
ctx,
tx,
dbsqlc.CreateWorkflowTriggerEventRefParams{
Workflowtriggersid: sqlcWorkflowTriggers.ID,
@@ -832,7 +833,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
for _, cronTrigger := range opts.CronTriggers {
_, err := r.queries.CreateWorkflowTriggerCronRef(
context.Background(),
ctx,
tx,
dbsqlc.CreateWorkflowTriggerCronRefParams{
Workflowtriggersid: sqlcWorkflowTriggers.ID,
@@ -848,7 +849,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
for _, scheduledTrigger := range opts.ScheduledTriggers {
_, err := r.queries.CreateWorkflowTriggerScheduledRef(
context.Background(),
ctx,
tx,
dbsqlc.CreateWorkflowTriggerScheduledRefParams{
Workflowversionid: sqlcWorkflowVersion.ID,

View File

@@ -138,8 +138,8 @@ func NewWorkflowRunEngineRepository(pool *pgxpool.Pool, v validator.Validator, l
}
}
func (w *workflowRunEngineRepository) GetWorkflowRunById(tenantId, id string) (*dbsqlc.GetWorkflowRunRow, error) {
runs, err := w.queries.GetWorkflowRun(context.Background(), w.pool, dbsqlc.GetWorkflowRunParams{
func (w *workflowRunEngineRepository) GetWorkflowRunById(ctx context.Context, tenantId, id string) (*dbsqlc.GetWorkflowRunRow, error) {
runs, err := w.queries.GetWorkflowRun(ctx, w.pool, dbsqlc.GetWorkflowRunParams{
Ids: []pgtype.UUID{
sqlchelpers.UUIDFromStr(id),
},
@@ -157,15 +157,15 @@ func (w *workflowRunEngineRepository) GetWorkflowRunById(tenantId, id string) (*
return runs[0], nil
}
func (w *workflowRunEngineRepository) ListWorkflowRuns(tenantId string, opts *repository.ListWorkflowRunsOpts) (*repository.ListWorkflowRunsResult, error) {
func (w *workflowRunEngineRepository) ListWorkflowRuns(ctx context.Context, tenantId string, opts *repository.ListWorkflowRunsOpts) (*repository.ListWorkflowRunsResult, error) {
if err := w.v.Validate(opts); err != nil {
return nil, err
}
return listWorkflowRuns(context.Background(), w.pool, w.queries, w.l, tenantId, opts)
return listWorkflowRuns(ctx, w.pool, w.queries, w.l, tenantId, opts)
}
func (w *workflowRunEngineRepository) GetChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error) {
func (w *workflowRunEngineRepository) GetChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error) {
params := dbsqlc.GetChildWorkflowRunParams{
Parentid: sqlchelpers.UUIDFromStr(parentId),
Parentsteprunid: sqlchelpers.UUIDFromStr(parentStepRunId),
@@ -179,10 +179,10 @@ func (w *workflowRunEngineRepository) GetChildWorkflowRun(parentId, parentStepRu
params.ChildKey = sqlchelpers.TextFromStr(*childkey)
}
return w.queries.GetChildWorkflowRun(context.Background(), w.pool, params)
return w.queries.GetChildWorkflowRun(ctx, w.pool, params)
}
func (w *workflowRunEngineRepository) GetScheduledChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error) {
func (w *workflowRunEngineRepository) GetScheduledChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error) {
params := dbsqlc.GetScheduledChildWorkflowRunParams{
Parentid: sqlchelpers.UUIDFromStr(parentId),
Parentsteprunid: sqlchelpers.UUIDFromStr(parentStepRunId),
@@ -196,11 +196,11 @@ func (w *workflowRunEngineRepository) GetScheduledChildWorkflowRun(parentId, par
params.ChildKey = sqlchelpers.TextFromStr(*childkey)
}
return w.queries.GetScheduledChildWorkflowRun(context.Background(), w.pool, params)
return w.queries.GetScheduledChildWorkflowRun(ctx, w.pool, params)
}
func (w *workflowRunEngineRepository) PopWorkflowRunsRoundRobin(tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error) {
return w.queries.PopWorkflowRunsRoundRobin(context.Background(), w.pool, dbsqlc.PopWorkflowRunsRoundRobinParams{
func (w *workflowRunEngineRepository) PopWorkflowRunsRoundRobin(ctx context.Context, tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error) {
return w.queries.PopWorkflowRunsRoundRobin(ctx, w.pool, dbsqlc.PopWorkflowRunsRoundRobinParams{
Maxruns: int32(maxRuns),
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
Workflowid: sqlchelpers.UUIDFromStr(workflowId),
@@ -254,6 +254,17 @@ func listWorkflowRuns(ctx context.Context, pool *pgxpool.Pool, queries *dbsqlc.Q
countParams.WorkflowVersionId = pgWorkflowVersionId
}
if opts.Ids != nil && len(opts.Ids) > 0 {
pgIds := make([]pgtype.UUID, len(opts.Ids))
for i, id := range opts.Ids {
pgIds[i] = sqlchelpers.UUIDFromStr(id)
}
queryParams.Ids = pgIds
countParams.Ids = pgIds
}
if opts.ParentId != nil {
pgParentId := sqlchelpers.UUIDFromStr(*opts.ParentId)
@@ -305,27 +316,27 @@ func listWorkflowRuns(ctx context.Context, pool *pgxpool.Pool, queries *dbsqlc.Q
queryParams.Orderby = orderByField + " " + orderByDirection
tx, err := pool.Begin(context.Background())
tx, err := pool.Begin(ctx)
if err != nil {
return nil, err
}
defer deferRollback(context.Background(), l, tx.Rollback)
defer deferRollback(ctx, l, tx.Rollback)
workflowRuns, err := queries.ListWorkflowRuns(context.Background(), tx, queryParams)
workflowRuns, err := queries.ListWorkflowRuns(ctx, tx, queryParams)
if err != nil {
return nil, err
}
count, err := queries.CountWorkflowRuns(context.Background(), tx, countParams)
count, err := queries.CountWorkflowRuns(ctx, tx, countParams)
if err != nil && !errors.Is(err, pgx.ErrNoRows) {
return nil, err
}
err = tx.Commit(context.Background())
err = tx.Commit(ctx)
if err != nil {
return nil, err
@@ -354,7 +365,7 @@ func createNewWorkflowRun(ctx context.Context, pool *pgxpool.Pool, queries *dbsq
return nil, err
}
defer deferRollback(context.Background(), l, tx.Rollback)
defer deferRollback(ctx, l, tx.Rollback)
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)

View File

@@ -1,6 +1,10 @@
package repository
import "github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
import (
"context"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
)
type UpsertRateLimitOpts struct {
// The rate limit max value
@@ -12,5 +16,5 @@ type UpsertRateLimitOpts struct {
type RateLimitEngineRepository interface {
// CreateRateLimit creates a new rate limit record
UpsertRateLimit(tenantId string, key string, opts *UpsertRateLimitOpts) (*dbsqlc.RateLimit, error)
UpsertRateLimit(ctx context.Context, tenantId string, key string, opts *UpsertRateLimitOpts) (*dbsqlc.RateLimit, error)
}

View File

@@ -9,20 +9,10 @@ import (
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
)
type ListAllStepRunsOpts struct {
TickerId *string
NoTickerId *bool
Status *db.StepRunStatus
}
type ListStepRunsOpts struct {
JobRunId *string
WorkflowRunIds []string `validate:"dive,uuid"`
WorkflowRunId *string
Status *db.StepRunStatus
Status *dbsqlc.StepRunStatus
}
type UpdateStepRunOpts struct {
@@ -81,35 +71,32 @@ type StepRunAPIRepository interface {
}
type StepRunEngineRepository interface {
// ListRunningStepRunsForTicker returns a list of step runs which are currently running for a ticker.
ListRunningStepRunsForTicker(tickerId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
// ListRunningStepRunsForWorkflowRun returns a list of step runs which are currently running for a workflow run.
ListRunningStepRunsForWorkflowRun(tenantId, workflowRunId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
// ListStepRunsForWorkflowRun returns a list of step runs for a workflow run.
ListStepRuns(ctx context.Context, tenantId string, opts *ListStepRunsOpts) ([]*dbsqlc.GetStepRunForEngineRow, error)
// ListStepRunsToRequeue returns a list of step runs which are in a requeueable state.
ListStepRunsToRequeue(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
ListStepRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
// ListStepRunsToReassign returns a list of step runs which are in a reassignable state.
ListStepRunsToReassign(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
ListStepRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
UpdateStepRun(ctx context.Context, tenantId, stepRunId string, opts *UpdateStepRunOpts) (*dbsqlc.GetStepRunForEngineRow, *StepRunUpdateInfo, error)
// UpdateStepRunOverridesData updates the overrides data field in the input for a step run. This returns the input
// bytes.
UpdateStepRunOverridesData(tenantId, stepRunId string, opts *UpdateStepRunOverridesDataOpts) ([]byte, error)
UpdateStepRunOverridesData(ctx context.Context, tenantId, stepRunId string, opts *UpdateStepRunOverridesDataOpts) ([]byte, error)
UpdateStepRunInputSchema(tenantId, stepRunId string, schema []byte) ([]byte, error)
UpdateStepRunInputSchema(ctx context.Context, tenantId, stepRunId string, schema []byte) ([]byte, error)
AssignStepRunToWorker(ctx context.Context, stepRun *dbsqlc.GetStepRunForEngineRow) (workerId string, dispatcherId string, err error)
GetStepRunForEngine(tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error)
GetStepRunForEngine(ctx context.Context, tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error)
// QueueStepRun is like UpdateStepRun, except that it will only update the step run if it is in
// a pending state.
QueueStepRun(ctx context.Context, tenantId, stepRunId string, opts *UpdateStepRunOpts) (*dbsqlc.GetStepRunForEngineRow, error)
ListStartableStepRuns(tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error)
ListStartableStepRuns(ctx context.Context, tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error)
ArchiveStepRunResult(tenantId, stepRunId string) error
ArchiveStepRunResult(ctx context.Context, tenantId, stepRunId string) error
}

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
@@ -22,11 +23,11 @@ type CreateStreamEventOpts struct {
type StreamEventsEngineRepository interface {
// PutStreamEvent creates a new StreamEvent line.
PutStreamEvent(tenantId string, opts *CreateStreamEventOpts) (*dbsqlc.StreamEvent, error)
PutStreamEvent(ctx context.Context, tenantId string, opts *CreateStreamEventOpts) (*dbsqlc.StreamEvent, error)
// GetStreamEvent returns a StreamEvent line by id.
GetStreamEvent(tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error)
GetStreamEvent(ctx context.Context, tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error)
// CleanupStreamEvents deletes all stale StreamEvents.
CleanupStreamEvents() error
CleanupStreamEvents(ctx context.Context) error
}

View File

@@ -1,6 +1,8 @@
package repository
import (
"context"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
)
@@ -59,8 +61,8 @@ type TenantAPIRepository interface {
type TenantEngineRepository interface {
// ListTenants lists all tenants in the instance
ListTenants() ([]*dbsqlc.Tenant, error)
ListTenants(ctx context.Context) ([]*dbsqlc.Tenant, error)
// GetTenantByID returns the tenant with the given id
GetTenantByID(tenantId string) (*dbsqlc.Tenant, error)
GetTenantByID(ctx context.Context, tenantId string) (*dbsqlc.Tenant, error)
}

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
@@ -23,27 +24,27 @@ type ListTickerOpts struct {
type TickerEngineRepository interface {
// CreateNewTicker creates a new ticker.
CreateNewTicker(opts *CreateTickerOpts) (*dbsqlc.Ticker, error)
CreateNewTicker(ctx context.Context, opts *CreateTickerOpts) (*dbsqlc.Ticker, error)
// UpdateTicker updates a ticker.
UpdateTicker(tickerId string, opts *UpdateTickerOpts) (*dbsqlc.Ticker, error)
UpdateTicker(ctx context.Context, tickerId string, opts *UpdateTickerOpts) (*dbsqlc.Ticker, error)
// ListTickers lists tickers.
ListTickers(opts *ListTickerOpts) ([]*dbsqlc.Ticker, error)
ListTickers(ctx context.Context, opts *ListTickerOpts) ([]*dbsqlc.Ticker, error)
// Delete deletes a ticker.
Delete(tickerId string) error
Delete(ctx context.Context, tickerId string) error
// PollStepRuns looks for step runs who are close to past their timeoutAt value and are in a running state
PollStepRuns(tickerId string) ([]*dbsqlc.StepRun, error)
PollStepRuns(ctx context.Context, tickerId string) ([]*dbsqlc.StepRun, error)
// PollJobRuns looks for get group key runs who are close to past their timeoutAt value and are in a running state
PollGetGroupKeyRuns(tickerId string) ([]*dbsqlc.GetGroupKeyRun, error)
PollGetGroupKeyRuns(ctx context.Context, tickerId string) ([]*dbsqlc.GetGroupKeyRun, error)
// PollCronSchedules returns all cron schedules which should be managed by the ticker
PollCronSchedules(tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error)
PollCronSchedules(ctx context.Context, tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error)
PollScheduledWorkflows(tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error)
PollScheduledWorkflows(ctx context.Context, tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error)
// // AddJobRun assigns a job run to a ticker.
// AddJobRun(tickerId string, jobRun *db.JobRunModel) (*db.TickerModel, error)

View File

@@ -1,6 +1,7 @@
package repository
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
@@ -64,13 +65,13 @@ type WorkerAPIRepository interface {
type WorkerEngineRepository interface {
// CreateNewWorker creates a new worker for a given tenant.
CreateNewWorker(tenantId string, opts *CreateWorkerOpts) (*dbsqlc.Worker, error)
CreateNewWorker(ctx context.Context, tenantId string, opts *CreateWorkerOpts) (*dbsqlc.Worker, error)
// UpdateWorker updates a worker for a given tenant.
UpdateWorker(tenantId, workerId string, opts *UpdateWorkerOpts) (*dbsqlc.Worker, error)
UpdateWorker(ctx context.Context, tenantId, workerId string, opts *UpdateWorkerOpts) (*dbsqlc.Worker, error)
// DeleteWorker removes the worker from the database
DeleteWorker(tenantId, workerId string) error
DeleteWorker(ctx context.Context, tenantId, workerId string) error
GetWorkerForEngine(tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error)
GetWorkerForEngine(ctx context.Context, tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error)
}

View File

@@ -215,22 +215,22 @@ type WorkflowAPIRepository interface {
type WorkflowEngineRepository interface {
// CreateNewWorkflow creates a new workflow for a given tenant. It will create the parent
// workflow based on the version's name.
CreateNewWorkflow(tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
CreateNewWorkflow(ctx context.Context, tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
// CreateWorkflowVersion creates a new workflow version for a given tenant. This will fail if there is
// not a parent workflow with the same name already in the database.
CreateWorkflowVersion(tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
CreateWorkflowVersion(ctx context.Context, tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
// CreateSchedules creates schedules for a given workflow version.
CreateSchedules(tenantId, workflowVersionId string, opts *CreateWorkflowSchedulesOpts) ([]*dbsqlc.WorkflowTriggerScheduledRef, error)
CreateSchedules(ctx context.Context, tenantId, workflowVersionId string, opts *CreateWorkflowSchedulesOpts) ([]*dbsqlc.WorkflowTriggerScheduledRef, error)
// GetScheduledById returns a scheduled workflow by its id.
// GetScheduledById(tenantId, scheduleTriggerId string) (*db.WorkflowTriggerScheduledRefModel, error)
GetLatestWorkflowVersion(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
GetLatestWorkflowVersion(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
// GetWorkflowByName returns a workflow by its name. It will return db.ErrNotFound if the workflow does not exist.
GetWorkflowByName(tenantId, workflowName string) (*dbsqlc.Workflow, error)
GetWorkflowByName(ctx context.Context, tenantId, workflowName string) (*dbsqlc.Workflow, error)
// ListWorkflowsForEvent returns the latest workflow versions for a given tenant that are triggered by the
// given event.
@@ -238,5 +238,5 @@ type WorkflowEngineRepository interface {
// GetWorkflowVersionById returns a workflow version by its id. It will return db.ErrNotFound if the workflow
// version does not exist.
GetWorkflowVersionById(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
GetWorkflowVersionById(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
}

View File

@@ -202,6 +202,9 @@ type ListWorkflowRunsOpts struct {
// (optional) the workflow version id
WorkflowVersionId *string `validate:"omitempty,uuid"`
// (optional) a list of workflow run ids to filter by
Ids []string `validate:"omitempty,dive,uuid"`
// (optional) the parent workflow run id
ParentId *string `validate:"omitempty,uuid"`
@@ -284,17 +287,17 @@ type WorkflowRunAPIRepository interface {
type WorkflowRunEngineRepository interface {
// ListWorkflowRuns returns workflow runs for a given workflow version id.
ListWorkflowRuns(tenantId string, opts *ListWorkflowRunsOpts) (*ListWorkflowRunsResult, error)
ListWorkflowRuns(ctx context.Context, tenantId string, opts *ListWorkflowRunsOpts) (*ListWorkflowRunsResult, error)
GetChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error)
GetChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error)
GetScheduledChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error)
GetScheduledChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error)
PopWorkflowRunsRoundRobin(tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error)
PopWorkflowRunsRoundRobin(ctx context.Context, tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error)
// CreateNewWorkflowRun creates a new workflow run for a workflow version.
CreateNewWorkflowRun(ctx context.Context, tenantId string, opts *CreateWorkflowRunOpts) (string, error)
// GetWorkflowRunById returns a workflow run by id.
GetWorkflowRunById(tenantId, runId string) (*dbsqlc.GetWorkflowRunRow, error)
GetWorkflowRunById(ctx context.Context, tenantId, runId string) (*dbsqlc.GetWorkflowRunRow, error)
}

View File

@@ -43,6 +43,7 @@ func (a *AdminServiceImpl) TriggerWorkflow(ctx context.Context, req *contracts.T
}
workflowRun, err := a.repo.WorkflowRun().GetChildWorkflowRun(
ctx,
*req.ParentId,
*req.ParentStepRunId,
int(*req.ChildIndex),
@@ -63,6 +64,7 @@ func (a *AdminServiceImpl) TriggerWorkflow(ctx context.Context, req *contracts.T
}
workflow, err := a.repo.Workflow().GetWorkflowByName(
ctx,
tenantId,
req.Name,
)
@@ -79,6 +81,7 @@ func (a *AdminServiceImpl) TriggerWorkflow(ctx context.Context, req *contracts.T
}
workflowVersion, err := a.repo.Workflow().GetLatestWorkflowVersion(
ctx,
tenantId,
sqlchelpers.UUIDToStr(workflow.ID),
)
@@ -144,6 +147,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
var oldWorkflowVersion *dbsqlc.GetWorkflowVersionForEngineRow
currWorkflow, err := a.repo.Workflow().GetWorkflowByName(
ctx,
tenantId,
req.Opts.Name,
)
@@ -155,6 +159,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
// workflow does not exist, create it
workflowVersion, err = a.repo.Workflow().CreateNewWorkflow(
ctx,
tenantId,
createOpts,
)
@@ -164,6 +169,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
}
} else {
oldWorkflowVersion, err = a.repo.Workflow().GetLatestWorkflowVersion(
ctx,
tenantId,
sqlchelpers.UUIDToStr(currWorkflow.ID),
)
@@ -181,6 +187,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
if oldWorkflowVersion.WorkflowVersion.Checksum != newCS {
workflowVersion, err = a.repo.Workflow().CreateWorkflowVersion(
ctx,
tenantId,
createOpts,
)
@@ -203,6 +210,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
workflow, err := a.repo.Workflow().GetWorkflowByName(
ctx,
tenantId,
req.Name,
)
@@ -221,6 +229,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
workflowId := sqlchelpers.UUIDToStr(workflow.ID)
currWorkflow, err := a.repo.Workflow().GetLatestWorkflowVersion(
ctx,
tenantId,
workflowId,
)
@@ -251,6 +260,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
}
existing, err := a.repo.WorkflowRun().GetScheduledChildWorkflowRun(
ctx,
*req.ParentId,
*req.ParentStepRunId,
int(*req.ChildIndex),
@@ -277,6 +287,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
workflowVersionId := sqlchelpers.UUIDToStr(currWorkflow.WorkflowVersion.ID)
_, err = a.repo.Workflow().CreateSchedules(
ctx,
tenantId,
workflowVersionId,
&repository.CreateWorkflowSchedulesOpts{
@@ -313,7 +324,7 @@ func (a *AdminServiceImpl) PutRateLimit(ctx context.Context, req *contracts.PutR
Duration: &duration,
}
_, err := a.repo.RateLimit().UpsertRateLimit(tenantId, req.Key, createOpts)
_, err := a.repo.RateLimit().UpsertRateLimit(ctx, tenantId, req.Key, createOpts)
if err != nil {
return nil, err

View File

@@ -239,14 +239,14 @@ func (ec *JobsControllerImpl) handleJobRunQueued(ctx context.Context, task *msgq
return fmt.Errorf("could not decode job task metadata: %w", err)
}
err = ec.repo.JobRun().SetJobRunStatusRunning(metadata.TenantId, payload.JobRunId)
err = ec.repo.JobRun().SetJobRunStatusRunning(ctx, metadata.TenantId, payload.JobRunId)
if err != nil {
return fmt.Errorf("could not set job run status to running: %w", err)
}
// list the step runs which are startable
startableStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(metadata.TenantId, payload.JobRunId, nil)
startableStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(ctx, metadata.TenantId, payload.JobRunId, nil)
if err != nil {
return fmt.Errorf("could not list startable step runs: %w", err)
}
@@ -294,7 +294,7 @@ func (ec *JobsControllerImpl) handleStepRunRetry(ctx context.Context, task *msgq
return fmt.Errorf("could not decode job task metadata: %w", err)
}
err = ec.repo.StepRun().ArchiveStepRunResult(metadata.TenantId, payload.StepRunId)
err = ec.repo.StepRun().ArchiveStepRunResult(ctx, metadata.TenantId, payload.StepRunId)
if err != nil {
return fmt.Errorf("could not archive step run result: %w", err)
@@ -302,7 +302,7 @@ func (ec *JobsControllerImpl) handleStepRunRetry(ctx context.Context, task *msgq
ec.l.Error().Err(fmt.Errorf("starting step run retry"))
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
if err != nil {
return fmt.Errorf("could not get step run: %w", err)
@@ -405,10 +405,13 @@ func (ec *JobsControllerImpl) handleStepRunQueued(ctx context.Context, task *msg
func (jc *JobsControllerImpl) runStepRunRequeue(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
jc.l.Debug().Msgf("jobs controller: checking step run requeue")
// list all tenants
tenants, err := jc.repo.Tenant().ListTenants()
tenants, err := jc.repo.Tenant().ListTenants(ctx)
if err != nil {
jc.l.Err(err).Msg("could not list tenants")
@@ -438,7 +441,7 @@ func (ec *JobsControllerImpl) runStepRunRequeueTenant(ctx context.Context, tenan
ctx, span := telemetry.NewSpan(ctx, "handle-step-run-requeue")
defer span.End()
stepRuns, err := ec.repo.StepRun().ListStepRunsToRequeue(tenantId)
stepRuns, err := ec.repo.StepRun().ListStepRunsToRequeue(ctx, tenantId)
if err != nil {
return fmt.Errorf("could not list step runs to requeue: %w", err)
@@ -495,10 +498,13 @@ func (ec *JobsControllerImpl) runStepRunRequeueTenant(ctx context.Context, tenan
func (jc *JobsControllerImpl) runStepRunReassign(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
jc.l.Debug().Msgf("jobs controller: checking step run reassignment")
// list all tenants
tenants, err := jc.repo.Tenant().ListTenants()
tenants, err := jc.repo.Tenant().ListTenants(ctx)
if err != nil {
jc.l.Err(err).Msg("could not list tenants")
@@ -529,7 +535,7 @@ func (ec *JobsControllerImpl) runStepRunReassignTenant(ctx context.Context, tena
ctx, span := telemetry.NewSpan(ctx, "handle-step-run-reassign")
defer span.End()
stepRuns, err := ec.repo.StepRun().ListStepRunsToReassign(tenantId)
stepRuns, err := ec.repo.StepRun().ListStepRunsToReassign(ctx, tenantId)
if err != nil {
return fmt.Errorf("could not list step runs to reassign: %w", err)
@@ -589,7 +595,7 @@ func (ec *JobsControllerImpl) queueStepRun(ctx context.Context, tenantId, stepId
defer span.End()
// add the rendered data to the step run
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(tenantId, stepRunId)
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(ctx, tenantId, stepRunId)
errData := map[string]interface{}{
"tenant_id": tenantId,
@@ -798,7 +804,7 @@ func (ec *JobsControllerImpl) handleStepRunFinished(ctx context.Context, task *m
jobRunId := sqlchelpers.UUIDToStr(stepRun.JobRunId)
stepRunId := sqlchelpers.UUIDToStr(stepRun.StepRun.ID)
nextStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(metadata.TenantId, jobRunId, &stepRunId)
nextStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(ctx, metadata.TenantId, jobRunId, &stepRunId)
if err != nil {
return fmt.Errorf("could not list startable step runs: %w", err)
@@ -843,7 +849,7 @@ func (ec *JobsControllerImpl) handleStepRunFailed(ctx context.Context, task *msg
return fmt.Errorf("could not parse failed at: %w", err)
}
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
if err != nil {
return fmt.Errorf("could not get step run: %w", err)
@@ -957,7 +963,7 @@ func (ec *JobsControllerImpl) cancelStepRun(ctx context.Context, tenantId, stepR
workerId := sqlchelpers.UUIDToStr(stepRun.StepRun.WorkerId)
worker, err := ec.repo.Worker().GetWorkerForEngine(tenantId, workerId)
worker, err := ec.repo.Worker().GetWorkerForEngine(ctx, tenantId, workerId)
if err != nil {
return fmt.Errorf("could not get worker: %w", err)

View File

@@ -224,7 +224,7 @@ func (ec *WorkflowsControllerImpl) handleGroupKeyRunStarted(ctx context.Context,
return fmt.Errorf("could not parse started at: %w", err)
}
_, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
_, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
StartedAt: &startedAt,
Status: repository.StepRunStatusPtr(db.StepRunStatusRunning),
})
@@ -258,7 +258,7 @@ func (wc *WorkflowsControllerImpl) handleGroupKeyRunFinished(ctx context.Context
return fmt.Errorf("could not parse started at: %w", err)
}
groupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
groupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
FinishedAt: &finishedAt,
Status: repository.StepRunStatusPtr(db.StepRunStatusSucceeded),
Output: &payload.GroupKey,
@@ -272,7 +272,7 @@ func (wc *WorkflowsControllerImpl) handleGroupKeyRunFinished(ctx context.Context
errGroup.Go(func() error {
workflowVersionId := sqlchelpers.UUIDToStr(groupKeyRun.WorkflowVersionId)
workflowVersion, err := wc.repo.Workflow().GetWorkflowVersionById(metadata.TenantId, workflowVersionId)
workflowVersion, err := wc.repo.Workflow().GetWorkflowVersionById(ctx, metadata.TenantId, workflowVersionId)
if err != nil {
return fmt.Errorf("could not get workflow version: %w", err)
@@ -320,7 +320,7 @@ func (wc *WorkflowsControllerImpl) handleGroupKeyRunFailed(ctx context.Context,
return fmt.Errorf("could not parse started at: %w", err)
}
_, err = wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
_, err = wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
FinishedAt: &failedAt,
Error: &payload.Error,
Status: repository.StepRunStatusPtr(db.StepRunStatusFailed),
@@ -362,7 +362,7 @@ func (wc *WorkflowsControllerImpl) cancelGetGroupKeyRun(ctx context.Context, ten
// cancel current step run
now := time.Now().UTC()
_, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
_, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
CancelledAt: &now,
CancelledReason: repository.StringPtr(reason),
Status: repository.StepRunStatusPtr(db.StepRunStatusCancelled),

View File

@@ -40,7 +40,7 @@ func (wc *WorkflowsControllerImpl) handleWorkflowRunQueued(ctx context.Context,
}
// get the workflow run in the database
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId)
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, metadata.TenantId, payload.WorkflowRunId)
if err != nil {
return fmt.Errorf("could not get job run: %w", err)
@@ -63,7 +63,7 @@ func (wc *WorkflowsControllerImpl) handleWorkflowRunQueued(ctx context.Context,
return fmt.Errorf("could not get group key run")
}
sqlcGroupKeyRun, err := wc.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(metadata.TenantId, groupKeyRunId)
sqlcGroupKeyRun, err := wc.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(ctx, metadata.TenantId, groupKeyRunId)
if err != nil {
return fmt.Errorf("could not get group key run for engine: %w", err)
@@ -107,7 +107,7 @@ func (wc *WorkflowsControllerImpl) handleWorkflowRunFinished(ctx context.Context
}
// get the workflow run in the database
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId)
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, metadata.TenantId, payload.WorkflowRunId)
if err != nil {
return fmt.Errorf("could not get job run: %w", err)
@@ -159,7 +159,7 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
getGroupKeyRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.GetGroupKeyRun.ID)
workflowRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.WorkflowRunId)
getGroupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
getGroupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
Status: repository.StepRunStatusPtr(db.StepRunStatusPendingAssignment),
})
@@ -168,6 +168,7 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
}
selectedWorkerId, dispatcherId, err := wc.repo.GetGroupKeyRun().AssignGetGroupKeyRunToWorker(
ctx,
tenantId,
getGroupKeyRunId,
)
@@ -207,7 +208,7 @@ func (wc *WorkflowsControllerImpl) queueWorkflowRunJobs(ctx context.Context, wor
tenantId := sqlchelpers.UUIDToStr(workflowRun.WorkflowRun.TenantId)
workflowRunId := sqlchelpers.UUIDToStr(workflowRun.WorkflowRun.ID)
jobRuns, err := wc.repo.JobRun().ListJobRunsForWorkflowRun(tenantId, workflowRunId)
jobRuns, err := wc.repo.JobRun().ListJobRunsForWorkflowRun(ctx, tenantId, workflowRunId)
if err != nil {
return fmt.Errorf("could not list job runs: %w", err)
@@ -237,7 +238,7 @@ func (wc *WorkflowsControllerImpl) runGetGroupKeyRunRequeue(ctx context.Context)
wc.l.Debug().Msgf("workflows controller: checking get group key run requeue")
// list all tenants
tenants, err := wc.repo.Tenant().ListTenants()
tenants, err := wc.repo.Tenant().ListTenants(ctx)
if err != nil {
wc.l.Err(err).Msg("could not list tenants")
@@ -268,7 +269,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunRequeueTenant(ctx context.Co
ctx, span := telemetry.NewSpan(ctx, "handle-get-group-key-run-requeue")
defer span.End()
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToRequeue(tenantId)
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToRequeue(ctx, tenantId)
if err != nil {
return fmt.Errorf("could not list group key runs: %w", err)
@@ -300,7 +301,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunRequeueTenant(ctx context.Co
isTimedOut := !scheduleTimeoutAt.IsZero() && scheduleTimeoutAt.Before(now)
if isTimedOut {
_, err := ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
_, err := ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
CancelledAt: &now,
CancelledReason: repository.StringPtr("SCHEDULING_TIMED_OUT"),
Status: repository.StepRunStatusPtr(db.StepRunStatusCancelled),
@@ -315,7 +316,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunRequeueTenant(ctx context.Co
requeueAfter := time.Now().UTC().Add(time.Second * 4)
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
RequeueAfter: &requeueAfter,
})
@@ -335,7 +336,7 @@ func (wc *WorkflowsControllerImpl) runGetGroupKeyRunReassign(ctx context.Context
wc.l.Debug().Msgf("workflows controller: checking get group key run reassign")
// list all tenants
tenants, err := wc.repo.Tenant().ListTenants()
tenants, err := wc.repo.Tenant().ListTenants(ctx)
if err != nil {
wc.l.Err(err).Msg("could not list tenants")
@@ -365,7 +366,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunReassignTenant(ctx context.C
ctx, span := telemetry.NewSpan(ctx, "handle-get-group-key-run-reassign")
defer span.End()
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToReassign(tenantId)
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToReassign(ctx, tenantId)
if err != nil {
return fmt.Errorf("could not list get group key runs: %w", err)
@@ -389,7 +390,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunReassignTenant(ctx context.C
requeueAfter := time.Now().UTC().Add(time.Second * 4)
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
RequeueAfter: &requeueAfter,
Status: repository.StepRunStatusPtr(db.StepRunStatusPendingAssignment),
})
@@ -416,7 +417,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
workflowVersionId := sqlchelpers.UUIDToStr(workflowVersion.WorkflowVersion.ID)
runningWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(tenantId, &repository.ListWorkflowRunsOpts{
runningWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(ctx, tenantId, &repository.ListWorkflowRunsOpts{
WorkflowVersionId: &workflowVersionId,
GroupKey: &groupKey,
Statuses: &[]db.WorkflowRunStatus{running},
@@ -434,7 +435,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
maxRuns := int(workflowVersion.ConcurrencyMaxRuns.Int32)
queuedWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(tenantId, &repository.ListWorkflowRunsOpts{
queuedWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(ctx, tenantId, &repository.ListWorkflowRunsOpts{
WorkflowVersionId: &workflowVersionId,
GroupKey: &groupKey,
Statuses: &[]db.WorkflowRunStatus{queued},
@@ -462,7 +463,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
errGroup.Go(func() error {
workflowRunId := sqlchelpers.UUIDToStr(row.WorkflowRun.ID)
return wc.cancelWorkflowRun(tenantId, workflowRunId)
return wc.cancelWorkflowRun(ctx, tenantId, workflowRunId)
})
}
@@ -481,7 +482,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
errGroup.Go(func() error {
workflowRunId := sqlchelpers.UUIDToStr(row.WorkflowRun.ID)
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(tenantId, workflowRunId)
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, tenantId, workflowRunId)
if err != nil {
return fmt.Errorf("could not get workflow run: %w", err)
@@ -509,7 +510,7 @@ func (wc *WorkflowsControllerImpl) queueByGroupRoundRobin(ctx context.Context, t
wc.l.Info().Msgf("handling queue with strategy GROUP_ROUND_ROBIN for workflow version %s", workflowVersionId)
// get workflow runs which are queued for this group key
poppedWorkflowRuns, err := wc.repo.WorkflowRun().PopWorkflowRunsRoundRobin(tenantId, workflowId, maxRuns)
poppedWorkflowRuns, err := wc.repo.WorkflowRun().PopWorkflowRunsRoundRobin(ctx, tenantId, workflowId, maxRuns)
if err != nil {
return fmt.Errorf("could not list queued workflow runs: %w", err)
@@ -524,7 +525,7 @@ func (wc *WorkflowsControllerImpl) queueByGroupRoundRobin(ctx context.Context, t
workflowRunId := sqlchelpers.UUIDToStr(row.ID)
wc.l.Info().Msgf("popped workflow run %s", workflowRunId)
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(tenantId, workflowRunId)
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, tenantId, workflowRunId)
if err != nil {
return fmt.Errorf("could not get workflow run: %w", err)
@@ -541,9 +542,15 @@ func (wc *WorkflowsControllerImpl) queueByGroupRoundRobin(ctx context.Context, t
return nil
}
func (wc *WorkflowsControllerImpl) cancelWorkflowRun(tenantId, workflowRunId string) error {
func (wc *WorkflowsControllerImpl) cancelWorkflowRun(ctx context.Context, tenantId, workflowRunId string) error {
// cancel all running step runs
stepRuns, err := wc.repo.StepRun().ListRunningStepRunsForWorkflowRun(tenantId, workflowRunId)
runningStatus := dbsqlc.StepRunStatusRUNNING
stepRuns, err := wc.repo.StepRun().ListStepRuns(ctx, tenantId, &repository.ListStepRunsOpts{
Status: &runningStatus,
WorkflowRunIds: []string{
workflowRunId,
},
})
if err != nil {
return fmt.Errorf("could not list step runs: %w", err)

View File

@@ -284,6 +284,49 @@ func (ResourceEventType) EnumDescriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{4}
}
type WorkflowRunEventType int32
const (
WorkflowRunEventType_WORKFLOW_RUN_EVENT_TYPE_FINISHED WorkflowRunEventType = 0
)
// Enum value maps for WorkflowRunEventType.
var (
WorkflowRunEventType_name = map[int32]string{
0: "WORKFLOW_RUN_EVENT_TYPE_FINISHED",
}
WorkflowRunEventType_value = map[string]int32{
"WORKFLOW_RUN_EVENT_TYPE_FINISHED": 0,
}
)
func (x WorkflowRunEventType) Enum() *WorkflowRunEventType {
p := new(WorkflowRunEventType)
*p = x
return p
}
func (x WorkflowRunEventType) String() string {
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
}
func (WorkflowRunEventType) Descriptor() protoreflect.EnumDescriptor {
return file_dispatcher_proto_enumTypes[5].Descriptor()
}
func (WorkflowRunEventType) Type() protoreflect.EnumType {
return &file_dispatcher_proto_enumTypes[5]
}
func (x WorkflowRunEventType) Number() protoreflect.EnumNumber {
return protoreflect.EnumNumber(x)
}
// Deprecated: Use WorkflowRunEventType.Descriptor instead.
func (WorkflowRunEventType) EnumDescriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{5}
}
type WorkerRegisterRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1049,6 +1092,54 @@ func (x *SubscribeToWorkflowEventsRequest) GetWorkflowRunId() string {
return ""
}
type SubscribeToWorkflowRunsRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// the id of the workflow run
WorkflowRunId string `protobuf:"bytes,1,opt,name=workflowRunId,proto3" json:"workflowRunId,omitempty"`
}
func (x *SubscribeToWorkflowRunsRequest) Reset() {
*x = SubscribeToWorkflowRunsRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[10]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *SubscribeToWorkflowRunsRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*SubscribeToWorkflowRunsRequest) ProtoMessage() {}
func (x *SubscribeToWorkflowRunsRequest) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[10]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use SubscribeToWorkflowRunsRequest.ProtoReflect.Descriptor instead.
func (*SubscribeToWorkflowRunsRequest) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{10}
}
func (x *SubscribeToWorkflowRunsRequest) GetWorkflowRunId() string {
if x != nil {
return x.WorkflowRunId
}
return ""
}
type WorkflowEvent struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1074,7 +1165,7 @@ type WorkflowEvent struct {
func (x *WorkflowEvent) Reset() {
*x = WorkflowEvent{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[10]
mi := &file_dispatcher_proto_msgTypes[11]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1087,7 +1178,7 @@ func (x *WorkflowEvent) String() string {
func (*WorkflowEvent) ProtoMessage() {}
func (x *WorkflowEvent) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[10]
mi := &file_dispatcher_proto_msgTypes[11]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1100,7 +1191,7 @@ func (x *WorkflowEvent) ProtoReflect() protoreflect.Message {
// Deprecated: Use WorkflowEvent.ProtoReflect.Descriptor instead.
func (*WorkflowEvent) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{10}
return file_dispatcher_proto_rawDescGZIP(), []int{11}
}
func (x *WorkflowEvent) GetWorkflowRunId() string {
@@ -1166,6 +1257,157 @@ func (x *WorkflowEvent) GetRetryCount() int32 {
return 0
}
type WorkflowRunEvent struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// the id of the workflow run
WorkflowRunId string `protobuf:"bytes,1,opt,name=workflowRunId,proto3" json:"workflowRunId,omitempty"`
EventType WorkflowRunEventType `protobuf:"varint,2,opt,name=eventType,proto3,enum=WorkflowRunEventType" json:"eventType,omitempty"`
EventTimestamp *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=eventTimestamp,proto3" json:"eventTimestamp,omitempty"`
Results []*StepRunResult `protobuf:"bytes,4,rep,name=results,proto3" json:"results,omitempty"`
}
func (x *WorkflowRunEvent) Reset() {
*x = WorkflowRunEvent{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[12]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *WorkflowRunEvent) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*WorkflowRunEvent) ProtoMessage() {}
func (x *WorkflowRunEvent) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[12]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use WorkflowRunEvent.ProtoReflect.Descriptor instead.
func (*WorkflowRunEvent) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{12}
}
func (x *WorkflowRunEvent) GetWorkflowRunId() string {
if x != nil {
return x.WorkflowRunId
}
return ""
}
func (x *WorkflowRunEvent) GetEventType() WorkflowRunEventType {
if x != nil {
return x.EventType
}
return WorkflowRunEventType_WORKFLOW_RUN_EVENT_TYPE_FINISHED
}
func (x *WorkflowRunEvent) GetEventTimestamp() *timestamppb.Timestamp {
if x != nil {
return x.EventTimestamp
}
return nil
}
func (x *WorkflowRunEvent) GetResults() []*StepRunResult {
if x != nil {
return x.Results
}
return nil
}
type StepRunResult struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
StepRunId string `protobuf:"bytes,1,opt,name=stepRunId,proto3" json:"stepRunId,omitempty"`
StepReadableId string `protobuf:"bytes,2,opt,name=stepReadableId,proto3" json:"stepReadableId,omitempty"`
JobRunId string `protobuf:"bytes,3,opt,name=jobRunId,proto3" json:"jobRunId,omitempty"`
Error *string `protobuf:"bytes,4,opt,name=error,proto3,oneof" json:"error,omitempty"`
Output *string `protobuf:"bytes,5,opt,name=output,proto3,oneof" json:"output,omitempty"`
}
func (x *StepRunResult) Reset() {
*x = StepRunResult{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[13]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *StepRunResult) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*StepRunResult) ProtoMessage() {}
func (x *StepRunResult) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[13]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use StepRunResult.ProtoReflect.Descriptor instead.
func (*StepRunResult) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{13}
}
func (x *StepRunResult) GetStepRunId() string {
if x != nil {
return x.StepRunId
}
return ""
}
func (x *StepRunResult) GetStepReadableId() string {
if x != nil {
return x.StepReadableId
}
return ""
}
func (x *StepRunResult) GetJobRunId() string {
if x != nil {
return x.JobRunId
}
return ""
}
func (x *StepRunResult) GetError() string {
if x != nil && x.Error != nil {
return *x.Error
}
return ""
}
func (x *StepRunResult) GetOutput() string {
if x != nil && x.Output != nil {
return *x.Output
}
return ""
}
type OverridesData struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1184,7 +1426,7 @@ type OverridesData struct {
func (x *OverridesData) Reset() {
*x = OverridesData{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[11]
mi := &file_dispatcher_proto_msgTypes[14]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1197,7 +1439,7 @@ func (x *OverridesData) String() string {
func (*OverridesData) ProtoMessage() {}
func (x *OverridesData) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[11]
mi := &file_dispatcher_proto_msgTypes[14]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1210,7 +1452,7 @@ func (x *OverridesData) ProtoReflect() protoreflect.Message {
// Deprecated: Use OverridesData.ProtoReflect.Descriptor instead.
func (*OverridesData) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{11}
return file_dispatcher_proto_rawDescGZIP(), []int{14}
}
func (x *OverridesData) GetStepRunId() string {
@@ -1250,7 +1492,7 @@ type OverridesDataResponse struct {
func (x *OverridesDataResponse) Reset() {
*x = OverridesDataResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[12]
mi := &file_dispatcher_proto_msgTypes[15]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1263,7 +1505,7 @@ func (x *OverridesDataResponse) String() string {
func (*OverridesDataResponse) ProtoMessage() {}
func (x *OverridesDataResponse) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[12]
mi := &file_dispatcher_proto_msgTypes[15]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1276,7 +1518,7 @@ func (x *OverridesDataResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use OverridesDataResponse.ProtoReflect.Descriptor instead.
func (*OverridesDataResponse) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{12}
return file_dispatcher_proto_rawDescGZIP(), []int{15}
}
type HeartbeatRequest struct {
@@ -1293,7 +1535,7 @@ type HeartbeatRequest struct {
func (x *HeartbeatRequest) Reset() {
*x = HeartbeatRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[13]
mi := &file_dispatcher_proto_msgTypes[16]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1306,7 +1548,7 @@ func (x *HeartbeatRequest) String() string {
func (*HeartbeatRequest) ProtoMessage() {}
func (x *HeartbeatRequest) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[13]
mi := &file_dispatcher_proto_msgTypes[16]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1319,7 +1561,7 @@ func (x *HeartbeatRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use HeartbeatRequest.ProtoReflect.Descriptor instead.
func (*HeartbeatRequest) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{13}
return file_dispatcher_proto_rawDescGZIP(), []int{16}
}
func (x *HeartbeatRequest) GetWorkerId() string {
@@ -1345,7 +1587,7 @@ type HeartbeatResponse struct {
func (x *HeartbeatResponse) Reset() {
*x = HeartbeatResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_dispatcher_proto_msgTypes[14]
mi := &file_dispatcher_proto_msgTypes[17]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -1358,7 +1600,7 @@ func (x *HeartbeatResponse) String() string {
func (*HeartbeatResponse) ProtoMessage() {}
func (x *HeartbeatResponse) ProtoReflect() protoreflect.Message {
mi := &file_dispatcher_proto_msgTypes[14]
mi := &file_dispatcher_proto_msgTypes[17]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -1371,7 +1613,7 @@ func (x *HeartbeatResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use HeartbeatResponse.ProtoReflect.Descriptor instead.
func (*HeartbeatResponse) Descriptor() ([]byte, []int) {
return file_dispatcher_proto_rawDescGZIP(), []int{14}
return file_dispatcher_proto_rawDescGZIP(), []int{17}
}
var File_dispatcher_proto protoreflect.FileDescriptor
@@ -1484,140 +1726,180 @@ var file_dispatcher_proto_rawDesc = []byte{
0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b,
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x22, 0xa5, 0x03, 0x0a, 0x0d, 0x57, 0x6f,
0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77,
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x22, 0x46, 0x0a, 0x1e, 0x53, 0x75, 0x62,
0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
0x52, 0x75, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77,
0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49,
0x64, 0x12, 0x31, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70,
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0d, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x12, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65,
0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x42, 0x0a, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54,
0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a,
0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e,
0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76,
0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09,
0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x16,
0x0a, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06,
0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x12, 0x25, 0x0a, 0x0b, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65,
0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0b, 0x73,
0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x88, 0x01, 0x01, 0x12, 0x23, 0x0a,
0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28,
0x05, 0x48, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x88,
0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69,
0x65, 0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e,
0x74, 0x22, 0x7f, 0x0a, 0x0d, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61,
0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18,
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64,
0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20,
0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x61,
0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61,
0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44,
0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6c, 0x0a, 0x10, 0x48,
0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x68,
0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x68, 0x65,
0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41, 0x74, 0x22, 0x13, 0x0a, 0x11, 0x48, 0x65, 0x61,
0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x4e,
0x0a, 0x0a, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x0e,
0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x00,
0x12, 0x13, 0x0a, 0x0f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f,
0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x47,
0x45, 0x54, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x02, 0x2a, 0xa2,
0x01, 0x0a, 0x17, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f,
0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52,
0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c,
0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f,
0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x22,
0x0a, 0x1e, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e,
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44,
0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f,
0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45,
0x44, 0x10, 0x03, 0x2a, 0x8a, 0x01, 0x0a, 0x13, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69,
0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x53,
0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55,
0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50,
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52,
0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56,
0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54,
0x45, 0x44, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45,
0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03,
0x2a, 0x65, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65,
0x12, 0x19, 0x0a, 0x15, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50,
0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x52,
0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x45,
0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55,
0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f,
0x57, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x02, 0x2a, 0xfe, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a,
0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f,
0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1f,
0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54,
0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12,
0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e,
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44,
0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45,
0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44,
0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45,
0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c,
0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43,
0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d,
0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x05, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f,
0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x10, 0x06, 0x32, 0xd1, 0x04, 0x0a, 0x0a, 0x44, 0x69, 0x73,
0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x08, 0x52, 0x65, 0x67, 0x69, 0x73,
0x74, 0x65, 0x72, 0x12, 0x16, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69,
0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x57, 0x6f,
0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x33, 0x0a, 0x06, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e,
0x12, 0x14, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65,
0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x30, 0x01, 0x12, 0x35, 0x0a, 0x08, 0x4c,
0x69, 0x73, 0x74, 0x65, 0x6e, 0x56, 0x32, 0x12, 0x14, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72,
0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e,
0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00,
0x30, 0x01, 0x12, 0x34, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12,
0x11, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x12, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x19, 0x53, 0x75, 0x62, 0x73,
0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45,
0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74,
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66,
0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3f, 0x0a, 0x13,
0x53, 0x65, 0x6e, 0x64, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x12, 0x10, 0x2e, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a,
0x17, 0x53, 0x65, 0x6e, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x14, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70,
0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14,
0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3c, 0x0a, 0x10, 0x50, 0x75, 0x74, 0x4f, 0x76, 0x65,
0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x0e, 0x2e, 0x4f, 0x76, 0x65,
0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x16, 0x2e, 0x4f, 0x76, 0x65,
0x64, 0x22, 0xa5, 0x03, 0x0a, 0x0d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b,
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x0c, 0x72, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x0d, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x09,
0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x12, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54,
0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e,
0x0a, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x42,
0x0a, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70,
0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x12, 0x25,
0x0a, 0x0b, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x08, 0x20,
0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69,
0x65, 0x73, 0x88, 0x01, 0x01, 0x12, 0x23, 0x0a, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f,
0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x74,
0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x88, 0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73,
0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x72,
0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xdb, 0x01, 0x0a, 0x10, 0x57, 0x6f,
0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24,
0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18,
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
0x75, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c,
0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09,
0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x42, 0x0a, 0x0e, 0x65, 0x76, 0x65,
0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65,
0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a,
0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e,
0x2e, 0x53, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07,
0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xbe, 0x01, 0x0a, 0x0d, 0x53, 0x74, 0x65, 0x70,
0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65,
0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74,
0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x0e, 0x73, 0x74, 0x65, 0x70, 0x52,
0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0e, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12,
0x1a, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
0x09, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x05, 0x65,
0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x05, 0x65, 0x72,
0x72, 0x6f, 0x72, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74,
0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74,
0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x09, 0x0a,
0x07, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x22, 0x7f, 0x0a, 0x0d, 0x4f, 0x76, 0x65, 0x72,
0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65,
0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74,
0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18,
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x76,
0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e,
0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65,
0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4f, 0x76, 0x65,
0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x00, 0x12, 0x46, 0x0a, 0x0b, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72,
0x69, 0x62, 0x65, 0x12, 0x19, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75,
0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a,
0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x47, 0x5a, 0x45,
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68,
0x65, 0x74, 0x2d, 0x64, 0x65, 0x76, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2f, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73,
0x2f, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x2f, 0x63, 0x6f, 0x6e, 0x74,
0x72, 0x61, 0x63, 0x74, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x73, 0x65, 0x22, 0x6c, 0x0a, 0x10, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41,
0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74,
0x61, 0x6d, 0x70, 0x52, 0x0b, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41, 0x74,
0x22, 0x13, 0x0a, 0x11, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x4e, 0x0a, 0x0a, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54,
0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x45,
0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x41, 0x4e, 0x43, 0x45,
0x4c, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13,
0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x47, 0x45, 0x54, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f,
0x4b, 0x45, 0x59, 0x10, 0x02, 0x2a, 0xa2, 0x01, 0x0a, 0x17, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b,
0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
0x65, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45,
0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57,
0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59,
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52,
0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b,
0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f,
0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x47, 0x52, 0x4f,
0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50,
0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x8a, 0x01, 0x0a, 0x13, 0x53,
0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79,
0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54,
0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12,
0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19,
0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x53,
0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46,
0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x65, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x45, 0x53, 0x4f, 0x55,
0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e,
0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54,
0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x1e,
0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x02, 0x2a, 0xfe,
0x01, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74,
0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45,
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e,
0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43,
0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41,
0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52,
0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f,
0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53,
0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,
0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53,
0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,
0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x21, 0x0a, 0x1d,
0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54,
0x59, 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x05, 0x12,
0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e,
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x10, 0x06, 0x2a,
0x3c, 0x0a, 0x14, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x57, 0x4f, 0x52, 0x4b, 0x46,
0x4c, 0x4f, 0x57, 0x5f, 0x52, 0x55, 0x4e, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x45, 0x44, 0x10, 0x00, 0x32, 0xa6, 0x05,
0x0a, 0x0a, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x08,
0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x12, 0x16, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65,
0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x17, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65,
0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x33, 0x0a, 0x06, 0x4c,
0x69, 0x73, 0x74, 0x65, 0x6e, 0x12, 0x14, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69,
0x73, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x41, 0x73,
0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x30, 0x01,
0x12, 0x35, 0x0a, 0x08, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x56, 0x32, 0x12, 0x14, 0x2e, 0x57,
0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x22, 0x00, 0x30, 0x01, 0x12, 0x34, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74,
0x62, 0x65, 0x61, 0x74, 0x12, 0x11, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x12, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62,
0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a,
0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b,
0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x53, 0x75, 0x62,
0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e,
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30,
0x01, 0x12, 0x53, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f,
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x73, 0x12, 0x1f, 0x2e, 0x53,
0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c,
0x6f, 0x77, 0x52, 0x75, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e,
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74,
0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x3f, 0x0a, 0x13, 0x53, 0x65, 0x6e, 0x64, 0x53, 0x74,
0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x10, 0x2e,
0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a,
0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a, 0x17, 0x53, 0x65, 0x6e, 0x64, 0x47,
0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65,
0x6e, 0x74, 0x12, 0x14, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f,
0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
0x12, 0x3c, 0x0a, 0x10, 0x50, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73,
0x44, 0x61, 0x74, 0x61, 0x12, 0x0e, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73,
0x44, 0x61, 0x74, 0x61, 0x1a, 0x16, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73,
0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x46,
0x0a, 0x0b, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x19, 0x2e,
0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65,
0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x47, 0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2d, 0x64, 0x65, 0x76,
0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x2f, 0x64, 0x69, 0x73, 0x70, 0x61,
0x74, 0x63, 0x68, 0x65, 0x72, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x73, 0x62,
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -1632,64 +1914,73 @@ func file_dispatcher_proto_rawDescGZIP() []byte {
return file_dispatcher_proto_rawDescData
}
var file_dispatcher_proto_enumTypes = make([]protoimpl.EnumInfo, 5)
var file_dispatcher_proto_msgTypes = make([]protoimpl.MessageInfo, 15)
var file_dispatcher_proto_enumTypes = make([]protoimpl.EnumInfo, 6)
var file_dispatcher_proto_msgTypes = make([]protoimpl.MessageInfo, 18)
var file_dispatcher_proto_goTypes = []interface{}{
(ActionType)(0), // 0: ActionType
(GroupKeyActionEventType)(0), // 1: GroupKeyActionEventType
(StepActionEventType)(0), // 2: StepActionEventType
(ResourceType)(0), // 3: ResourceType
(ResourceEventType)(0), // 4: ResourceEventType
(*WorkerRegisterRequest)(nil), // 5: WorkerRegisterRequest
(*WorkerRegisterResponse)(nil), // 6: WorkerRegisterResponse
(*AssignedAction)(nil), // 7: AssignedAction
(*WorkerListenRequest)(nil), // 8: WorkerListenRequest
(*WorkerUnsubscribeRequest)(nil), // 9: WorkerUnsubscribeRequest
(*WorkerUnsubscribeResponse)(nil), // 10: WorkerUnsubscribeResponse
(*GroupKeyActionEvent)(nil), // 11: GroupKeyActionEvent
(*StepActionEvent)(nil), // 12: StepActionEvent
(*ActionEventResponse)(nil), // 13: ActionEventResponse
(*SubscribeToWorkflowEventsRequest)(nil), // 14: SubscribeToWorkflowEventsRequest
(*WorkflowEvent)(nil), // 15: WorkflowEvent
(*OverridesData)(nil), // 16: OverridesData
(*OverridesDataResponse)(nil), // 17: OverridesDataResponse
(*HeartbeatRequest)(nil), // 18: HeartbeatRequest
(*HeartbeatResponse)(nil), // 19: HeartbeatResponse
(*timestamppb.Timestamp)(nil), // 20: google.protobuf.Timestamp
(WorkflowRunEventType)(0), // 5: WorkflowRunEventType
(*WorkerRegisterRequest)(nil), // 6: WorkerRegisterRequest
(*WorkerRegisterResponse)(nil), // 7: WorkerRegisterResponse
(*AssignedAction)(nil), // 8: AssignedAction
(*WorkerListenRequest)(nil), // 9: WorkerListenRequest
(*WorkerUnsubscribeRequest)(nil), // 10: WorkerUnsubscribeRequest
(*WorkerUnsubscribeResponse)(nil), // 11: WorkerUnsubscribeResponse
(*GroupKeyActionEvent)(nil), // 12: GroupKeyActionEvent
(*StepActionEvent)(nil), // 13: StepActionEvent
(*ActionEventResponse)(nil), // 14: ActionEventResponse
(*SubscribeToWorkflowEventsRequest)(nil), // 15: SubscribeToWorkflowEventsRequest
(*SubscribeToWorkflowRunsRequest)(nil), // 16: SubscribeToWorkflowRunsRequest
(*WorkflowEvent)(nil), // 17: WorkflowEvent
(*WorkflowRunEvent)(nil), // 18: WorkflowRunEvent
(*StepRunResult)(nil), // 19: StepRunResult
(*OverridesData)(nil), // 20: OverridesData
(*OverridesDataResponse)(nil), // 21: OverridesDataResponse
(*HeartbeatRequest)(nil), // 22: HeartbeatRequest
(*HeartbeatResponse)(nil), // 23: HeartbeatResponse
(*timestamppb.Timestamp)(nil), // 24: google.protobuf.Timestamp
}
var file_dispatcher_proto_depIdxs = []int32{
0, // 0: AssignedAction.actionType:type_name -> ActionType
20, // 1: GroupKeyActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
24, // 1: GroupKeyActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
1, // 2: GroupKeyActionEvent.eventType:type_name -> GroupKeyActionEventType
20, // 3: StepActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
24, // 3: StepActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
2, // 4: StepActionEvent.eventType:type_name -> StepActionEventType
3, // 5: WorkflowEvent.resourceType:type_name -> ResourceType
4, // 6: WorkflowEvent.eventType:type_name -> ResourceEventType
20, // 7: WorkflowEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
20, // 8: HeartbeatRequest.heartbeatAt:type_name -> google.protobuf.Timestamp
5, // 9: Dispatcher.Register:input_type -> WorkerRegisterRequest
8, // 10: Dispatcher.Listen:input_type -> WorkerListenRequest
8, // 11: Dispatcher.ListenV2:input_type -> WorkerListenRequest
18, // 12: Dispatcher.Heartbeat:input_type -> HeartbeatRequest
14, // 13: Dispatcher.SubscribeToWorkflowEvents:input_type -> SubscribeToWorkflowEventsRequest
12, // 14: Dispatcher.SendStepActionEvent:input_type -> StepActionEvent
11, // 15: Dispatcher.SendGroupKeyActionEvent:input_type -> GroupKeyActionEvent
16, // 16: Dispatcher.PutOverridesData:input_type -> OverridesData
9, // 17: Dispatcher.Unsubscribe:input_type -> WorkerUnsubscribeRequest
6, // 18: Dispatcher.Register:output_type -> WorkerRegisterResponse
7, // 19: Dispatcher.Listen:output_type -> AssignedAction
7, // 20: Dispatcher.ListenV2:output_type -> AssignedAction
19, // 21: Dispatcher.Heartbeat:output_type -> HeartbeatResponse
15, // 22: Dispatcher.SubscribeToWorkflowEvents:output_type -> WorkflowEvent
13, // 23: Dispatcher.SendStepActionEvent:output_type -> ActionEventResponse
13, // 24: Dispatcher.SendGroupKeyActionEvent:output_type -> ActionEventResponse
17, // 25: Dispatcher.PutOverridesData:output_type -> OverridesDataResponse
10, // 26: Dispatcher.Unsubscribe:output_type -> WorkerUnsubscribeResponse
18, // [18:27] is the sub-list for method output_type
9, // [9:18] is the sub-list for method input_type
9, // [9:9] is the sub-list for extension type_name
9, // [9:9] is the sub-list for extension extendee
0, // [0:9] is the sub-list for field type_name
24, // 7: WorkflowEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
5, // 8: WorkflowRunEvent.eventType:type_name -> WorkflowRunEventType
24, // 9: WorkflowRunEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
19, // 10: WorkflowRunEvent.results:type_name -> StepRunResult
24, // 11: HeartbeatRequest.heartbeatAt:type_name -> google.protobuf.Timestamp
6, // 12: Dispatcher.Register:input_type -> WorkerRegisterRequest
9, // 13: Dispatcher.Listen:input_type -> WorkerListenRequest
9, // 14: Dispatcher.ListenV2:input_type -> WorkerListenRequest
22, // 15: Dispatcher.Heartbeat:input_type -> HeartbeatRequest
15, // 16: Dispatcher.SubscribeToWorkflowEvents:input_type -> SubscribeToWorkflowEventsRequest
16, // 17: Dispatcher.SubscribeToWorkflowRuns:input_type -> SubscribeToWorkflowRunsRequest
13, // 18: Dispatcher.SendStepActionEvent:input_type -> StepActionEvent
12, // 19: Dispatcher.SendGroupKeyActionEvent:input_type -> GroupKeyActionEvent
20, // 20: Dispatcher.PutOverridesData:input_type -> OverridesData
10, // 21: Dispatcher.Unsubscribe:input_type -> WorkerUnsubscribeRequest
7, // 22: Dispatcher.Register:output_type -> WorkerRegisterResponse
8, // 23: Dispatcher.Listen:output_type -> AssignedAction
8, // 24: Dispatcher.ListenV2:output_type -> AssignedAction
23, // 25: Dispatcher.Heartbeat:output_type -> HeartbeatResponse
17, // 26: Dispatcher.SubscribeToWorkflowEvents:output_type -> WorkflowEvent
18, // 27: Dispatcher.SubscribeToWorkflowRuns:output_type -> WorkflowRunEvent
14, // 28: Dispatcher.SendStepActionEvent:output_type -> ActionEventResponse
14, // 29: Dispatcher.SendGroupKeyActionEvent:output_type -> ActionEventResponse
21, // 30: Dispatcher.PutOverridesData:output_type -> OverridesDataResponse
11, // 31: Dispatcher.Unsubscribe:output_type -> WorkerUnsubscribeResponse
22, // [22:32] is the sub-list for method output_type
12, // [12:22] is the sub-list for method input_type
12, // [12:12] is the sub-list for extension type_name
12, // [12:12] is the sub-list for extension extendee
0, // [0:12] is the sub-list for field type_name
}
func init() { file_dispatcher_proto_init() }
@@ -1819,7 +2110,7 @@ func file_dispatcher_proto_init() {
}
}
file_dispatcher_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*WorkflowEvent); i {
switch v := v.(*SubscribeToWorkflowRunsRequest); i {
case 0:
return &v.state
case 1:
@@ -1831,7 +2122,7 @@ func file_dispatcher_proto_init() {
}
}
file_dispatcher_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OverridesData); i {
switch v := v.(*WorkflowEvent); i {
case 0:
return &v.state
case 1:
@@ -1843,7 +2134,7 @@ func file_dispatcher_proto_init() {
}
}
file_dispatcher_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OverridesDataResponse); i {
switch v := v.(*WorkflowRunEvent); i {
case 0:
return &v.state
case 1:
@@ -1855,7 +2146,7 @@ func file_dispatcher_proto_init() {
}
}
file_dispatcher_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*HeartbeatRequest); i {
switch v := v.(*StepRunResult); i {
case 0:
return &v.state
case 1:
@@ -1867,6 +2158,42 @@ func file_dispatcher_proto_init() {
}
}
file_dispatcher_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OverridesData); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_dispatcher_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*OverridesDataResponse); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_dispatcher_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*HeartbeatRequest); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_dispatcher_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*HeartbeatResponse); i {
case 0:
return &v.state
@@ -1880,14 +2207,15 @@ func file_dispatcher_proto_init() {
}
}
file_dispatcher_proto_msgTypes[0].OneofWrappers = []interface{}{}
file_dispatcher_proto_msgTypes[10].OneofWrappers = []interface{}{}
file_dispatcher_proto_msgTypes[11].OneofWrappers = []interface{}{}
file_dispatcher_proto_msgTypes[13].OneofWrappers = []interface{}{}
type x struct{}
out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_dispatcher_proto_rawDesc,
NumEnums: 5,
NumMessages: 15,
NumEnums: 6,
NumMessages: 18,
NumExtensions: 0,
NumServices: 1,
},

View File

@@ -30,6 +30,7 @@ type DispatcherClient interface {
// Heartbeat is a method for workers to send heartbeats to the dispatcher
Heartbeat(ctx context.Context, in *HeartbeatRequest, opts ...grpc.CallOption) (*HeartbeatResponse, error)
SubscribeToWorkflowEvents(ctx context.Context, in *SubscribeToWorkflowEventsRequest, opts ...grpc.CallOption) (Dispatcher_SubscribeToWorkflowEventsClient, error)
SubscribeToWorkflowRuns(ctx context.Context, opts ...grpc.CallOption) (Dispatcher_SubscribeToWorkflowRunsClient, error)
SendStepActionEvent(ctx context.Context, in *StepActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error)
SendGroupKeyActionEvent(ctx context.Context, in *GroupKeyActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error)
PutOverridesData(ctx context.Context, in *OverridesData, opts ...grpc.CallOption) (*OverridesDataResponse, error)
@@ -158,6 +159,37 @@ func (x *dispatcherSubscribeToWorkflowEventsClient) Recv() (*WorkflowEvent, erro
return m, nil
}
func (c *dispatcherClient) SubscribeToWorkflowRuns(ctx context.Context, opts ...grpc.CallOption) (Dispatcher_SubscribeToWorkflowRunsClient, error) {
stream, err := c.cc.NewStream(ctx, &Dispatcher_ServiceDesc.Streams[3], "/Dispatcher/SubscribeToWorkflowRuns", opts...)
if err != nil {
return nil, err
}
x := &dispatcherSubscribeToWorkflowRunsClient{stream}
return x, nil
}
type Dispatcher_SubscribeToWorkflowRunsClient interface {
Send(*SubscribeToWorkflowRunsRequest) error
Recv() (*WorkflowRunEvent, error)
grpc.ClientStream
}
type dispatcherSubscribeToWorkflowRunsClient struct {
grpc.ClientStream
}
func (x *dispatcherSubscribeToWorkflowRunsClient) Send(m *SubscribeToWorkflowRunsRequest) error {
return x.ClientStream.SendMsg(m)
}
func (x *dispatcherSubscribeToWorkflowRunsClient) Recv() (*WorkflowRunEvent, error) {
m := new(WorkflowRunEvent)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *dispatcherClient) SendStepActionEvent(ctx context.Context, in *StepActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) {
out := new(ActionEventResponse)
err := c.cc.Invoke(ctx, "/Dispatcher/SendStepActionEvent", in, out, opts...)
@@ -206,6 +238,7 @@ type DispatcherServer interface {
// Heartbeat is a method for workers to send heartbeats to the dispatcher
Heartbeat(context.Context, *HeartbeatRequest) (*HeartbeatResponse, error)
SubscribeToWorkflowEvents(*SubscribeToWorkflowEventsRequest, Dispatcher_SubscribeToWorkflowEventsServer) error
SubscribeToWorkflowRuns(Dispatcher_SubscribeToWorkflowRunsServer) error
SendStepActionEvent(context.Context, *StepActionEvent) (*ActionEventResponse, error)
SendGroupKeyActionEvent(context.Context, *GroupKeyActionEvent) (*ActionEventResponse, error)
PutOverridesData(context.Context, *OverridesData) (*OverridesDataResponse, error)
@@ -232,6 +265,9 @@ func (UnimplementedDispatcherServer) Heartbeat(context.Context, *HeartbeatReques
func (UnimplementedDispatcherServer) SubscribeToWorkflowEvents(*SubscribeToWorkflowEventsRequest, Dispatcher_SubscribeToWorkflowEventsServer) error {
return status.Errorf(codes.Unimplemented, "method SubscribeToWorkflowEvents not implemented")
}
func (UnimplementedDispatcherServer) SubscribeToWorkflowRuns(Dispatcher_SubscribeToWorkflowRunsServer) error {
return status.Errorf(codes.Unimplemented, "method SubscribeToWorkflowRuns not implemented")
}
func (UnimplementedDispatcherServer) SendStepActionEvent(context.Context, *StepActionEvent) (*ActionEventResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method SendStepActionEvent not implemented")
}
@@ -356,6 +392,32 @@ func (x *dispatcherSubscribeToWorkflowEventsServer) Send(m *WorkflowEvent) error
return x.ServerStream.SendMsg(m)
}
func _Dispatcher_SubscribeToWorkflowRuns_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(DispatcherServer).SubscribeToWorkflowRuns(&dispatcherSubscribeToWorkflowRunsServer{stream})
}
type Dispatcher_SubscribeToWorkflowRunsServer interface {
Send(*WorkflowRunEvent) error
Recv() (*SubscribeToWorkflowRunsRequest, error)
grpc.ServerStream
}
type dispatcherSubscribeToWorkflowRunsServer struct {
grpc.ServerStream
}
func (x *dispatcherSubscribeToWorkflowRunsServer) Send(m *WorkflowRunEvent) error {
return x.ServerStream.SendMsg(m)
}
func (x *dispatcherSubscribeToWorkflowRunsServer) Recv() (*SubscribeToWorkflowRunsRequest, error) {
m := new(SubscribeToWorkflowRunsRequest)
if err := x.ServerStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func _Dispatcher_SendStepActionEvent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(StepActionEvent)
if err := dec(in); err != nil {
@@ -476,6 +538,12 @@ var Dispatcher_ServiceDesc = grpc.ServiceDesc{
Handler: _Dispatcher_SubscribeToWorkflowEvents_Handler,
ServerStreams: true,
},
{
StreamName: "SubscribeToWorkflowRuns",
Handler: _Dispatcher_SubscribeToWorkflowRuns_Handler,
ServerStreams: true,
ClientStreams: true,
},
},
Metadata: "dispatcher.proto",
}

View File

@@ -142,17 +142,18 @@ func New(fs ...DispatcherOpt) (*DispatcherImpl, error) {
}
func (d *DispatcherImpl) Start() (func() error, error) {
ctx, cancel := context.WithCancel(context.Background())
// register the dispatcher by creating a new dispatcher in the database
dispatcher, err := d.repo.Dispatcher().CreateNewDispatcher(&repository.CreateDispatcherOpts{
dispatcher, err := d.repo.Dispatcher().CreateNewDispatcher(ctx, &repository.CreateDispatcherOpts{
ID: d.dispatcherId,
})
if err != nil {
cancel()
return nil, err
}
ctx, cancel := context.WithCancel(context.Background())
_, err = d.s.NewJob(
gocron.DurationJob(time.Second*5),
gocron.NewTask(
@@ -212,17 +213,20 @@ func (d *DispatcherImpl) Start() (func() error, error) {
return true
})
err = d.repo.Dispatcher().Delete(dispatcherId)
if err := d.s.Shutdown(); err != nil {
return fmt.Errorf("could not shutdown scheduler: %w", err)
}
deleteCtx, deleteCancel := context.WithTimeout(context.Background(), 5*time.Second)
defer deleteCancel()
err = d.repo.Dispatcher().Delete(deleteCtx, dispatcherId)
if err != nil {
return fmt.Errorf("could not delete dispatcher: %w", err)
}
d.l.Debug().Msgf("deleted dispatcher %s", dispatcherId)
if err := d.s.Shutdown(); err != nil {
return fmt.Errorf("could not shutdown scheduler: %w", err)
}
d.l.Debug().Msgf("dispatcher has shutdown")
return nil
}
@@ -278,7 +282,7 @@ func (d *DispatcherImpl) handleGroupKeyActionAssignedTask(ctx context.Context, t
}
// load the workflow run from the database
workflowRun, err := d.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId)
workflowRun, err := d.repo.WorkflowRun().GetWorkflowRunById(ctx, metadata.TenantId, payload.WorkflowRunId)
if err != nil {
return fmt.Errorf("could not get workflow run: %w", err)
@@ -292,7 +296,7 @@ func (d *DispatcherImpl) handleGroupKeyActionAssignedTask(ctx context.Context, t
return fmt.Errorf("could not get group key run")
}
sqlcGroupKeyRun, err := d.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(metadata.TenantId, groupKeyRunId)
sqlcGroupKeyRun, err := d.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(ctx, metadata.TenantId, groupKeyRunId)
if err != nil {
return fmt.Errorf("could not get group key run for engine: %w", err)
@@ -334,7 +338,7 @@ func (d *DispatcherImpl) handleStepRunAssignedTask(ctx context.Context, task *ms
}
// load the step run from the database
stepRun, err := d.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
stepRun, err := d.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
if err != nil {
return fmt.Errorf("could not get step run: %w", err)
@@ -378,7 +382,7 @@ func (d *DispatcherImpl) handleStepRunCancelled(ctx context.Context, task *msgqu
}
// load the step run from the database
stepRun, err := d.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
stepRun, err := d.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
if err != nil {
return fmt.Errorf("could not get step run: %w", err)
@@ -402,7 +406,7 @@ func (d *DispatcherImpl) runUpdateHeartbeat(ctx context.Context) func() {
now := time.Now().UTC()
// update the heartbeat
_, err := d.repo.Dispatcher().UpdateDispatcher(d.dispatcherId, &repository.UpdateDispatcherOpts{
_, err := d.repo.Dispatcher().UpdateDispatcher(ctx, d.dispatcherId, &repository.UpdateDispatcherOpts{
LastHeartbeatAt: &now,
})

View File

@@ -2,12 +2,17 @@ package dispatcher
import (
"context"
"errors"
"fmt"
"io"
"strconv"
"sync"
"time"
"github.com/rs/zerolog"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"google.golang.org/protobuf/types/known/timestamppb"
"github.com/hatchet-dev/hatchet/internal/datautils"
"github.com/hatchet-dev/hatchet/internal/msgqueue"
@@ -144,7 +149,7 @@ func (s *DispatcherImpl) Register(ctx context.Context, request *contracts.Worker
}
// create a worker in the database
worker, err := s.repo.Worker().CreateNewWorker(tenantId, opts)
worker, err := s.repo.Worker().CreateNewWorker(ctx, tenantId, opts)
if err != nil {
s.l.Error().Err(err).Msgf("could not create worker for tenant %s", tenantId)
@@ -170,7 +175,9 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
s.l.Debug().Msgf("Received subscribe request from ID: %s", request.WorkerId)
worker, err := s.repo.Worker().GetWorkerForEngine(tenantId, request.WorkerId)
ctx := stream.Context()
worker, err := s.repo.Worker().GetWorkerForEngine(ctx, tenantId, request.WorkerId)
if err != nil {
s.l.Error().Err(err).Msgf("could not get worker %s", request.WorkerId)
@@ -179,7 +186,7 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
// check the worker's dispatcher against the current dispatcher. if they don't match, then update the worker
if worker.DispatcherId.Valid && sqlchelpers.UUIDToStr(worker.DispatcherId) != s.dispatcherId {
_, err = s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
_, err = s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
DispatcherId: &s.dispatcherId,
})
@@ -203,8 +210,6 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
s.workers.Delete(request.WorkerId)
}()
ctx := stream.Context()
// update the worker with a last heartbeat time every 5 seconds as long as the worker is connected
go func() {
timer := time.NewTicker(100 * time.Millisecond)
@@ -225,7 +230,7 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
if now := time.Now().UTC(); lastHeartbeat.Add(4 * time.Second).Before(now) {
s.l.Debug().Msgf("updating worker %s heartbeat", request.WorkerId)
_, err := s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
LastHeartbeatAt: &now,
})
@@ -259,9 +264,11 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
tenant := stream.Context().Value("tenant").(*dbsqlc.Tenant)
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
ctx := stream.Context()
s.l.Debug().Msgf("Received subscribe request from ID: %s", request.WorkerId)
worker, err := s.repo.Worker().GetWorkerForEngine(tenantId, request.WorkerId)
worker, err := s.repo.Worker().GetWorkerForEngine(ctx, tenantId, request.WorkerId)
if err != nil {
s.l.Error().Err(err).Msgf("could not get worker %s", request.WorkerId)
@@ -270,7 +277,7 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
// check the worker's dispatcher against the current dispatcher. if they don't match, then update the worker
if worker.DispatcherId.Valid && sqlchelpers.UUIDToStr(worker.DispatcherId) != s.dispatcherId {
_, err = s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
_, err = s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
DispatcherId: &s.dispatcherId,
})
@@ -295,7 +302,7 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
inactive := db.WorkerStatusInactive
_, err := s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
Status: &inactive,
})
@@ -304,8 +311,6 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
}
}()
ctx := stream.Context()
// Keep the connection alive for sending messages
for {
select {
@@ -335,7 +340,7 @@ func (s *DispatcherImpl) Heartbeat(ctx context.Context, req *contracts.Heartbeat
s.l.Warn().Msgf("heartbeat time is greater than expected heartbeat interval")
}
_, err := s.repo.Worker().UpdateWorker(tenantId, req.WorkerId, &repository.UpdateWorkerOpts{
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, req.WorkerId, &repository.UpdateWorkerOpts{
// use the system time for heartbeat
LastHeartbeatAt: &heartbeatAt,
})
@@ -411,6 +416,219 @@ func (s *DispatcherImpl) SubscribeToWorkflowEvents(request *contracts.SubscribeT
return nil
}
// map of workflow run ids to whether the workflow runs are finished and have sent a message
// that the workflow run is finished
type workflowRunAcks struct {
acks map[string]bool
mu sync.RWMutex
}
func (w *workflowRunAcks) addWorkflowRun(id string) {
w.mu.Lock()
defer w.mu.Unlock()
w.acks[id] = false
}
func (w *workflowRunAcks) getNonAckdWorkflowRuns() []string {
w.mu.RLock()
defer w.mu.RUnlock()
ids := make([]string, 0, len(w.acks))
for id := range w.acks {
if !w.acks[id] {
ids = append(ids, id)
}
}
return ids
}
func (w *workflowRunAcks) ackWorkflowRun(id string) {
w.mu.Lock()
defer w.mu.Unlock()
w.acks[id] = true
}
type sendTimeFilter struct {
mu sync.Mutex
}
func (s *sendTimeFilter) canSend() bool {
if !s.mu.TryLock() {
return false
}
go func() {
time.Sleep(time.Second - 10*time.Millisecond)
s.mu.Unlock()
}()
return true
}
// SubscribeToWorkflowEvents registers workflow events with the dispatcher
func (s *DispatcherImpl) SubscribeToWorkflowRuns(server contracts.Dispatcher_SubscribeToWorkflowRunsServer) error {
tenant := server.Context().Value("tenant").(*dbsqlc.Tenant)
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
s.l.Debug().Msgf("Received subscribe request for tenant: %s", tenantId)
acks := &workflowRunAcks{
acks: make(map[string]bool),
}
ctx, cancel := context.WithCancel(server.Context())
defer cancel()
// subscribe to the task queue for the tenant
q, err := msgqueue.TenantEventConsumerQueue(tenantId)
if err != nil {
return err
}
wg := sync.WaitGroup{}
sendEvent := func(e *contracts.WorkflowRunEvent) error {
// send the task to the client
err := server.Send(e)
if err != nil {
cancel() // FIXME is this necessary?
s.l.Error().Err(err).Msgf("could not send workflow event to client")
return err
}
acks.ackWorkflowRun(e.WorkflowRunId)
return nil
}
immediateSendFilter := &sendTimeFilter{}
iterSendFilter := &sendTimeFilter{}
iter := func(workflowRunIds []string) error {
limit := 1000
workflowRuns, err := s.repo.WorkflowRun().ListWorkflowRuns(ctx, tenantId, &repository.ListWorkflowRunsOpts{
Ids: workflowRunIds,
Limit: &limit,
})
if err != nil {
s.l.Error().Err(err).Msg("could not get workflow runs")
return nil
}
events, err := s.toWorkflowRunEvent(tenantId, workflowRuns.Rows)
if err != nil {
s.l.Error().Err(err).Msg("could not convert workflow run to event")
return nil
} else if events == nil {
return nil
}
for _, event := range events {
err := sendEvent(event)
if err != nil {
return err
}
}
return nil
}
// start a new goroutine to handle client-side streaming
go func() {
for {
req, err := server.Recv()
if err != nil {
cancel()
if errors.Is(err, io.EOF) || status.Code(err) == codes.Canceled {
return
}
s.l.Error().Err(err).Msg("could not receive message from client")
return
}
acks.addWorkflowRun(req.WorkflowRunId)
if immediateSendFilter.canSend() {
if err := iter([]string{req.WorkflowRunId}); err != nil {
s.l.Error().Err(err).Msg("could not iterate over workflow runs")
}
}
}
}()
f := func(task *msgqueue.Message) error {
wg.Add(1)
defer wg.Done()
workflowRunIds := acks.getNonAckdWorkflowRuns()
if matchedWorkflowRunId, ok := s.isMatchingWorkflowRun(task, workflowRunIds...); ok {
if immediateSendFilter.canSend() {
if err := iter([]string{matchedWorkflowRunId}); err != nil {
s.l.Error().Err(err).Msg("could not iterate over workflow runs")
}
}
}
return nil
}
// subscribe to the task queue for the tenant
cleanupQueue, err := s.mq.Subscribe(q, msgqueue.NoOpHook, f)
if err != nil {
return err
}
// new goroutine to poll every second for finished workflow runs which are not ackd
go func() {
ticker := time.NewTicker(1 * time.Second)
for {
select {
case <-ctx.Done():
return
case <-ticker.C:
if !iterSendFilter.canSend() {
continue
}
workflowRunIds := acks.getNonAckdWorkflowRuns()
if len(workflowRunIds) == 0 {
continue
}
if err := iter(workflowRunIds); err != nil {
s.l.Error().Err(err).Msg("could not iterate over workflow runs")
}
}
}
}()
<-ctx.Done()
if err := cleanupQueue(); err != nil {
return fmt.Errorf("could not cleanup queue: %w", err)
}
waitFor(&wg, 60*time.Second, s.l)
return nil
}
func waitFor(wg *sync.WaitGroup, timeout time.Duration, l *zerolog.Logger) {
done := make(chan struct{})
@@ -470,7 +688,7 @@ func (s *DispatcherImpl) PutOverridesData(ctx context.Context, request *contract
opts.CallerFile = &request.CallerFilename
}
_, err := s.repo.StepRun().UpdateStepRunOverridesData(tenantId, request.StepRunId, opts)
_, err := s.repo.StepRun().UpdateStepRunOverridesData(ctx, tenantId, request.StepRunId, opts)
if err != nil {
return nil, err
@@ -488,7 +706,7 @@ func (s *DispatcherImpl) Unsubscribe(ctx context.Context, request *contracts.Wor
inactive := db.WorkerStatusInactive
_, err := s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
Status: &inactive,
})
@@ -716,7 +934,7 @@ func (s *DispatcherImpl) handleGetGroupKeyRunFailed(ctx context.Context, request
}, nil
}
func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenantId, workflowRunId string) (*contracts.WorkflowEvent, error) {
func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenantId string, workflowRunIds ...string) (*contracts.WorkflowEvent, error) {
workflowEvent := &contracts.WorkflowEvent{}
var stepRunId string
@@ -764,13 +982,13 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
if workflowEvent.ResourceType == contracts.ResourceType_RESOURCE_TYPE_STEP_RUN {
// determine if this step run matches the workflow run id
stepRun, err := s.repo.StepRun().GetStepRunForEngine(tenantId, stepRunId)
stepRun, err := s.repo.StepRun().GetStepRunForEngine(context.Background(), tenantId, stepRunId)
if err != nil {
return nil, err
}
if sqlchelpers.UUIDToStr(stepRun.WorkflowRunId) != workflowRunId {
if !contains(workflowRunIds, sqlchelpers.UUIDToStr(stepRun.WorkflowRunId)) {
// this is an expected error, so we don't return it
return nil, nil
}
@@ -784,7 +1002,7 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
return nil, err
}
streamEvent, err := s.repo.StreamEvent().GetStreamEvent(tenantId, streamEventId)
streamEvent, err := s.repo.StreamEvent().GetStreamEvent(context.Background(), tenantId, streamEventId)
if err != nil {
return nil, err
@@ -794,7 +1012,7 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
}
} else if workflowEvent.ResourceType == contracts.ResourceType_RESOURCE_TYPE_WORKFLOW_RUN {
if workflowEvent.ResourceId != workflowRunId {
if !contains(workflowRunIds, workflowEvent.ResourceId) {
return nil, nil
}
@@ -803,3 +1021,104 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
return workflowEvent, nil
}
func (s *DispatcherImpl) isMatchingWorkflowRun(task *msgqueue.Message, workflowRunIds ...string) (string, bool) {
if task.ID != "workflow-run-finished" {
return "", false
}
workflowRunId := task.Payload["workflow_run_id"].(string)
if contains(workflowRunIds, workflowRunId) {
return workflowRunId, true
}
return "", false
}
func (s *DispatcherImpl) toWorkflowRunEvent(tenantId string, workflowRuns []*dbsqlc.ListWorkflowRunsRow) ([]*contracts.WorkflowRunEvent, error) {
workflowRunIds := make([]string, 0)
for _, workflowRun := range workflowRuns {
if workflowRun.WorkflowRun.Status != dbsqlc.WorkflowRunStatusFAILED && workflowRun.WorkflowRun.Status != dbsqlc.WorkflowRunStatusSUCCEEDED {
continue
}
workflowRunId := sqlchelpers.UUIDToStr(workflowRun.WorkflowRun.ID)
workflowRunIds = append(workflowRunIds, workflowRunId)
}
res := make([]*contracts.WorkflowRunEvent, 0)
// get step run results for each workflow run
mappedStepRunResults, err := s.getStepResultsForWorkflowRun(tenantId, workflowRunIds)
if err != nil {
return nil, err
}
for workflowRunId, stepRunResults := range mappedStepRunResults {
res = append(res, &contracts.WorkflowRunEvent{
WorkflowRunId: workflowRunId,
EventType: contracts.WorkflowRunEventType_WORKFLOW_RUN_EVENT_TYPE_FINISHED,
EventTimestamp: timestamppb.Now(),
Results: stepRunResults,
})
}
return res, nil
}
func (s *DispatcherImpl) getStepResultsForWorkflowRun(tenantId string, workflowRunIds []string) (map[string][]*contracts.StepRunResult, error) {
stepRuns, err := s.repo.StepRun().ListStepRuns(context.Background(), tenantId, &repository.ListStepRunsOpts{
WorkflowRunIds: workflowRunIds,
})
if err != nil {
return nil, err
}
res := make(map[string][]*contracts.StepRunResult)
for _, stepRun := range stepRuns {
resStepRun := &contracts.StepRunResult{
StepRunId: sqlchelpers.UUIDToStr(stepRun.StepRun.ID),
StepReadableId: stepRun.StepReadableId.String,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
}
if stepRun.StepRun.Error.Valid {
resStepRun.Error = &stepRun.StepRun.Error.String
}
if stepRun.StepRun.CancelledReason.Valid {
errString := fmt.Sprintf("this step run was cancelled due to %s", stepRun.StepRun.CancelledReason.String)
resStepRun.Error = &errString
}
if stepRun.StepRun.Output != nil {
resStepRun.Output = repository.StringPtr(string(stepRun.StepRun.Output))
}
workflowRunId := sqlchelpers.UUIDToStr(stepRun.WorkflowRunId)
if currResults, ok := res[workflowRunId]; ok {
res[workflowRunId] = append(currResults, resStepRun)
} else {
res[workflowRunId] = []*contracts.StepRunResult{resStepRun}
}
}
return res, nil
}
func contains(s []string, e string) bool {
for _, a := range s {
if a == e {
return true
}
}
return false
}

View File

@@ -33,7 +33,7 @@ func (a *GRPCAuthN) Middleware(ctx context.Context) (context.Context, error) {
return nil, forbidden
}
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(token)
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(ctx, token)
if err != nil {
a.l.Debug().Err(err).Msgf("error validating tenant token: %s", err)
@@ -42,7 +42,7 @@ func (a *GRPCAuthN) Middleware(ctx context.Context) (context.Context, error) {
}
// get the tenant id
queriedTenant, err := a.config.EngineRepository.Tenant().GetTenantByID(tenantId)
queriedTenant, err := a.config.EngineRepository.Tenant().GetTenantByID(ctx, tenantId)
if err != nil {
a.l.Debug().Err(err).Msgf("error getting tenant by id: %s", err)

View File

@@ -41,7 +41,7 @@ func (i *IngestorImpl) ReplaySingleEvent(ctx context.Context, req *contracts.Rep
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
oldEvent, err := i.eventRepository.GetEventForEngine(tenantId, req.EventId)
oldEvent, err := i.eventRepository.GetEventForEngine(ctx, tenantId, req.EventId)
if err != nil {
return nil, err
@@ -79,7 +79,7 @@ func (i *IngestorImpl) PutStreamEvent(ctx context.Context, req *contracts.PutStr
metadata = []byte(req.Metadata)
}
streamEvent, err := i.streamEventRepository.PutStreamEvent(tenantId, &repository.CreateStreamEventOpts{
streamEvent, err := i.streamEventRepository.PutStreamEvent(ctx, tenantId, &repository.CreateStreamEventOpts{
StepRunId: req.StepRunId,
CreatedAt: createdAt,
Message: req.Message,
@@ -122,7 +122,7 @@ func (i *IngestorImpl) PutLog(ctx context.Context, req *contracts.PutLogRequest)
metadata = []byte(req.Metadata)
}
_, err := i.logRepository.PutLog(tenantId, &repository.CreateLogLineOpts{
_, err := i.logRepository.PutLog(ctx, tenantId, &repository.CreateLogLineOpts{
StepRunId: req.StepRunId,
CreatedAt: createdAt,
Message: req.Message,

View File

@@ -16,9 +16,12 @@ import (
func (t *TickerImpl) runPollCronSchedules(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: polling cron schedules")
crons, err := t.repo.Ticker().PollCronSchedules(t.tickerId)
crons, err := t.repo.Ticker().PollCronSchedules(ctx, t.tickerId)
if err != nil {
t.l.Err(err).Msg("could not poll cron schedules")
@@ -98,9 +101,12 @@ func (t *TickerImpl) handleScheduleCron(ctx context.Context, cron *dbsqlc.PollCr
func (t *TickerImpl) runCronWorkflow(ctx context.Context, tenantId, workflowVersionId, cron, cronParentId string, input []byte) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: running workflow %s", workflowVersionId)
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(tenantId, workflowVersionId)
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(ctx, tenantId, workflowVersionId)
if err != nil {
t.l.Err(err).Msg("could not get workflow version")
@@ -122,7 +128,7 @@ func (t *TickerImpl) runCronWorkflow(ctx context.Context, tenantId, workflowVers
return
}
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(tenantId, workflowRunId)
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(ctx, tenantId, workflowRunId)
if err != nil {
t.l.Err(err).Msg("could not list job runs for workflow run")

View File

@@ -2,6 +2,7 @@ package ticker
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/datautils"
"github.com/hatchet-dev/hatchet/internal/msgqueue"
@@ -11,9 +12,12 @@ import (
func (t *TickerImpl) runPollGetGroupKeyRuns(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: polling get group key runs")
getGroupKeyRuns, err := t.repo.Ticker().PollGetGroupKeyRuns(t.tickerId)
getGroupKeyRuns, err := t.repo.Ticker().PollGetGroupKeyRuns(ctx, t.tickerId)
if err != nil {
t.l.Err(err).Msg("could not poll get group key runs")

View File

@@ -16,9 +16,12 @@ import (
func (t *TickerImpl) runPollSchedules(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: polling workflow schedules")
scheduledWorkflows, err := t.repo.Ticker().PollScheduledWorkflows(t.tickerId)
scheduledWorkflows, err := t.repo.Ticker().PollScheduledWorkflows(ctx, t.tickerId)
if err != nil {
t.l.Err(err).Msg("could not poll workflow schedules")
@@ -111,9 +114,12 @@ func (t *TickerImpl) handleScheduleWorkflow(ctx context.Context, scheduledWorkfl
func (t *TickerImpl) runScheduledWorkflow(ctx context.Context, tenantId, workflowVersionId, scheduledWorkflowId string, scheduled *dbsqlc.PollScheduledWorkflowsRow) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: running workflow %s", workflowVersionId)
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(tenantId, workflowVersionId)
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(ctx, tenantId, workflowVersionId)
if err != nil {
t.l.Err(err).Msg("could not get workflow version")
@@ -157,7 +163,7 @@ func (t *TickerImpl) runScheduledWorkflow(ctx context.Context, tenantId, workflo
return
}
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(tenantId, workflowRunId)
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(ctx, tenantId, workflowRunId)
if err != nil {
t.l.Err(err).Msg("could not list job runs for workflow run")

View File

@@ -2,6 +2,7 @@ package ticker
import (
"context"
"time"
"github.com/hatchet-dev/hatchet/internal/datautils"
"github.com/hatchet-dev/hatchet/internal/msgqueue"
@@ -11,9 +12,12 @@ import (
func (t *TickerImpl) runPollStepRuns(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: polling step runs")
stepRuns, err := t.repo.Ticker().PollStepRuns(t.tickerId)
stepRuns, err := t.repo.Ticker().PollStepRuns(ctx, t.tickerId)
if err != nil {
t.l.Err(err).Msg("could not poll step runs")

View File

@@ -112,7 +112,7 @@ func (t *TickerImpl) Start() (func() error, error) {
t.l.Debug().Msgf("starting ticker %s", t.tickerId)
// register the ticker
_, err := t.repo.Ticker().CreateNewTicker(&repository.CreateTickerOpts{
_, err := t.repo.Ticker().CreateNewTicker(ctx, &repository.CreateTickerOpts{
ID: t.tickerId,
})
@@ -186,7 +186,7 @@ func (t *TickerImpl) Start() (func() error, error) {
_, err = t.s.NewJob(
gocron.DurationJob(time.Minute*5),
gocron.NewTask(
t.runStreamEventCleanup(),
t.runStreamEventCleanup(ctx),
),
)
@@ -202,18 +202,21 @@ func (t *TickerImpl) Start() (func() error, error) {
cancel()
if err := t.s.Shutdown(); err != nil {
return fmt.Errorf("could not shutdown scheduler: %w", err)
}
deleteCtx, deleteCancel := context.WithTimeout(context.Background(), 5*time.Second)
defer deleteCancel()
// delete the ticker
err = t.repo.Ticker().Delete(t.tickerId)
err = t.repo.Ticker().Delete(deleteCtx, t.tickerId)
if err != nil {
t.l.Err(err).Msg("could not delete ticker")
return err
}
if err := t.s.Shutdown(); err != nil {
return fmt.Errorf("could not shutdown scheduler: %w", err)
}
return nil
}
@@ -227,7 +230,7 @@ func (t *TickerImpl) runUpdateHeartbeat(ctx context.Context) func() {
now := time.Now().UTC()
// update the heartbeat
_, err := t.repo.Ticker().UpdateTicker(t.tickerId, &repository.UpdateTickerOpts{
_, err := t.repo.Ticker().UpdateTicker(ctx, t.tickerId, &repository.UpdateTickerOpts{
LastHeartbeatAt: &now,
})
@@ -237,11 +240,14 @@ func (t *TickerImpl) runUpdateHeartbeat(ctx context.Context) func() {
}
}
func (t *TickerImpl) runStreamEventCleanup() func() {
func (t *TickerImpl) runStreamEventCleanup(ctx context.Context) func() {
return func() {
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
t.l.Debug().Msgf("ticker: cleaning up stream event")
err := t.repo.StreamEvent().CleanupStreamEvents()
err := t.repo.StreamEvent().CleanupStreamEvents(ctx)
if err != nil {
t.l.Err(err).Msg("could not cleanup stream events")

View File

@@ -1,6 +1,7 @@
package testutils
import (
"context"
"errors"
"os"
"path"
@@ -72,7 +73,7 @@ func Prepare(t *testing.T) {
}
}
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(tenantId, "default")
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(context.Background(), tenantId, "default")
if err != nil {
t.Fatalf("could not generate default token: %v", err)
}

View File

@@ -3,32 +3,236 @@ package client
import (
"context"
"errors"
"fmt"
"io"
"sync"
"time"
"github.com/rs/zerolog"
"golang.org/x/sync/errgroup"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
dispatchercontracts "github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
"github.com/hatchet-dev/hatchet/internal/validator"
)
type RunEvent *dispatchercontracts.WorkflowEvent
type WorkflowEvent *dispatchercontracts.WorkflowEvent
type WorkflowRunEvent *dispatchercontracts.WorkflowRunEvent
type StreamEvent struct {
Message []byte
}
type RunHandler func(event RunEvent) error
type RunHandler func(event WorkflowEvent) error
type StreamHandler func(event StreamEvent) error
type WorkflowRunEventHandler func(event WorkflowRunEvent) error
type WorkflowRunsListener struct {
constructor func(context.Context) (dispatchercontracts.Dispatcher_SubscribeToWorkflowRunsClient, error)
client dispatchercontracts.Dispatcher_SubscribeToWorkflowRunsClient
clientMu sync.RWMutex
l *zerolog.Logger
// map of workflow run ids to a list of handlers
handlers sync.Map
}
func (r *subscribeClientImpl) newWorkflowRunsListener(
ctx context.Context,
) (*WorkflowRunsListener, error) {
constructor := func(ctx context.Context) (dispatchercontracts.Dispatcher_SubscribeToWorkflowRunsClient, error) {
return r.client.SubscribeToWorkflowRuns(r.ctx.newContext(ctx))
}
w := &WorkflowRunsListener{
constructor: constructor,
l: r.l,
}
err := w.retrySubscribe(ctx)
if err != nil {
return nil, err
}
return w, nil
}
func (w *WorkflowRunsListener) retrySubscribe(ctx context.Context) error {
w.clientMu.Lock()
defer w.clientMu.Unlock()
retries := 0
for retries < DefaultActionListenerRetryCount {
if retries > 0 {
time.Sleep(DefaultActionListenerRetryInterval)
}
client, err := w.constructor(ctx)
if err != nil {
retries++
w.l.Error().Err(err).Msgf("could not resubscribe to the listener")
continue
}
w.client = client
// listen for all the same workflow runs
var rangeErr error
w.handlers.Range(func(key, value interface{}) bool {
workflowRunId := key.(string)
err := w.client.Send(&dispatchercontracts.SubscribeToWorkflowRunsRequest{
WorkflowRunId: workflowRunId,
})
if err != nil {
w.l.Error().Err(err).Msgf("could not subscribe to the worker")
rangeErr = err
return false
}
return true
})
if rangeErr != nil {
continue
}
return nil
}
return fmt.Errorf("could not subscribe to the worker after %d retries", retries)
}
type threadSafeHandlers struct {
handlers []WorkflowRunEventHandler
mu sync.RWMutex
}
func (l *WorkflowRunsListener) AddWorkflowRun(
workflowRunId string,
handler WorkflowRunEventHandler,
) error {
handlers, _ := l.handlers.LoadOrStore(workflowRunId, &threadSafeHandlers{
handlers: []WorkflowRunEventHandler{},
})
h := handlers.(*threadSafeHandlers)
h.mu.Lock()
h.handlers = append(h.handlers, handler)
l.handlers.Store(workflowRunId, h)
h.mu.Unlock()
err := l.retrySend(workflowRunId)
if err != nil {
return err
}
return nil
}
func (l *WorkflowRunsListener) retrySend(workflowRunId string) error {
l.clientMu.RLock()
defer l.clientMu.RUnlock()
if l.client == nil {
return fmt.Errorf("client is not connected")
}
for i := 0; i < DefaultActionListenerRetryCount; i++ {
err := l.client.Send(&dispatchercontracts.SubscribeToWorkflowRunsRequest{
WorkflowRunId: workflowRunId,
})
if err == nil {
return nil
}
time.Sleep(DefaultActionListenerRetryInterval)
}
return fmt.Errorf("could not send to the worker after %d retries", DefaultActionListenerRetryCount)
}
func (l *WorkflowRunsListener) Listen(ctx context.Context) error {
for {
l.clientMu.RLock()
event, err := l.client.Recv()
l.clientMu.RUnlock()
if err != nil {
if errors.Is(err, io.EOF) || status.Code(err) == codes.Canceled {
return nil
}
retryErr := l.retrySubscribe(ctx)
if retryErr != nil {
return retryErr
}
continue
}
if err := l.handleWorkflowRun(event); err != nil {
return err
}
}
}
func (l *WorkflowRunsListener) Close() error {
return l.client.CloseSend()
}
func (l *WorkflowRunsListener) handleWorkflowRun(event *dispatchercontracts.WorkflowRunEvent) error {
// find all handlers for this workflow run
handlers, ok := l.handlers.Load(event.WorkflowRunId)
if !ok {
return nil
}
eg := errgroup.Group{}
h := handlers.(*threadSafeHandlers)
h.mu.RLock()
for _, handler := range h.handlers {
handlerCp := handler
eg.Go(func() error {
return handlerCp(event)
})
}
h.mu.RUnlock()
err := eg.Wait()
return err
}
type SubscribeClient interface {
On(ctx context.Context, workflowRunId string, handler RunHandler) error
Stream(ctx context.Context, workflowRunId string, handler StreamHandler) error
SubscribeToWorkflowRunEvents(ctx context.Context) (*WorkflowRunsListener, error)
}
type ClientEventListener interface {
OnRunEvent(ctx context.Context, event *RunEvent) error
OnWorkflowEvent(ctx context.Context, event *WorkflowEvent) error
}
type subscribeClientImpl struct {
@@ -111,3 +315,29 @@ func (r *subscribeClientImpl) Stream(ctx context.Context, workflowRunId string,
}
}
}
func (r *subscribeClientImpl) SubscribeToWorkflowRunEvents(ctx context.Context) (*WorkflowRunsListener, error) {
l, err := r.newWorkflowRunsListener(ctx)
if err != nil {
return nil, err
}
go func() {
defer func() {
err := l.Close()
if err != nil {
r.l.Error().Err(err).Msg("failed to close workflow run events listener")
}
}()
err := l.Listen(ctx)
if err != nil {
r.l.Error().Err(err).Msg("failed to listen for workflow run events")
}
}()
return l, nil
}

View File

@@ -1,36 +1,37 @@
package worker
import (
"context"
"encoding/json"
"fmt"
"time"
"github.com/google/uuid"
"github.com/rs/zerolog"
"github.com/hatchet-dev/hatchet/pkg/client"
"github.com/hatchet-dev/hatchet/pkg/client/rest"
dispatchercontracts "github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
)
type ChildWorkflow struct {
workflowRunId string
client client.Client
l *zerolog.Logger
// client client.Client
l *zerolog.Logger
listener *client.WorkflowRunsListener
}
type ChildWorkflowResult struct {
workflowRun *rest.WorkflowRun
workflowRun *dispatchercontracts.WorkflowRunEvent
}
func (r *ChildWorkflowResult) StepOutput(key string, v interface{}) error {
var outputBytes []byte
for _, jobRun := range *r.workflowRun.JobRuns {
for _, stepRun := range *jobRun.StepRuns {
if stepRun.Step.ReadableId == key && stepRun.Output != nil {
outputBytes = []byte(*stepRun.Output)
for _, stepRunResult := range r.workflowRun.Results {
if stepRunResult.StepReadableId == key {
if stepRunResult.Error != nil {
return fmt.Errorf("step run failed: %s", *stepRunResult.Error)
}
if stepRunResult.Output != nil {
outputBytes = []byte(*stepRunResult.Output)
}
}
}
@@ -47,89 +48,28 @@ func (r *ChildWorkflowResult) StepOutput(key string, v interface{}) error {
}
func (c *ChildWorkflow) Result() (*ChildWorkflowResult, error) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
resChan := make(chan *ChildWorkflowResult)
workflowErrChan := make(chan error)
errChan := make(chan error)
f := func() {
resp, err := c.client.API().WorkflowRunGetWithResponse(
ctx,
uuid.MustParse(c.client.TenantId()),
uuid.MustParse(c.workflowRunId),
)
if err != nil {
errChan <- fmt.Errorf("failed to get workflow run: %w", err)
return
}
if workflowRun := resp.JSON200; workflowRun != nil {
if workflowRun.Status == rest.SUCCEEDED {
// write the workflow run to the channel
resChan <- &ChildWorkflowResult{
workflowRun: workflowRun,
}
}
if workflowRun.Status == rest.FAILED || workflowRun.Status == rest.CANCELLED {
// write the error to the channel
workflowErrChan <- fmt.Errorf("workflow run failed with status %s", workflowRun.Status)
}
} else {
errChan <- fmt.Errorf("request failed with status %d", resp.StatusCode())
return
}
}
// start two goroutines: one which polls the API for the workflow run result, and one which listens for
// workflow finished events
go func() {
f()
// poll the API for the workflow run result
ticker := time.NewTicker(5 * time.Second)
for {
err := c.listener.AddWorkflowRun(
c.workflowRunId,
func(event client.WorkflowRunEvent) error {
// non-blocking send
select {
case <-ticker.C:
f()
case <-ctx.Done():
return
case resChan <- &ChildWorkflowResult{
workflowRun: event,
}: // continue
default:
}
}
}()
go func() {
// listen for workflow finished events
err := c.client.Subscribe().On(
ctx,
c.workflowRunId,
func(event client.RunEvent) error {
if event.ResourceType == dispatchercontracts.ResourceType_RESOURCE_TYPE_WORKFLOW_RUN {
if event.EventType == dispatchercontracts.ResourceEventType_RESOURCE_EVENT_TYPE_COMPLETED {
f()
}
}
return nil
},
)
return nil
},
)
if err != nil {
errChan <- fmt.Errorf("failed to listen for workflow events: %w", err)
}
}()
select {
case res := <-resChan:
return res, nil
case err := <-workflowErrChan:
return nil, err
case err := <-errChan:
c.l.Err(err).Msg("error occurred")
return nil, err
if err != nil {
return nil, fmt.Errorf("failed to listen for workflow events: %w", err)
}
res := <-resChan
return res, nil
}

View File

@@ -72,8 +72,10 @@ type hatchetContext struct {
c client.Client
l *zerolog.Logger
i int
indexMu sync.Mutex
i int
indexMu sync.Mutex
listener *client.WorkflowRunsListener
listenerMu sync.Mutex
}
func newHatchetContext(
@@ -176,7 +178,32 @@ type SpawnWorkflowOpts struct {
Key *string
}
func (h *hatchetContext) saveOrLoadListener() (*client.WorkflowRunsListener, error) {
h.listenerMu.Lock()
defer h.listenerMu.Unlock()
if h.listener != nil {
return h.listener, nil
}
listener, err := h.client().Subscribe().SubscribeToWorkflowRunEvents(h)
if err != nil {
return nil, fmt.Errorf("failed to subscribe to workflow run events: %w", err)
}
h.listener = listener
return listener, nil
}
func (h *hatchetContext) SpawnWorkflow(workflowName string, input any, opts *SpawnWorkflowOpts) (*ChildWorkflow, error) {
listener, err := h.saveOrLoadListener()
if err != nil {
return nil, err
}
workflowRunId, err := h.client().Admin().RunChildWorkflow(
workflowName,
input,
@@ -197,8 +224,8 @@ func (h *hatchetContext) SpawnWorkflow(workflowName string, input any, opts *Spa
return &ChildWorkflow{
workflowRunId: workflowRunId,
client: h.client(),
l: h.l,
listener: listener,
}, nil
}

View File

@@ -407,6 +407,8 @@ func (w *Worker) startStepRun(ctx context.Context, assignedAction *client.Action
// the service-specific middleware
return w.middlewares.runAll(hCtx, func(ctx HatchetContext) error {
return svc.mws.runAll(ctx, func(ctx HatchetContext) error {
defer cancel()
args := []any{ctx}
if arg != nil {