From d63b66a837fb703fe19934823343a2e4e509b984 Mon Sep 17 00:00:00 2001 From: abelanger5 Date: Mon, 29 Jan 2024 21:00:28 -0800 Subject: [PATCH] feat: concurrency groups (#135) * first pass at moving controllers around * feat: concurrency limits for strategy CANCEL_IN_PROGRESS * fix: linting * chore: bump python sdk version --- .github/workflows/lint.yml | 5 +- api-contracts/dispatcher/dispatcher.proto | 61 +- api-contracts/workflows/workflows.proto | 14 + cmd/hatchet-admin/cli/seed.go | 3 + cmd/hatchet-engine/main.go | 43 +- examples/limit-concurrency/main.go | 145 ++++ examples/simple/main.go | 7 +- .../docs/pages/go-sdk/creating-a-workflow.mdx | 57 ++ .../pages/python-sdk/creating-a-workflow.mdx | 39 + internal/config/server/server.go | 2 +- internal/repository/get_group_key_run.go | 44 + .../prisma/dbsqlc/get_group_key_runs.sql | 21 + .../prisma/dbsqlc/get_group_key_runs.sql.go | 88 ++ internal/repository/prisma/dbsqlc/models.go | 98 ++- internal/repository/prisma/dbsqlc/schema.sql | 77 +- internal/repository/prisma/dbsqlc/sqlc.yaml | 1 + .../prisma/dbsqlc/workflow_runs.sql | 98 +++ .../prisma/dbsqlc/workflow_runs.sql.go | 226 ++++- .../repository/prisma/dbsqlc/workflows.sql | 26 +- .../repository/prisma/dbsqlc/workflows.sql.go | 85 +- .../repository/prisma/get_group_key_run.go | 188 +++++ internal/repository/prisma/repository.go | 62 +- internal/repository/prisma/sqlchelpers/int.go | 10 + internal/repository/prisma/step_run.go | 6 + internal/repository/prisma/ticker.go | 10 + internal/repository/prisma/worker.go | 20 + internal/repository/prisma/workflow.go | 55 +- internal/repository/prisma/workflow_run.go | 59 +- internal/repository/repository.go | 1 + internal/repository/step_run.go | 2 + internal/repository/ticker.go | 3 + internal/repository/worker.go | 3 + internal/repository/workflow.go | 14 + internal/repository/workflow_run.go | 53 +- .../services/admin/contracts/workflows.pb.go | 784 +++++++++++------- internal/services/admin/server.go | 20 + .../events}/controller.go | 36 +- .../jobs}/controller.go | 80 +- .../controllers/workflows/controller.go | 321 +++++++ .../services/controllers/workflows/queue.go | 508 ++++++++++++ .../dispatcher/contracts/dispatcher.pb.go | 580 +++++++++---- .../contracts/dispatcher_grpc.pb.go | 62 +- internal/services/dispatcher/dispatcher.go | 51 +- internal/services/dispatcher/server.go | 196 ++++- .../services/shared/tasktypes/group_key.go | 80 ++ internal/services/shared/tasktypes/step.go | 9 + internal/services/shared/tasktypes/ticker.go | 18 + .../services/shared/tasktypes/workflow.go | 34 + .../ticker/get_group_key_run_timeout.go | 142 ++++ internal/services/ticker/ticker.go | 44 + internal/taskqueue/rabbitmq/rabbitmq.go | 71 +- internal/taskqueue/taskqueue.go | 9 +- pkg/client/admin.go | 18 + pkg/client/dispatcher.go | 101 ++- pkg/client/types/file.go | 16 + pkg/worker/service.go | 9 +- pkg/worker/worker.go | 136 ++- pkg/worker/workflow.go | 48 +- .../20240129040510_v0_8_0/migration.sql | 79 ++ prisma/schema.prisma | 119 ++- .../examples/concurrency-limit/event_test.py | 13 + .../examples/concurrency-limit/worker.py | 32 + python-sdk/hatchet_sdk/clients/dispatcher.py | 23 +- python-sdk/hatchet_sdk/dispatcher_pb2.py | 40 +- python-sdk/hatchet_sdk/dispatcher_pb2.pyi | 63 +- python-sdk/hatchet_sdk/dispatcher_pb2_grpc.py | 53 +- python-sdk/hatchet_sdk/hatchet.py | 9 + python-sdk/hatchet_sdk/worker.py | 149 +++- python-sdk/hatchet_sdk/workflow.py | 22 +- python-sdk/hatchet_sdk/workflows_pb2.py | 72 +- python-sdk/hatchet_sdk/workflows_pb2.pyi | 26 +- python-sdk/pyproject.toml | 2 +- 72 files changed, 4777 insertions(+), 924 deletions(-) create mode 100644 examples/limit-concurrency/main.go create mode 100644 internal/repository/get_group_key_run.go create mode 100644 internal/repository/prisma/dbsqlc/get_group_key_runs.sql create mode 100644 internal/repository/prisma/dbsqlc/get_group_key_runs.sql.go create mode 100644 internal/repository/prisma/get_group_key_run.go create mode 100644 internal/repository/prisma/sqlchelpers/int.go rename internal/services/{eventscontroller => controllers/events}/controller.go (87%) rename internal/services/{jobscontroller => controllers/jobs}/controller.go (95%) create mode 100644 internal/services/controllers/workflows/controller.go create mode 100644 internal/services/controllers/workflows/queue.go create mode 100644 internal/services/shared/tasktypes/group_key.go create mode 100644 internal/services/shared/tasktypes/workflow.go create mode 100644 internal/services/ticker/get_group_key_run_timeout.go create mode 100644 prisma/migrations/20240129040510_v0_8_0/migration.sql create mode 100644 python-sdk/examples/concurrency-limit/event_test.py create mode 100644 python-sdk/examples/concurrency-limit/worker.py diff --git a/.github/workflows/lint.yml b/.github/workflows/lint.yml index ed70caede..e49470eea 100644 --- a/.github/workflows/lint.yml +++ b/.github/workflows/lint.yml @@ -9,7 +9,10 @@ jobs: steps: - uses: actions/checkout@v4 - - uses: actions/setup-go@v5 + - name: Setup Go + uses: actions/setup-go@v5 + with: + go-version: "1.21" - uses: actions/cache@v3 with: diff --git a/api-contracts/dispatcher/dispatcher.proto b/api-contracts/dispatcher/dispatcher.proto index a3985a3b4..e2237b8fa 100644 --- a/api-contracts/dispatcher/dispatcher.proto +++ b/api-contracts/dispatcher/dispatcher.proto @@ -9,7 +9,9 @@ service Dispatcher { rpc Listen(WorkerListenRequest) returns (stream AssignedAction) {} - rpc SendActionEvent(ActionEvent) returns (ActionEventResponse) {} + rpc SendStepActionEvent(StepActionEvent) returns (ActionEventResponse) {} + + rpc SendGroupKeyActionEvent(GroupKeyActionEvent) returns (ActionEventResponse) {} rpc Unsubscribe(WorkerUnsubscribeRequest) returns (WorkerUnsubscribeResponse) {} } @@ -39,35 +41,42 @@ message WorkerRegisterResponse { enum ActionType { START_STEP_RUN = 0; CANCEL_STEP_RUN = 1; + START_GET_GROUP_KEY = 2; } message AssignedAction { // the tenant id string tenantId = 1; + // the workflow run id (optional) + string workflowRunId = 2; + + // the get group key run id (optional) + string getGroupKeyRunId = 3; + // the job id - string jobId = 2; + string jobId = 4; // the job name - string jobName = 3; + string jobName = 5; // the job run id - string jobRunId = 4; + string jobRunId = 6; // the step id - string stepId = 5; + string stepId = 7; // the step run id - string stepRunId = 6; + string stepRunId = 8; // the action id - string actionId = 7; + string actionId = 9; // the action type - ActionType actionType = 8; + ActionType actionType = 10; // the action payload - string actionPayload = 9; + string actionPayload = 11; } message WorkerListenRequest { @@ -88,14 +97,42 @@ message WorkerUnsubscribeResponse { string workerId = 2; } -enum ActionEventType { +enum GroupKeyActionEventType { + GROUP_KEY_EVENT_TYPE_UNKNOWN = 0; + GROUP_KEY_EVENT_TYPE_STARTED = 1; + GROUP_KEY_EVENT_TYPE_COMPLETED = 2; + GROUP_KEY_EVENT_TYPE_FAILED = 3; +} + +message GroupKeyActionEvent { + // the id of the worker + string workerId = 1; + + // the id of the job + string workflowRunId = 2; + + string getGroupKeyRunId = 3; + + // the action id + string actionId = 4; + + google.protobuf.Timestamp eventTimestamp = 5; + + // the step event type + GroupKeyActionEventType eventType = 6; + + // the event payload + string eventPayload = 7; +} + +enum StepActionEventType { STEP_EVENT_TYPE_UNKNOWN = 0; STEP_EVENT_TYPE_STARTED = 1; STEP_EVENT_TYPE_COMPLETED = 2; STEP_EVENT_TYPE_FAILED = 3; } -message ActionEvent { +message StepActionEvent { // the id of the worker string workerId = 1; @@ -117,7 +154,7 @@ message ActionEvent { google.protobuf.Timestamp eventTimestamp = 7; // the step event type - ActionEventType eventType = 8; + StepActionEventType eventType = 8; // the event payload string eventPayload = 9; diff --git a/api-contracts/workflows/workflows.proto b/api-contracts/workflows/workflows.proto index 3d900d5e5..ffd3d108d 100644 --- a/api-contracts/workflows/workflows.proto +++ b/api-contracts/workflows/workflows.proto @@ -13,6 +13,7 @@ service WorkflowService { rpc GetWorkflowByName(GetWorkflowByNameRequest) returns (Workflow); rpc ListWorkflowsForEvent(ListWorkflowsForEventRequest) returns (ListWorkflowsResponse); rpc DeleteWorkflow(DeleteWorkflowRequest) returns (Workflow); + } message PutWorkflowRequest { @@ -28,6 +29,19 @@ message CreateWorkflowVersionOpts { repeated string cron_triggers = 5; // (optional) cron triggers for the workflow repeated google.protobuf.Timestamp scheduled_triggers = 6; // (optional) scheduled triggers for the workflow repeated CreateWorkflowJobOpts jobs = 7; // (required) the workflow jobs + WorkflowConcurrencyOpts concurrency = 8; // (optional) the workflow concurrency options +} + +enum ConcurrencyLimitStrategy { + CANCEL_IN_PROGRESS = 0; + DROP_NEWEST = 1; + QUEUE_NEWEST = 2; +} + +message WorkflowConcurrencyOpts { + string action = 1; // (required) the action id for getting the concurrency group + int32 max_runs = 2; // (optional) the maximum number of concurrent workflow runs, default 1 + ConcurrencyLimitStrategy limit_strategy = 3; // (optional) the strategy to use when the concurrency limit is reached, default CANCEL_IN_PROGRESS } // CreateWorkflowJobOpts represents options to create a workflow job. diff --git a/cmd/hatchet-admin/cli/seed.go b/cmd/hatchet-admin/cli/seed.go index 50e2e8c83..fe31404cc 100644 --- a/cmd/hatchet-admin/cli/seed.go +++ b/cmd/hatchet-admin/cli/seed.go @@ -139,6 +139,9 @@ func seedDev(repo repository.Repository, tenantId string) error { Name: "Preview", }, }, + Concurrency: &repository.CreateWorkflowConcurrencyOpts{ + Action: "test:concurrency", + }, Jobs: []repository.CreateWorkflowJobOpts{ { Name: "job-name", diff --git a/cmd/hatchet-engine/main.go b/cmd/hatchet-engine/main.go index 87f4b622a..c78ed5e49 100644 --- a/cmd/hatchet-engine/main.go +++ b/cmd/hatchet-engine/main.go @@ -9,12 +9,13 @@ import ( "github.com/hatchet-dev/hatchet/internal/config/loader" "github.com/hatchet-dev/hatchet/internal/services/admin" + "github.com/hatchet-dev/hatchet/internal/services/controllers/events" + "github.com/hatchet-dev/hatchet/internal/services/controllers/jobs" + "github.com/hatchet-dev/hatchet/internal/services/controllers/workflows" "github.com/hatchet-dev/hatchet/internal/services/dispatcher" - "github.com/hatchet-dev/hatchet/internal/services/eventscontroller" "github.com/hatchet-dev/hatchet/internal/services/grpc" "github.com/hatchet-dev/hatchet/internal/services/heartbeat" "github.com/hatchet-dev/hatchet/internal/services/ingestor" - "github.com/hatchet-dev/hatchet/internal/services/jobscontroller" "github.com/hatchet-dev/hatchet/internal/services/ticker" "github.com/hatchet-dev/hatchet/internal/telemetry" "github.com/hatchet-dev/hatchet/pkg/cmdutils" @@ -171,10 +172,10 @@ func startEngineOrDie(cf *loader.ConfigLoader, interruptCh <-chan interface{}) { if sc.HasService("eventscontroller") { // create separate events controller process go func() { - ec, err := eventscontroller.New( - eventscontroller.WithTaskQueue(sc.TaskQueue), - eventscontroller.WithRepository(sc.Repository), - eventscontroller.WithLogger(sc.Logger), + ec, err := events.New( + events.WithTaskQueue(sc.TaskQueue), + events.WithRepository(sc.Repository), + events.WithLogger(sc.Logger), ) if err != nil { @@ -193,10 +194,32 @@ func startEngineOrDie(cf *loader.ConfigLoader, interruptCh <-chan interface{}) { if sc.HasService("jobscontroller") { // create separate jobs controller process go func() { - jc, err := jobscontroller.New( - jobscontroller.WithTaskQueue(sc.TaskQueue), - jobscontroller.WithRepository(sc.Repository), - jobscontroller.WithLogger(sc.Logger), + jc, err := jobs.New( + jobs.WithTaskQueue(sc.TaskQueue), + jobs.WithRepository(sc.Repository), + jobs.WithLogger(sc.Logger), + ) + + if err != nil { + errCh <- err + return + } + + err = jc.Start(ctx) + + if err != nil { + errCh <- err + } + }() + } + + if sc.HasService("workflowscontroller") { + // create separate jobs controller process + go func() { + jc, err := workflows.New( + workflows.WithTaskQueue(sc.TaskQueue), + workflows.WithRepository(sc.Repository), + workflows.WithLogger(sc.Logger), ) if err != nil { diff --git a/examples/limit-concurrency/main.go b/examples/limit-concurrency/main.go new file mode 100644 index 000000000..f05eba38a --- /dev/null +++ b/examples/limit-concurrency/main.go @@ -0,0 +1,145 @@ +package main + +import ( + "context" + "fmt" + "time" + + "github.com/joho/godotenv" + + "github.com/hatchet-dev/hatchet/pkg/client" + "github.com/hatchet-dev/hatchet/pkg/cmdutils" + "github.com/hatchet-dev/hatchet/pkg/worker" +) + +type concurrencyLimitEvent struct { + Index int `json:"index"` +} + +type stepOneOutput struct { + Message string `json:"message"` +} + +func main() { + err := godotenv.Load() + if err != nil { + panic(err) + } + + events := make(chan string, 50) + if err := run(cmdutils.InterruptChan(), events); err != nil { + panic(err) + } +} + +func getConcurrencyKey(ctx worker.HatchetContext) (string, error) { + return "user-create", nil +} + +func run(ch <-chan interface{}, events chan<- string) error { + c, err := client.New() + + if err != nil { + return fmt.Errorf("error creating client: %w", err) + } + + // Create a worker. This automatically reads in a TemporalClient from .env and workflow files from the .hatchet + // directory, but this can be customized with the `worker.WithTemporalClient` and `worker.WithWorkflowFiles` options. + w, err := worker.NewWorker( + worker.WithClient( + c, + ), + ) + if err != nil { + return fmt.Errorf("error creating worker: %w", err) + } + + testSvc := w.NewService("test") + + err = testSvc.On( + worker.Events("concurrency-test-event"), + &worker.WorkflowJob{ + Name: "concurrency-limit", + Description: "This limits concurrency to 1 run at a time.", + Concurrency: worker.Concurrency(getConcurrencyKey).MaxRuns(1), + Steps: []*worker.WorkflowStep{ + worker.Fn(func(ctx worker.HatchetContext) (result *stepOneOutput, err error) { + <-ctx.Done() + fmt.Println("context done, returning") + return nil, nil + }, + ).SetName("step-one"), + }, + }, + ) + if err != nil { + return fmt.Errorf("error registering workflow: %w", err) + } + + interruptCtx, cancel := cmdutils.InterruptContextFromChan(ch) + defer cancel() + + go func() { + err = w.Start(interruptCtx) + + if err != nil { + panic(err) + } + + cancel() + }() + + go func() { + // sleep with interrupt context + select { + case <-interruptCtx.Done(): // context cancelled + fmt.Println("interrupted") + return + case <-time.After(2 * time.Second): // timeout + } + + firstEvent := concurrencyLimitEvent{ + Index: 0, + } + + // push an event + err = c.Event().Push( + context.Background(), + "concurrency-test-event", + firstEvent, + ) + + if err != nil { + panic(err) + } + + select { + case <-interruptCtx.Done(): // context cancelled + fmt.Println("interrupted") + return + case <-time.After(10 * time.Second): //timeout + } + + // push a second event + err = c.Event().Push( + context.Background(), + "concurrency-test-event", + concurrencyLimitEvent{ + Index: 1, + }, + ) + + if err != nil { + panic(err) + } + }() + + for { + select { + case <-interruptCtx.Done(): + return nil + default: + time.Sleep(time.Second) + } + } +} diff --git a/examples/simple/main.go b/examples/simple/main.go index e6be771d8..6d8f7cd7c 100644 --- a/examples/simple/main.go +++ b/examples/simple/main.go @@ -35,6 +35,10 @@ func main() { } } +func getConcurrencyKey(ctx worker.HatchetContext) (string, error) { + return "user-create", nil +} + func run(ch <-chan interface{}, events chan<- string) error { c, err := client.New() @@ -60,6 +64,7 @@ func run(ch <-chan interface{}, events chan<- string) error { &worker.WorkflowJob{ Name: "simple", Description: "This runs after an update to the user model.", + Concurrency: worker.Concurrency(getConcurrencyKey), Steps: []*worker.WorkflowStep{ worker.Fn(func(ctx worker.HatchetContext) (result *stepOneOutput, err error) { input := &userCreateEvent{} @@ -142,6 +147,4 @@ func run(ch <-chan interface{}, events chan<- string) error { time.Sleep(time.Second) } } - - return nil } diff --git a/frontend/docs/pages/go-sdk/creating-a-workflow.mdx b/frontend/docs/pages/go-sdk/creating-a-workflow.mdx index ac3330ddf..bfc86e1c0 100644 --- a/frontend/docs/pages/go-sdk/creating-a-workflow.mdx +++ b/frontend/docs/pages/go-sdk/creating-a-workflow.mdx @@ -155,6 +155,63 @@ userService.On( While this is mostly a convenience method at the moment, we plan to add more features to services in the future, like service-level metrics and service-level retries. +## Concurrency Limits and Fairness + +> __**Note:** this feature is currently in beta, and currently only supports a concurrency strategy which terminates the oldest running workflow run to make room for the new one. This will be expanded in the future to support other strategies.__ + +By default, there are no concurrency limits for Hatchet workflows. Workflow runs are immediately executed as soon as they are triggered (by an event, cron, or schedule). However, you can enforce a concurrency limit by setting the `Concurrency` field on the `WorkflowJob` struct. You can use `worker.Concurrency` and pass in a function with a signature `func (ctx worker.HatchetContext) (string, error)`. This function returns a __concurrency group key__, which is a string that is used to group concurrent executions. For example, the following workflow will only allow 5 concurrent executions for any workflow execution of `concurrency-limit`, since the key is statically set to `my-key`: + +```go +func getConcurrencyKey(ctx worker.HatchetContext) (string, error) { + return "my-key", nil +} + +err = testSvc.On( + worker.Events("concurrency-test-event"), + &worker.WorkflowJob{ + Name: "concurrency-limit", + Description: "This limits concurrency to 1 run at a time.", + Concurrency: worker.Concurrency(getConcurrencyKey).MaxRuns(1), + Steps: []*worker.WorkflowStep{ + // your steps here... + }, + }, +) +``` + +### Use-Case: Enforcing Per-User Concurrency Limits + +You can use the custom concurrency function to enforce per-user concurrency limits. For example, the following workflow will only allow 1 concurrent execution per user: + +```go +type MyUser struct { + UserId string `json:"user_id"` +} + +func getConcurrencyKey(ctx worker.HatchetContext) (string, error) { + event := &MyEvent{} + err := ctx.WorkflowInput(event) + + if err != nil { + return "", err + } + + return event.UserId, nil +} + +err = testSvc.On( + worker.Events("concurrency-test-event"), + &worker.WorkflowJob{ + Name: "concurrency-limit-per-user", + Description: "This limits concurrency to 1 run at a time per user.", + Concurrency: worker.Concurrency(getConcurrencyKey).MaxRuns(1), + Steps: []*worker.WorkflowStep{ + // your steps here... + }, + }, +) +``` + ## Cron Schedules You can declare a cron schedule by passing `worker.Cron` to the `worker.On` method. For example, to trigger a workflow every 5 minutes, you can do the following: diff --git a/frontend/docs/pages/python-sdk/creating-a-workflow.mdx b/frontend/docs/pages/python-sdk/creating-a-workflow.mdx index d486b8019..15154077d 100644 --- a/frontend/docs/pages/python-sdk/creating-a-workflow.mdx +++ b/frontend/docs/pages/python-sdk/creating-a-workflow.mdx @@ -96,6 +96,45 @@ def step1(self, context): pass ``` +## Concurrency Limits and Fairness + +> __**Note:** this feature is currently in beta, and currently only supports a concurrency strategy which terminates the oldest running workflow run to make room for the new one. This will be expanded in the future to support other strategies.__ + +By default, there are no concurrency limits for Hatchet workflows. Workflow runs are immediately executed as soon as they are triggered (by an event, cron, or schedule). However, you can enforce a concurrency limit by decorating a custom function with `hatchet.concurrency`. This function returns a __concurrency group key__, which is a string that is used to group concurrent executions. **Note that this function should not also be used as a `hatchet.step`.** For example, the following workflow will only allow 5 concurrent executions for any workflow execution of `ConcurrencyDemoWorkflow`, since the key is statically set to `my-key`: + +```py +@hatchet.workflow(on_events=["concurrency-test"]) +class ConcurrencyDemoWorkflow: + def __init__(self): + self.my_value = "test" + + @hatchet.concurrency(max_runs=5) + def concurrency(self, context) -> str: + return "concurrency-key" + + @hatchet.step() + def step1(self, context): + print("executed step1") + pass +``` + +### Use-Case: Enforcing Per-User Concurrency Limits + +You can use the custom concurrency function to enforce per-user concurrency limits. For example, the following workflow will only allow 1 concurrent execution per user: + +```py +@hatchet.workflow(on_events=["concurrency-test"]) +class ConcurrencyDemoWorkflow: + @hatchet.concurrency(max_runs=1) + def concurrency(self, context) -> str: + return context.workflow_input()["user_id"] + + @hatchet.step() + def step1(self, context): + print("executed step1") + pass +``` + ## Termination, Sleeps and Threads Hatchet spawns a new thread per step, which means that there is a risk of thread leakage if your code is busy outside of the python interpreter. For example, this can happen if you call `time.sleep` within a step. To avoid this, you can use `context.sleep` instead. For example: diff --git a/internal/config/server/server.go b/internal/config/server/server.go index 714b3674e..273d27a6f 100644 --- a/internal/config/server/server.go +++ b/internal/config/server/server.go @@ -26,7 +26,7 @@ type ServerConfigFile struct { TaskQueue TaskQueueConfigFile `mapstructure:"taskQueue" json:"taskQueue,omitempty"` - Services []string `mapstructure:"services" json:"services,omitempty" default:"[\"ticker\", \"grpc\", \"eventscontroller\", \"jobscontroller\", \"heartbeater\"]"` + Services []string `mapstructure:"services" json:"services,omitempty" default:"[\"ticker\", \"grpc\", \"eventscontroller\", \"jobscontroller\", \"workflowscontroller\", \"heartbeater\"]"` TLS shared.TLSConfigFile `mapstructure:"tls" json:"tls,omitempty"` diff --git a/internal/repository/get_group_key_run.go b/internal/repository/get_group_key_run.go new file mode 100644 index 000000000..f55355596 --- /dev/null +++ b/internal/repository/get_group_key_run.go @@ -0,0 +1,44 @@ +package repository + +import ( + "time" + + "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" +) + +type ListGetGroupKeyRunsOpts struct { + Requeuable *bool + + Status *db.StepRunStatus +} + +type UpdateGetGroupKeyRunOpts struct { + RequeueAfter *time.Time + + ScheduleTimeoutAt *time.Time + + Status *db.StepRunStatus + + StartedAt *time.Time + + FailedAt *time.Time + + FinishedAt *time.Time + + CancelledAt *time.Time + + CancelledReason *string + + Error *string + + Output *string +} + +type GetGroupKeyRunRepository interface { + // ListGetGroupKeyRuns returns a list of get group key runs for a tenant which match the given options. + ListGetGroupKeyRuns(tenantId string, opts *ListGetGroupKeyRunsOpts) ([]db.GetGroupKeyRunModel, error) + + UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId string, opts *UpdateGetGroupKeyRunOpts) (*db.GetGroupKeyRunModel, error) + + GetGroupKeyRunById(tenantId, getGroupKeyRunId string) (*db.GetGroupKeyRunModel, error) +} diff --git a/internal/repository/prisma/dbsqlc/get_group_key_runs.sql b/internal/repository/prisma/dbsqlc/get_group_key_runs.sql new file mode 100644 index 000000000..0fcc58e37 --- /dev/null +++ b/internal/repository/prisma/dbsqlc/get_group_key_runs.sql @@ -0,0 +1,21 @@ +-- name: UpdateGetGroupKeyRun :one +UPDATE + "GetGroupKeyRun" +SET + "requeueAfter" = COALESCE(sqlc.narg('requeueAfter')::timestamp, "requeueAfter"), + "startedAt" = COALESCE(sqlc.narg('startedAt')::timestamp, "startedAt"), + "finishedAt" = COALESCE(sqlc.narg('finishedAt')::timestamp, "finishedAt"), + "status" = CASE + -- Final states are final, cannot be updated + WHEN "status" IN ('SUCCEEDED', 'FAILED', 'CANCELLED') THEN "status" + ELSE COALESCE(sqlc.narg('status'), "status") + END, + "input" = COALESCE(sqlc.narg('input')::jsonb, "input"), + "output" = COALESCE(sqlc.narg('output')::text, "output"), + "error" = COALESCE(sqlc.narg('error')::text, "error"), + "cancelledAt" = COALESCE(sqlc.narg('cancelledAt')::timestamp, "cancelledAt"), + "cancelledReason" = COALESCE(sqlc.narg('cancelledReason')::text, "cancelledReason") +WHERE + "id" = @id::uuid AND + "tenantId" = @tenantId::uuid +RETURNING "GetGroupKeyRun".*; \ No newline at end of file diff --git a/internal/repository/prisma/dbsqlc/get_group_key_runs.sql.go b/internal/repository/prisma/dbsqlc/get_group_key_runs.sql.go new file mode 100644 index 000000000..018825984 --- /dev/null +++ b/internal/repository/prisma/dbsqlc/get_group_key_runs.sql.go @@ -0,0 +1,88 @@ +// Code generated by sqlc. DO NOT EDIT. +// versions: +// sqlc v1.24.0 +// source: get_group_key_runs.sql + +package dbsqlc + +import ( + "context" + + "github.com/jackc/pgx/v5/pgtype" +) + +const updateGetGroupKeyRun = `-- name: UpdateGetGroupKeyRun :one +UPDATE + "GetGroupKeyRun" +SET + "requeueAfter" = COALESCE($1::timestamp, "requeueAfter"), + "startedAt" = COALESCE($2::timestamp, "startedAt"), + "finishedAt" = COALESCE($3::timestamp, "finishedAt"), + "status" = CASE + -- Final states are final, cannot be updated + WHEN "status" IN ('SUCCEEDED', 'FAILED', 'CANCELLED') THEN "status" + ELSE COALESCE($4, "status") + END, + "input" = COALESCE($5::jsonb, "input"), + "output" = COALESCE($6::text, "output"), + "error" = COALESCE($7::text, "error"), + "cancelledAt" = COALESCE($8::timestamp, "cancelledAt"), + "cancelledReason" = COALESCE($9::text, "cancelledReason") +WHERE + "id" = $10::uuid AND + "tenantId" = $11::uuid +RETURNING "GetGroupKeyRun".id, "GetGroupKeyRun"."createdAt", "GetGroupKeyRun"."updatedAt", "GetGroupKeyRun"."deletedAt", "GetGroupKeyRun"."tenantId", "GetGroupKeyRun"."workflowRunId", "GetGroupKeyRun"."workerId", "GetGroupKeyRun"."tickerId", "GetGroupKeyRun".status, "GetGroupKeyRun".input, "GetGroupKeyRun".output, "GetGroupKeyRun"."requeueAfter", "GetGroupKeyRun".error, "GetGroupKeyRun"."startedAt", "GetGroupKeyRun"."finishedAt", "GetGroupKeyRun"."timeoutAt", "GetGroupKeyRun"."cancelledAt", "GetGroupKeyRun"."cancelledReason", "GetGroupKeyRun"."cancelledError" +` + +type UpdateGetGroupKeyRunParams struct { + RequeueAfter pgtype.Timestamp `json:"requeueAfter"` + StartedAt pgtype.Timestamp `json:"startedAt"` + FinishedAt pgtype.Timestamp `json:"finishedAt"` + Status NullStepRunStatus `json:"status"` + Input []byte `json:"input"` + Output pgtype.Text `json:"output"` + Error pgtype.Text `json:"error"` + CancelledAt pgtype.Timestamp `json:"cancelledAt"` + CancelledReason pgtype.Text `json:"cancelledReason"` + ID pgtype.UUID `json:"id"` + Tenantid pgtype.UUID `json:"tenantid"` +} + +func (q *Queries) UpdateGetGroupKeyRun(ctx context.Context, db DBTX, arg UpdateGetGroupKeyRunParams) (*GetGroupKeyRun, error) { + row := db.QueryRow(ctx, updateGetGroupKeyRun, + arg.RequeueAfter, + arg.StartedAt, + arg.FinishedAt, + arg.Status, + arg.Input, + arg.Output, + arg.Error, + arg.CancelledAt, + arg.CancelledReason, + arg.ID, + arg.Tenantid, + ) + var i GetGroupKeyRun + err := row.Scan( + &i.ID, + &i.CreatedAt, + &i.UpdatedAt, + &i.DeletedAt, + &i.TenantId, + &i.WorkflowRunId, + &i.WorkerId, + &i.TickerId, + &i.Status, + &i.Input, + &i.Output, + &i.RequeueAfter, + &i.Error, + &i.StartedAt, + &i.FinishedAt, + &i.TimeoutAt, + &i.CancelledAt, + &i.CancelledReason, + &i.CancelledError, + ) + return &i, err +} diff --git a/internal/repository/prisma/dbsqlc/models.go b/internal/repository/prisma/dbsqlc/models.go index 1166eca49..91aa5c840 100644 --- a/internal/repository/prisma/dbsqlc/models.go +++ b/internal/repository/prisma/dbsqlc/models.go @@ -11,6 +11,49 @@ import ( "github.com/jackc/pgx/v5/pgtype" ) +type ConcurrencyLimitStrategy string + +const ( + ConcurrencyLimitStrategyCANCELINPROGRESS ConcurrencyLimitStrategy = "CANCEL_IN_PROGRESS" + ConcurrencyLimitStrategyDROPNEWEST ConcurrencyLimitStrategy = "DROP_NEWEST" + ConcurrencyLimitStrategyQUEUENEWEST ConcurrencyLimitStrategy = "QUEUE_NEWEST" +) + +func (e *ConcurrencyLimitStrategy) Scan(src interface{}) error { + switch s := src.(type) { + case []byte: + *e = ConcurrencyLimitStrategy(s) + case string: + *e = ConcurrencyLimitStrategy(s) + default: + return fmt.Errorf("unsupported scan type for ConcurrencyLimitStrategy: %T", src) + } + return nil +} + +type NullConcurrencyLimitStrategy struct { + ConcurrencyLimitStrategy ConcurrencyLimitStrategy `json:"ConcurrencyLimitStrategy"` + Valid bool `json:"valid"` // Valid is true if ConcurrencyLimitStrategy is not NULL +} + +// Scan implements the Scanner interface. +func (ns *NullConcurrencyLimitStrategy) Scan(value interface{}) error { + if value == nil { + ns.ConcurrencyLimitStrategy, ns.Valid = "", false + return nil + } + ns.Valid = true + return ns.ConcurrencyLimitStrategy.Scan(value) +} + +// Value implements the driver Valuer interface. +func (ns NullConcurrencyLimitStrategy) Value() (driver.Value, error) { + if !ns.Valid { + return nil, nil + } + return string(ns.ConcurrencyLimitStrategy), nil +} + type InviteLinkStatus string const ( @@ -235,6 +278,7 @@ type WorkflowRunStatus string const ( WorkflowRunStatusPENDING WorkflowRunStatus = "PENDING" + WorkflowRunStatusQUEUED WorkflowRunStatus = "QUEUED" WorkflowRunStatusRUNNING WorkflowRunStatus = "RUNNING" WorkflowRunStatusSUCCEEDED WorkflowRunStatus = "SUCCEEDED" WorkflowRunStatusFAILED WorkflowRunStatus = "FAILED" @@ -317,6 +361,28 @@ type Event struct { Data []byte `json:"data"` } +type GetGroupKeyRun struct { + ID pgtype.UUID `json:"id"` + CreatedAt pgtype.Timestamp `json:"createdAt"` + UpdatedAt pgtype.Timestamp `json:"updatedAt"` + DeletedAt pgtype.Timestamp `json:"deletedAt"` + TenantId pgtype.UUID `json:"tenantId"` + WorkflowRunId string `json:"workflowRunId"` + WorkerId pgtype.UUID `json:"workerId"` + TickerId pgtype.UUID `json:"tickerId"` + Status StepRunStatus `json:"status"` + Input []byte `json:"input"` + Output pgtype.Text `json:"output"` + RequeueAfter pgtype.Timestamp `json:"requeueAfter"` + Error pgtype.Text `json:"error"` + StartedAt pgtype.Timestamp `json:"startedAt"` + FinishedAt pgtype.Timestamp `json:"finishedAt"` + TimeoutAt pgtype.Timestamp `json:"timeoutAt"` + CancelledAt pgtype.Timestamp `json:"cancelledAt"` + CancelledReason pgtype.Text `json:"cancelledReason"` + CancelledError pgtype.Text `json:"cancelledError"` +} + type Job struct { ID pgtype.UUID `json:"id"` CreatedAt pgtype.Timestamp `json:"createdAt"` @@ -516,17 +582,28 @@ type Workflow struct { Description pgtype.Text `json:"description"` } +type WorkflowConcurrency struct { + ID pgtype.UUID `json:"id"` + CreatedAt pgtype.Timestamp `json:"createdAt"` + UpdatedAt pgtype.Timestamp `json:"updatedAt"` + WorkflowVersionId pgtype.UUID `json:"workflowVersionId"` + GetConcurrencyGroupId pgtype.UUID `json:"getConcurrencyGroupId"` + MaxRuns int32 `json:"maxRuns"` + LimitStrategy ConcurrencyLimitStrategy `json:"limitStrategy"` +} + type WorkflowRun struct { - ID string `json:"id"` - CreatedAt pgtype.Timestamp `json:"createdAt"` - UpdatedAt pgtype.Timestamp `json:"updatedAt"` - DeletedAt pgtype.Timestamp `json:"deletedAt"` - TenantId pgtype.UUID `json:"tenantId"` - WorkflowVersionId pgtype.UUID `json:"workflowVersionId"` - Status WorkflowRunStatus `json:"status"` - Error pgtype.Text `json:"error"` - StartedAt pgtype.Timestamp `json:"startedAt"` - FinishedAt pgtype.Timestamp `json:"finishedAt"` + ID string `json:"id"` + CreatedAt pgtype.Timestamp `json:"createdAt"` + UpdatedAt pgtype.Timestamp `json:"updatedAt"` + DeletedAt pgtype.Timestamp `json:"deletedAt"` + TenantId pgtype.UUID `json:"tenantId"` + WorkflowVersionId pgtype.UUID `json:"workflowVersionId"` + ConcurrencyGroupId pgtype.Text `json:"concurrencyGroupId"` + Status WorkflowRunStatus `json:"status"` + Error pgtype.Text `json:"error"` + StartedAt pgtype.Timestamp `json:"startedAt"` + FinishedAt pgtype.Timestamp `json:"finishedAt"` } type WorkflowRunTriggeredBy struct { @@ -560,6 +637,7 @@ type WorkflowTriggerCronRef struct { ParentId pgtype.UUID `json:"parentId"` Cron string `json:"cron"` TickerId pgtype.UUID `json:"tickerId"` + Input []byte `json:"input"` } type WorkflowTriggerEventRef struct { diff --git a/internal/repository/prisma/dbsqlc/schema.sql b/internal/repository/prisma/dbsqlc/schema.sql index 5d136e9de..abedc5d3d 100644 --- a/internal/repository/prisma/dbsqlc/schema.sql +++ b/internal/repository/prisma/dbsqlc/schema.sql @@ -1,3 +1,6 @@ +-- CreateEnum +CREATE TYPE "ConcurrencyLimitStrategy" AS ENUM ('CANCEL_IN_PROGRESS', 'DROP_NEWEST', 'QUEUE_NEWEST'); + -- CreateEnum CREATE TYPE "InviteLinkStatus" AS ENUM ('PENDING', 'ACCEPTED', 'REJECTED'); @@ -14,7 +17,7 @@ CREATE TYPE "TenantMemberRole" AS ENUM ('OWNER', 'ADMIN', 'MEMBER'); CREATE TYPE "WorkerStatus" AS ENUM ('ACTIVE', 'INACTIVE'); -- CreateEnum -CREATE TYPE "WorkflowRunStatus" AS ENUM ('PENDING', 'RUNNING', 'SUCCEEDED', 'FAILED'); +CREATE TYPE "WorkflowRunStatus" AS ENUM ('PENDING', 'QUEUED', 'RUNNING', 'SUCCEEDED', 'FAILED'); -- CreateTable CREATE TABLE "APIToken" ( @@ -65,6 +68,31 @@ CREATE TABLE "Event" ( CONSTRAINT "Event_pkey" PRIMARY KEY ("id") ); +-- CreateTable +CREATE TABLE "GetGroupKeyRun" ( + "id" UUID NOT NULL, + "createdAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "updatedAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "deletedAt" TIMESTAMP(3), + "tenantId" UUID NOT NULL, + "workflowRunId" TEXT NOT NULL, + "workerId" UUID, + "tickerId" UUID, + "status" "StepRunStatus" NOT NULL DEFAULT 'PENDING', + "input" JSONB, + "output" TEXT, + "requeueAfter" TIMESTAMP(3), + "error" TEXT, + "startedAt" TIMESTAMP(3), + "finishedAt" TIMESTAMP(3), + "timeoutAt" TIMESTAMP(3), + "cancelledAt" TIMESTAMP(3), + "cancelledReason" TEXT, + "cancelledError" TEXT, + + CONSTRAINT "GetGroupKeyRun_pkey" PRIMARY KEY ("id") +); + -- CreateTable CREATE TABLE "Job" ( "id" UUID NOT NULL, @@ -295,6 +323,19 @@ CREATE TABLE "Workflow" ( CONSTRAINT "Workflow_pkey" PRIMARY KEY ("id") ); +-- CreateTable +CREATE TABLE "WorkflowConcurrency" ( + "id" UUID NOT NULL, + "createdAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "updatedAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "workflowVersionId" UUID NOT NULL, + "getConcurrencyGroupId" UUID, + "maxRuns" INTEGER NOT NULL DEFAULT 1, + "limitStrategy" "ConcurrencyLimitStrategy" NOT NULL DEFAULT 'CANCEL_IN_PROGRESS', + + CONSTRAINT "WorkflowConcurrency_pkey" PRIMARY KEY ("id") +); + -- CreateTable CREATE TABLE "WorkflowRun" ( "id" TEXT NOT NULL, @@ -303,6 +344,7 @@ CREATE TABLE "WorkflowRun" ( "deletedAt" TIMESTAMP(3), "tenantId" UUID NOT NULL, "workflowVersionId" UUID NOT NULL, + "concurrencyGroupId" TEXT, "status" "WorkflowRunStatus" NOT NULL DEFAULT 'PENDING', "error" TEXT, "startedAt" TIMESTAMP(3), @@ -343,7 +385,8 @@ CREATE TABLE "WorkflowTag" ( CREATE TABLE "WorkflowTriggerCronRef" ( "parentId" UUID NOT NULL, "cron" TEXT NOT NULL, - "tickerId" UUID + "tickerId" UUID, + "input" JSONB ); -- CreateTable @@ -434,6 +477,12 @@ CREATE UNIQUE INDEX "Dispatcher_id_key" ON "Dispatcher"("id" ASC); -- CreateIndex CREATE UNIQUE INDEX "Event_id_key" ON "Event"("id" ASC); +-- CreateIndex +CREATE UNIQUE INDEX "GetGroupKeyRun_id_key" ON "GetGroupKeyRun"("id" ASC); + +-- CreateIndex +CREATE UNIQUE INDEX "GetGroupKeyRun_tenantId_workflowRunId_key" ON "GetGroupKeyRun"("tenantId" ASC, "workflowRunId" ASC); + -- CreateIndex CREATE UNIQUE INDEX "Job_id_key" ON "Job"("id" ASC); @@ -515,6 +564,12 @@ CREATE UNIQUE INDEX "Workflow_id_key" ON "Workflow"("id" ASC); -- CreateIndex CREATE UNIQUE INDEX "Workflow_tenantId_name_key" ON "Workflow"("tenantId" ASC, "name" ASC); +-- CreateIndex +CREATE UNIQUE INDEX "WorkflowConcurrency_id_key" ON "WorkflowConcurrency"("id" ASC); + +-- CreateIndex +CREATE UNIQUE INDEX "WorkflowConcurrency_workflowVersionId_key" ON "WorkflowConcurrency"("workflowVersionId" ASC); + -- CreateIndex CREATE UNIQUE INDEX "WorkflowRun_tenantId_id_key" ON "WorkflowRun"("tenantId" ASC, "id" ASC); @@ -596,6 +651,18 @@ ALTER TABLE "Event" ADD CONSTRAINT "Event_replayedFromId_fkey" FOREIGN KEY ("rep -- AddForeignKey ALTER TABLE "Event" ADD CONSTRAINT "Event_tenantId_fkey" FOREIGN KEY ("tenantId") REFERENCES "Tenant"("id") ON DELETE CASCADE ON UPDATE CASCADE; +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_tenantId_fkey" FOREIGN KEY ("tenantId") REFERENCES "Tenant"("id") ON DELETE CASCADE ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_tenantId_workflowRunId_fkey" FOREIGN KEY ("tenantId", "workflowRunId") REFERENCES "WorkflowRun"("tenantId", "id") ON DELETE CASCADE ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_tickerId_fkey" FOREIGN KEY ("tickerId") REFERENCES "Ticker"("id") ON DELETE SET NULL ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_workerId_fkey" FOREIGN KEY ("workerId") REFERENCES "Worker"("id") ON DELETE SET NULL ON UPDATE CASCADE; + -- AddForeignKey ALTER TABLE "Job" ADD CONSTRAINT "Job_tenantId_fkey" FOREIGN KEY ("tenantId") REFERENCES "Tenant"("id") ON DELETE CASCADE ON UPDATE CASCADE; @@ -674,6 +741,12 @@ ALTER TABLE "Worker" ADD CONSTRAINT "Worker_tenantId_fkey" FOREIGN KEY ("tenantI -- AddForeignKey ALTER TABLE "Workflow" ADD CONSTRAINT "Workflow_tenantId_fkey" FOREIGN KEY ("tenantId") REFERENCES "Tenant"("id") ON DELETE CASCADE ON UPDATE CASCADE; +-- AddForeignKey +ALTER TABLE "WorkflowConcurrency" ADD CONSTRAINT "WorkflowConcurrency_getConcurrencyGroupId_fkey" FOREIGN KEY ("getConcurrencyGroupId") REFERENCES "Action"("id") ON DELETE SET NULL ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "WorkflowConcurrency" ADD CONSTRAINT "WorkflowConcurrency_workflowVersionId_fkey" FOREIGN KEY ("workflowVersionId") REFERENCES "WorkflowVersion"("id") ON DELETE CASCADE ON UPDATE CASCADE; + -- AddForeignKey ALTER TABLE "WorkflowRun" ADD CONSTRAINT "WorkflowRun_tenantId_fkey" FOREIGN KEY ("tenantId") REFERENCES "Tenant"("id") ON DELETE CASCADE ON UPDATE CASCADE; diff --git a/internal/repository/prisma/dbsqlc/sqlc.yaml b/internal/repository/prisma/dbsqlc/sqlc.yaml index a16a1f548..49e8489c8 100644 --- a/internal/repository/prisma/dbsqlc/sqlc.yaml +++ b/internal/repository/prisma/dbsqlc/sqlc.yaml @@ -6,6 +6,7 @@ sql: - workflow_runs.sql - workflows.sql - step_runs.sql + - get_group_key_runs.sql - job_runs.sql - tickers.sql - dispatchers.sql diff --git a/internal/repository/prisma/dbsqlc/workflow_runs.sql b/internal/repository/prisma/dbsqlc/workflow_runs.sql index f32cf25cb..31d62351c 100644 --- a/internal/repository/prisma/dbsqlc/workflow_runs.sql +++ b/internal/repository/prisma/dbsqlc/workflow_runs.sql @@ -13,6 +13,10 @@ LEFT JOIN "Workflow" as workflow ON workflowVersion."workflowId" = workflow."id" WHERE runs."tenantId" = $1 AND + ( + sqlc.narg('workflowVersionId')::uuid IS NULL OR + workflowVersion."id" = sqlc.narg('workflowVersionId')::uuid + ) AND ( sqlc.narg('workflowId')::uuid IS NULL OR workflow."id" = sqlc.narg('workflowId')::uuid @@ -20,6 +24,14 @@ WHERE ( sqlc.narg('eventId')::uuid IS NULL OR events."id" = sqlc.narg('eventId')::uuid + ) AND + ( + sqlc.narg('groupKey')::text IS NULL OR + runs."concurrencyGroupId" = sqlc.narg('groupKey')::text + ) AND + ( + sqlc.narg('status')::"WorkflowRunStatus" IS NULL OR + runs."status" = sqlc.narg('status')::"WorkflowRunStatus" ); -- name: ListWorkflowRuns :many @@ -42,6 +54,10 @@ LEFT JOIN "Workflow" as workflow ON workflowVersion."workflowId" = workflow."id" WHERE runs."tenantId" = $1 AND + ( + sqlc.narg('workflowVersionId')::uuid IS NULL OR + workflowVersion."id" = sqlc.narg('workflowVersionId')::uuid + ) AND ( sqlc.narg('workflowId')::uuid IS NULL OR workflow."id" = sqlc.narg('workflowId')::uuid @@ -49,6 +65,14 @@ WHERE ( sqlc.narg('eventId')::uuid IS NULL OR events."id" = sqlc.narg('eventId')::uuid + ) AND + ( + sqlc.narg('groupKey')::text IS NULL OR + runs."concurrencyGroupId" = sqlc.narg('groupKey')::text + ) AND + ( + sqlc.narg('status')::"WorkflowRunStatus" IS NULL OR + runs."status" = sqlc.narg('status')::"WorkflowRunStatus" ) ORDER BY case when @orderBy = 'createdAt ASC' THEN runs."createdAt" END ASC , @@ -58,6 +82,37 @@ OFFSET LIMIT COALESCE(sqlc.narg('limit'), 50); +-- name: UpdateWorkflowRunGroupKey :one +WITH groupKeyRun AS ( + SELECT "id", "status" as groupKeyRunStatus, "output", "workflowRunId" + FROM "GetGroupKeyRun" as groupKeyRun + WHERE + "id" = @groupKeyRunId::uuid AND + "tenantId" = @tenantId::uuid +) +UPDATE "WorkflowRun" workflowRun +SET "status" = CASE + -- Final states are final, cannot be updated. We also can't move out of a queued state + WHEN "status" IN ('SUCCEEDED', 'FAILED', 'QUEUED') THEN "status" + -- When the GetGroupKeyRun failed or been cancelled, then the workflow is failed + WHEN groupKeyRun.groupKeyRunStatus IN ('FAILED', 'CANCELLED') THEN 'FAILED' + WHEN groupKeyRun.output IS NOT NULL THEN 'QUEUED' + ELSE "status" +END, "finishedAt" = CASE + -- Final states are final, cannot be updated + WHEN "finishedAt" IS NOT NULL THEN "finishedAt" + -- When one job run has failed or been cancelled, then the workflow is failed + WHEN groupKeyRun.groupKeyRunStatus IN ('FAILED', 'CANCELLED') THEN NOW() + ELSE "finishedAt" +END, +"concurrencyGroupId" = groupKeyRun."output" +FROM + groupKeyRun +WHERE +workflowRun."id" = groupKeyRun."workflowRunId" AND +workflowRun."tenantId" = @tenantId::uuid +RETURNING workflowRun.*; + -- name: ResolveWorkflowRunStatus :one WITH jobRuns AS ( SELECT sum(case when runs."status" = 'PENDING' then 1 else 0 end) AS pendingRuns, @@ -159,6 +214,49 @@ INSERT INTO "WorkflowRunTriggeredBy" ( sqlc.narg('scheduledId')::uuid -- NULL if not provided ) RETURNING *; +-- name: CreateGetGroupKeyRun :one +INSERT INTO "GetGroupKeyRun" ( + "id", + "createdAt", + "updatedAt", + "deletedAt", + "tenantId", + "workflowRunId", + "workerId", + "tickerId", + "status", + "input", + "output", + "requeueAfter", + "error", + "startedAt", + "finishedAt", + "timeoutAt", + "cancelledAt", + "cancelledReason", + "cancelledError" +) VALUES ( + COALESCE(sqlc.narg('id')::uuid, gen_random_uuid()), + CURRENT_TIMESTAMP, + CURRENT_TIMESTAMP, + NULL, + @tenantId::uuid, + @workflowRunId::text, + NULL, + NULL, + 'PENDING', -- default status + @input::jsonb, + NULL, + @requeueAfter::timestamp, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL +) RETURNING *; + -- name: CreateJobRun :one INSERT INTO "JobRun" ( "id", diff --git a/internal/repository/prisma/dbsqlc/workflow_runs.sql.go b/internal/repository/prisma/dbsqlc/workflow_runs.sql.go index c2709607e..0eb16a6b7 100644 --- a/internal/repository/prisma/dbsqlc/workflow_runs.sql.go +++ b/internal/repository/prisma/dbsqlc/workflow_runs.sql.go @@ -28,27 +28,134 @@ WHERE runs."tenantId" = $1 AND ( $2::uuid IS NULL OR - workflow."id" = $2::uuid + workflowVersion."id" = $2::uuid ) AND ( $3::uuid IS NULL OR - events."id" = $3::uuid + workflow."id" = $3::uuid + ) AND + ( + $4::uuid IS NULL OR + events."id" = $4::uuid + ) AND + ( + $5::text IS NULL OR + runs."concurrencyGroupId" = $5::text + ) AND + ( + $6::"WorkflowRunStatus" IS NULL OR + runs."status" = $6::"WorkflowRunStatus" ) ` type CountWorkflowRunsParams struct { - TenantId pgtype.UUID `json:"tenantId"` - WorkflowId pgtype.UUID `json:"workflowId"` - EventId pgtype.UUID `json:"eventId"` + TenantId pgtype.UUID `json:"tenantId"` + WorkflowVersionId pgtype.UUID `json:"workflowVersionId"` + WorkflowId pgtype.UUID `json:"workflowId"` + EventId pgtype.UUID `json:"eventId"` + GroupKey pgtype.Text `json:"groupKey"` + Status NullWorkflowRunStatus `json:"status"` } func (q *Queries) CountWorkflowRuns(ctx context.Context, db DBTX, arg CountWorkflowRunsParams) (int64, error) { - row := db.QueryRow(ctx, countWorkflowRuns, arg.TenantId, arg.WorkflowId, arg.EventId) + row := db.QueryRow(ctx, countWorkflowRuns, + arg.TenantId, + arg.WorkflowVersionId, + arg.WorkflowId, + arg.EventId, + arg.GroupKey, + arg.Status, + ) var total int64 err := row.Scan(&total) return total, err } +const createGetGroupKeyRun = `-- name: CreateGetGroupKeyRun :one +INSERT INTO "GetGroupKeyRun" ( + "id", + "createdAt", + "updatedAt", + "deletedAt", + "tenantId", + "workflowRunId", + "workerId", + "tickerId", + "status", + "input", + "output", + "requeueAfter", + "error", + "startedAt", + "finishedAt", + "timeoutAt", + "cancelledAt", + "cancelledReason", + "cancelledError" +) VALUES ( + COALESCE($1::uuid, gen_random_uuid()), + CURRENT_TIMESTAMP, + CURRENT_TIMESTAMP, + NULL, + $2::uuid, + $3::text, + NULL, + NULL, + 'PENDING', -- default status + $4::jsonb, + NULL, + $5::timestamp, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL +) RETURNING id, "createdAt", "updatedAt", "deletedAt", "tenantId", "workflowRunId", "workerId", "tickerId", status, input, output, "requeueAfter", error, "startedAt", "finishedAt", "timeoutAt", "cancelledAt", "cancelledReason", "cancelledError" +` + +type CreateGetGroupKeyRunParams struct { + ID pgtype.UUID `json:"id"` + Tenantid pgtype.UUID `json:"tenantid"` + Workflowrunid string `json:"workflowrunid"` + Input []byte `json:"input"` + Requeueafter pgtype.Timestamp `json:"requeueafter"` +} + +func (q *Queries) CreateGetGroupKeyRun(ctx context.Context, db DBTX, arg CreateGetGroupKeyRunParams) (*GetGroupKeyRun, error) { + row := db.QueryRow(ctx, createGetGroupKeyRun, + arg.ID, + arg.Tenantid, + arg.Workflowrunid, + arg.Input, + arg.Requeueafter, + ) + var i GetGroupKeyRun + err := row.Scan( + &i.ID, + &i.CreatedAt, + &i.UpdatedAt, + &i.DeletedAt, + &i.TenantId, + &i.WorkflowRunId, + &i.WorkerId, + &i.TickerId, + &i.Status, + &i.Input, + &i.Output, + &i.RequeueAfter, + &i.Error, + &i.StartedAt, + &i.FinishedAt, + &i.TimeoutAt, + &i.CancelledAt, + &i.CancelledReason, + &i.CancelledError, + ) + return &i, err +} + const createJobRun = `-- name: CreateJobRun :one INSERT INTO "JobRun" ( "id", @@ -293,7 +400,7 @@ INSERT INTO "WorkflowRun" ( NULL, -- assuming error is not set on creation NULL, -- assuming startedAt is not set on creation NULL -- assuming finishedAt is not set on creation -) RETURNING id, "createdAt", "updatedAt", "deletedAt", "tenantId", "workflowVersionId", status, error, "startedAt", "finishedAt" +) RETURNING id, "createdAt", "updatedAt", "deletedAt", "tenantId", "workflowVersionId", "concurrencyGroupId", status, error, "startedAt", "finishedAt" ` type CreateWorkflowRunParams struct { @@ -312,6 +419,7 @@ func (q *Queries) CreateWorkflowRun(ctx context.Context, db DBTX, arg CreateWork &i.DeletedAt, &i.TenantId, &i.WorkflowVersionId, + &i.ConcurrencyGroupId, &i.Status, &i.Error, &i.StartedAt, @@ -471,7 +579,7 @@ func (q *Queries) ListStartableStepRuns(ctx context.Context, db DBTX, arg ListSt const listWorkflowRuns = `-- name: ListWorkflowRuns :many SELECT - runs.id, runs."createdAt", runs."updatedAt", runs."deletedAt", runs."tenantId", runs."workflowVersionId", runs.status, runs.error, runs."startedAt", runs."finishedAt", + runs.id, runs."createdAt", runs."updatedAt", runs."deletedAt", runs."tenantId", runs."workflowVersionId", runs."concurrencyGroupId", runs.status, runs.error, runs."startedAt", runs."finishedAt", workflow.id, workflow."createdAt", workflow."updatedAt", workflow."deletedAt", workflow."tenantId", workflow.name, workflow.description, runtriggers.id, runtriggers."createdAt", runtriggers."updatedAt", runtriggers."deletedAt", runtriggers."tenantId", runtriggers."parentId", runtriggers."eventId", runtriggers."cronParentId", runtriggers."cronSchedule", runtriggers."scheduledId", workflowversion.id, workflowversion."createdAt", workflowversion."updatedAt", workflowversion."deletedAt", workflowversion.checksum, workflowversion.version, workflowversion."order", workflowversion."workflowId", @@ -491,28 +599,43 @@ WHERE runs."tenantId" = $1 AND ( $2::uuid IS NULL OR - workflow."id" = $2::uuid + workflowVersion."id" = $2::uuid ) AND ( $3::uuid IS NULL OR - events."id" = $3::uuid + workflow."id" = $3::uuid + ) AND + ( + $4::uuid IS NULL OR + events."id" = $4::uuid + ) AND + ( + $5::text IS NULL OR + runs."concurrencyGroupId" = $5::text + ) AND + ( + $6::"WorkflowRunStatus" IS NULL OR + runs."status" = $6::"WorkflowRunStatus" ) ORDER BY - case when $4 = 'createdAt ASC' THEN runs."createdAt" END ASC , - case when $4 = 'createdAt DESC' then runs."createdAt" END DESC + case when $7 = 'createdAt ASC' THEN runs."createdAt" END ASC , + case when $7 = 'createdAt DESC' then runs."createdAt" END DESC OFFSET - COALESCE($5, 0) + COALESCE($8, 0) LIMIT - COALESCE($6, 50) + COALESCE($9, 50) ` type ListWorkflowRunsParams struct { - TenantId pgtype.UUID `json:"tenantId"` - WorkflowId pgtype.UUID `json:"workflowId"` - EventId pgtype.UUID `json:"eventId"` - 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"` + EventId pgtype.UUID `json:"eventId"` + GroupKey pgtype.Text `json:"groupKey"` + Status NullWorkflowRunStatus `json:"status"` + Orderby interface{} `json:"orderby"` + Offset interface{} `json:"offset"` + Limit interface{} `json:"limit"` } type ListWorkflowRunsRow struct { @@ -529,8 +652,11 @@ type ListWorkflowRunsRow struct { func (q *Queries) ListWorkflowRuns(ctx context.Context, db DBTX, arg ListWorkflowRunsParams) ([]*ListWorkflowRunsRow, error) { rows, err := db.Query(ctx, listWorkflowRuns, arg.TenantId, + arg.WorkflowVersionId, arg.WorkflowId, arg.EventId, + arg.GroupKey, + arg.Status, arg.Orderby, arg.Offset, arg.Limit, @@ -549,6 +675,7 @@ func (q *Queries) ListWorkflowRuns(ctx context.Context, db DBTX, arg ListWorkflo &i.WorkflowRun.DeletedAt, &i.WorkflowRun.TenantId, &i.WorkflowRun.WorkflowVersionId, + &i.WorkflowRun.ConcurrencyGroupId, &i.WorkflowRun.Status, &i.WorkflowRun.Error, &i.WorkflowRun.StartedAt, @@ -642,7 +769,7 @@ WHERE "id" = ( FROM "JobRun" WHERE "id" = $1::uuid ) AND "tenantId" = $2::uuid -RETURNING "WorkflowRun".id, "WorkflowRun"."createdAt", "WorkflowRun"."updatedAt", "WorkflowRun"."deletedAt", "WorkflowRun"."tenantId", "WorkflowRun"."workflowVersionId", "WorkflowRun".status, "WorkflowRun".error, "WorkflowRun"."startedAt", "WorkflowRun"."finishedAt" +RETURNING "WorkflowRun".id, "WorkflowRun"."createdAt", "WorkflowRun"."updatedAt", "WorkflowRun"."deletedAt", "WorkflowRun"."tenantId", "WorkflowRun"."workflowVersionId", "WorkflowRun"."concurrencyGroupId", "WorkflowRun".status, "WorkflowRun".error, "WorkflowRun"."startedAt", "WorkflowRun"."finishedAt" ` type ResolveWorkflowRunStatusParams struct { @@ -660,6 +787,63 @@ func (q *Queries) ResolveWorkflowRunStatus(ctx context.Context, db DBTX, arg Res &i.DeletedAt, &i.TenantId, &i.WorkflowVersionId, + &i.ConcurrencyGroupId, + &i.Status, + &i.Error, + &i.StartedAt, + &i.FinishedAt, + ) + return &i, err +} + +const updateWorkflowRunGroupKey = `-- name: UpdateWorkflowRunGroupKey :one +WITH groupKeyRun AS ( + SELECT "id", "status" as groupKeyRunStatus, "output", "workflowRunId" + FROM "GetGroupKeyRun" as groupKeyRun + WHERE + "id" = $2::uuid AND + "tenantId" = $1::uuid +) +UPDATE "WorkflowRun" workflowRun +SET "status" = CASE + -- Final states are final, cannot be updated. We also can't move out of a queued state + WHEN "status" IN ('SUCCEEDED', 'FAILED', 'QUEUED') THEN "status" + -- When the GetGroupKeyRun failed or been cancelled, then the workflow is failed + WHEN groupKeyRun.groupKeyRunStatus IN ('FAILED', 'CANCELLED') THEN 'FAILED' + WHEN groupKeyRun.output IS NOT NULL THEN 'QUEUED' + ELSE "status" +END, "finishedAt" = CASE + -- Final states are final, cannot be updated + WHEN "finishedAt" IS NOT NULL THEN "finishedAt" + -- When one job run has failed or been cancelled, then the workflow is failed + WHEN groupKeyRun.groupKeyRunStatus IN ('FAILED', 'CANCELLED') THEN NOW() + ELSE "finishedAt" +END, +"concurrencyGroupId" = groupKeyRun."output" +FROM + groupKeyRun +WHERE +workflowRun."id" = groupKeyRun."workflowRunId" AND +workflowRun."tenantId" = $1::uuid +RETURNING workflowrun.id, workflowrun."createdAt", workflowrun."updatedAt", workflowrun."deletedAt", workflowrun."tenantId", workflowrun."workflowVersionId", workflowrun."concurrencyGroupId", workflowrun.status, workflowrun.error, workflowrun."startedAt", workflowrun."finishedAt" +` + +type UpdateWorkflowRunGroupKeyParams struct { + Tenantid pgtype.UUID `json:"tenantid"` + Groupkeyrunid pgtype.UUID `json:"groupkeyrunid"` +} + +func (q *Queries) UpdateWorkflowRunGroupKey(ctx context.Context, db DBTX, arg UpdateWorkflowRunGroupKeyParams) (*WorkflowRun, error) { + row := db.QueryRow(ctx, updateWorkflowRunGroupKey, arg.Tenantid, arg.Groupkeyrunid) + var i WorkflowRun + err := row.Scan( + &i.ID, + &i.CreatedAt, + &i.UpdatedAt, + &i.DeletedAt, + &i.TenantId, + &i.WorkflowVersionId, + &i.ConcurrencyGroupId, &i.Status, &i.Error, &i.StartedAt, diff --git a/internal/repository/prisma/dbsqlc/workflows.sql b/internal/repository/prisma/dbsqlc/workflows.sql index 8bebd1623..f281c9c44 100644 --- a/internal/repository/prisma/dbsqlc/workflows.sql +++ b/internal/repository/prisma/dbsqlc/workflows.sql @@ -165,6 +165,25 @@ INSERT INTO "WorkflowVersion" ( @workflowId::uuid ) RETURNING *; +-- name: CreateWorkflowConcurrency :one +INSERT INTO "WorkflowConcurrency" ( + "id", + "createdAt", + "updatedAt", + "workflowVersionId", + "getConcurrencyGroupId", + "maxRuns", + "limitStrategy" +) VALUES ( + @id::uuid, + coalesce(sqlc.narg('createdAt')::timestamp, CURRENT_TIMESTAMP), + coalesce(sqlc.narg('updatedAt')::timestamp, CURRENT_TIMESTAMP), + @workflowVersionId::uuid, + @getConcurrencyGroupId::uuid, + coalesce(sqlc.narg('maxRuns')::integer, 1), + coalesce(sqlc.narg('limitStrategy')::"ConcurrencyLimitStrategy", 'CANCEL_IN_PROGRESS') +) RETURNING *; + -- name: CreateJob :one INSERT INTO "Job" ( "id", @@ -221,7 +240,7 @@ FROM JOIN "Step" AS step ON step."readableId" = parent_readable_id AND step."jobId" = @jobId::uuid; --- name: UpsertAction :exec +-- name: UpsertAction :one INSERT INTO "Action" ( "id", "actionId", @@ -229,14 +248,15 @@ INSERT INTO "Action" ( ) VALUES ( gen_random_uuid(), - @action::text, + LOWER(@action::text), @tenantId::uuid ) ON CONFLICT ("tenantId", "actionId") DO UPDATE SET "tenantId" = EXCLUDED."tenantId" WHERE - "Action"."tenantId" = @tenantId AND "Action"."actionId" = @action::text; + "Action"."tenantId" = @tenantId AND "Action"."actionId" = LOWER(@action::text) +RETURNING *; -- name: UpsertWorkflowTag :exec INSERT INTO "WorkflowTag" ( diff --git a/internal/repository/prisma/dbsqlc/workflows.sql.go b/internal/repository/prisma/dbsqlc/workflows.sql.go index 15c654b6a..e817aa69c 100644 --- a/internal/repository/prisma/dbsqlc/workflows.sql.go +++ b/internal/repository/prisma/dbsqlc/workflows.sql.go @@ -275,6 +275,59 @@ func (q *Queries) CreateWorkflow(ctx context.Context, db DBTX, arg CreateWorkflo return &i, err } +const createWorkflowConcurrency = `-- name: CreateWorkflowConcurrency :one +INSERT INTO "WorkflowConcurrency" ( + "id", + "createdAt", + "updatedAt", + "workflowVersionId", + "getConcurrencyGroupId", + "maxRuns", + "limitStrategy" +) VALUES ( + $1::uuid, + coalesce($2::timestamp, CURRENT_TIMESTAMP), + coalesce($3::timestamp, CURRENT_TIMESTAMP), + $4::uuid, + $5::uuid, + coalesce($6::integer, 1), + coalesce($7::"ConcurrencyLimitStrategy", 'CANCEL_IN_PROGRESS') +) RETURNING id, "createdAt", "updatedAt", "workflowVersionId", "getConcurrencyGroupId", "maxRuns", "limitStrategy" +` + +type CreateWorkflowConcurrencyParams struct { + ID pgtype.UUID `json:"id"` + CreatedAt pgtype.Timestamp `json:"createdAt"` + UpdatedAt pgtype.Timestamp `json:"updatedAt"` + Workflowversionid pgtype.UUID `json:"workflowversionid"` + Getconcurrencygroupid pgtype.UUID `json:"getconcurrencygroupid"` + MaxRuns pgtype.Int4 `json:"maxRuns"` + LimitStrategy NullConcurrencyLimitStrategy `json:"limitStrategy"` +} + +func (q *Queries) CreateWorkflowConcurrency(ctx context.Context, db DBTX, arg CreateWorkflowConcurrencyParams) (*WorkflowConcurrency, error) { + row := db.QueryRow(ctx, createWorkflowConcurrency, + arg.ID, + arg.CreatedAt, + arg.UpdatedAt, + arg.Workflowversionid, + arg.Getconcurrencygroupid, + arg.MaxRuns, + arg.LimitStrategy, + ) + var i WorkflowConcurrency + err := row.Scan( + &i.ID, + &i.CreatedAt, + &i.UpdatedAt, + &i.WorkflowVersionId, + &i.GetConcurrencyGroupId, + &i.MaxRuns, + &i.LimitStrategy, + ) + return &i, err +} + const createWorkflowTriggerCronRef = `-- name: CreateWorkflowTriggerCronRef :one INSERT INTO "WorkflowTriggerCronRef" ( "parentId", @@ -282,7 +335,7 @@ INSERT INTO "WorkflowTriggerCronRef" ( ) VALUES ( $1::uuid, $2::text -) RETURNING "parentId", cron, "tickerId" +) RETURNING "parentId", cron, "tickerId", input ` type CreateWorkflowTriggerCronRefParams struct { @@ -293,7 +346,12 @@ type CreateWorkflowTriggerCronRefParams struct { func (q *Queries) CreateWorkflowTriggerCronRef(ctx context.Context, db DBTX, arg CreateWorkflowTriggerCronRefParams) (*WorkflowTriggerCronRef, error) { row := db.QueryRow(ctx, createWorkflowTriggerCronRef, arg.Workflowtriggersid, arg.Crontrigger) var i WorkflowTriggerCronRef - err := row.Scan(&i.ParentId, &i.Cron, &i.TickerId) + err := row.Scan( + &i.ParentId, + &i.Cron, + &i.TickerId, + &i.Input, + ) return &i, err } @@ -549,7 +607,7 @@ func (q *Queries) ListWorkflows(ctx context.Context, db DBTX, arg ListWorkflowsP const listWorkflowsLatestRuns = `-- name: ListWorkflowsLatestRuns :many SELECT - DISTINCT ON (workflow."id") runs.id, runs."createdAt", runs."updatedAt", runs."deletedAt", runs."tenantId", runs."workflowVersionId", runs.status, runs.error, runs."startedAt", runs."finishedAt", workflow."id" as "workflowId" + DISTINCT ON (workflow."id") runs.id, runs."createdAt", runs."updatedAt", runs."deletedAt", runs."tenantId", runs."workflowVersionId", runs."concurrencyGroupId", runs.status, runs.error, runs."startedAt", runs."finishedAt", workflow."id" as "workflowId" FROM "WorkflowRun" as runs LEFT JOIN @@ -614,6 +672,7 @@ func (q *Queries) ListWorkflowsLatestRuns(ctx context.Context, db DBTX, arg List &i.WorkflowRun.DeletedAt, &i.WorkflowRun.TenantId, &i.WorkflowRun.WorkflowVersionId, + &i.WorkflowRun.ConcurrencyGroupId, &i.WorkflowRun.Status, &i.WorkflowRun.Error, &i.WorkflowRun.StartedAt, @@ -630,7 +689,7 @@ func (q *Queries) ListWorkflowsLatestRuns(ctx context.Context, db DBTX, arg List return items, nil } -const upsertAction = `-- name: UpsertAction :exec +const upsertAction = `-- name: UpsertAction :one INSERT INTO "Action" ( "id", "actionId", @@ -638,14 +697,15 @@ INSERT INTO "Action" ( ) VALUES ( gen_random_uuid(), - $1::text, + LOWER($1::text), $2::uuid ) ON CONFLICT ("tenantId", "actionId") DO UPDATE SET "tenantId" = EXCLUDED."tenantId" WHERE - "Action"."tenantId" = $2 AND "Action"."actionId" = $1::text + "Action"."tenantId" = $2 AND "Action"."actionId" = LOWER($1::text) +RETURNING id, "actionId", description, "tenantId" ` type UpsertActionParams struct { @@ -653,9 +713,16 @@ type UpsertActionParams struct { Tenantid pgtype.UUID `json:"tenantid"` } -func (q *Queries) UpsertAction(ctx context.Context, db DBTX, arg UpsertActionParams) error { - _, err := db.Exec(ctx, upsertAction, arg.Action, arg.Tenantid) - return err +func (q *Queries) UpsertAction(ctx context.Context, db DBTX, arg UpsertActionParams) (*Action, error) { + row := db.QueryRow(ctx, upsertAction, arg.Action, arg.Tenantid) + var i Action + err := row.Scan( + &i.ID, + &i.ActionId, + &i.Description, + &i.TenantId, + ) + return &i, err } const upsertWorkflowTag = `-- name: UpsertWorkflowTag :exec diff --git a/internal/repository/prisma/get_group_key_run.go b/internal/repository/prisma/get_group_key_run.go new file mode 100644 index 000000000..dbd771678 --- /dev/null +++ b/internal/repository/prisma/get_group_key_run.go @@ -0,0 +1,188 @@ +package prisma + +import ( + "context" + "fmt" + "time" + + "github.com/jackc/pgx/v5/pgxpool" + "github.com/rs/zerolog" + + "github.com/hatchet-dev/hatchet/internal/repository" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/sqlchelpers" + "github.com/hatchet-dev/hatchet/internal/validator" +) + +type getGroupKeyRunRepository struct { + client *db.PrismaClient + pool *pgxpool.Pool + v validator.Validator + l *zerolog.Logger + queries *dbsqlc.Queries +} + +func NewGetGroupKeyRunRepository(client *db.PrismaClient, pool *pgxpool.Pool, v validator.Validator, l *zerolog.Logger) repository.GetGroupKeyRunRepository { + queries := dbsqlc.New() + + return &getGroupKeyRunRepository{ + client: client, + pool: pool, + v: v, + l: l, + queries: queries, + } +} + +func (s *getGroupKeyRunRepository) ListGetGroupKeyRuns(tenantId string, opts *repository.ListGetGroupKeyRunsOpts) ([]db.GetGroupKeyRunModel, error) { + if err := s.v.Validate(opts); err != nil { + return nil, err + } + + params := []db.GetGroupKeyRunWhereParam{ + db.GetGroupKeyRun.TenantID.Equals(tenantId), + } + + if opts.Requeuable != nil { + // job runs are requeuable if they are past their requeue after time, don't have a worker assigned, have a pending status, + // and their previous step is completed + params = append( + params, + db.GetGroupKeyRun.RequeueAfter.Before(time.Now().UTC()), + db.GetGroupKeyRun.WorkerID.IsNull(), + db.GetGroupKeyRun.Status.Equals(db.StepRunStatusPendingAssignment), + ) + } + + if opts.Status != nil { + params = append(params, db.GetGroupKeyRun.Status.Equals(*opts.Status)) + } + + return s.client.GetGroupKeyRun.FindMany( + params..., + ).With( + db.GetGroupKeyRun.Ticker.Fetch(), + db.GetGroupKeyRun.WorkflowRun.Fetch().With( + db.WorkflowRun.WorkflowVersion.Fetch().With( + db.WorkflowVersion.Concurrency.Fetch().With( + db.WorkflowConcurrency.GetConcurrencyGroup.Fetch(), + ), + ), + ), + ).Exec(context.Background()) +} + +func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId string, opts *repository.UpdateGetGroupKeyRunOpts) (*db.GetGroupKeyRunModel, error) { + if err := s.v.Validate(opts); err != nil { + return nil, err + } + + pgTenantId := sqlchelpers.UUIDFromStr(tenantId) + pgGetGroupKeyRunId := sqlchelpers.UUIDFromStr(getGroupKeyRunId) + + updateParams := dbsqlc.UpdateGetGroupKeyRunParams{ + ID: pgGetGroupKeyRunId, + Tenantid: pgTenantId, + } + + updateWorkflowRunParams := dbsqlc.UpdateWorkflowRunGroupKeyParams{ + Tenantid: pgTenantId, + Groupkeyrunid: sqlchelpers.UUIDFromStr(getGroupKeyRunId), + } + + if opts.RequeueAfter != nil { + updateParams.RequeueAfter = sqlchelpers.TimestampFromTime(*opts.RequeueAfter) + } + + if opts.StartedAt != nil { + updateParams.StartedAt = sqlchelpers.TimestampFromTime(*opts.StartedAt) + } + + if opts.FinishedAt != nil { + updateParams.FinishedAt = sqlchelpers.TimestampFromTime(*opts.FinishedAt) + } + + if opts.Status != nil { + runStatus := dbsqlc.NullStepRunStatus{} + + if err := runStatus.Scan(string(*opts.Status)); err != nil { + return nil, err + } + + updateParams.Status = runStatus + } + + if opts.Output != nil { + updateParams.Output = sqlchelpers.TextFromStr(*opts.Output) + } + + if opts.Error != nil { + updateParams.Error = sqlchelpers.TextFromStr(*opts.Error) + } + + if opts.CancelledAt != nil { + updateParams.CancelledAt = sqlchelpers.TimestampFromTime(*opts.CancelledAt) + } + + if opts.CancelledReason != nil { + updateParams.CancelledReason = sqlchelpers.TextFromStr(*opts.CancelledReason) + } + + tx, err := s.pool.Begin(context.Background()) + + if err != nil { + return nil, err + } + + defer deferRollback(context.Background(), s.l, tx.Rollback) + + _, err = s.queries.UpdateGetGroupKeyRun(context.Background(), tx, updateParams) + + if err != nil { + return nil, fmt.Errorf("could not update get group key run: %w", err) + } + + // 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) + + if err != nil { + return nil, fmt.Errorf("could not resolve workflow run status from get group key run: %w", err) + } + } + + err = tx.Commit(context.Background()) + + if err != nil { + return nil, err + } + + return s.client.GetGroupKeyRun.FindUnique( + db.GetGroupKeyRun.ID.Equals(getGroupKeyRunId), + ).With( + db.GetGroupKeyRun.Ticker.Fetch(), + db.GetGroupKeyRun.WorkflowRun.Fetch().With( + db.WorkflowRun.WorkflowVersion.Fetch().With( + db.WorkflowVersion.Concurrency.Fetch().With( + db.WorkflowConcurrency.GetConcurrencyGroup.Fetch(), + ), + ), + ), + ).Exec(context.Background()) +} + +func (s *getGroupKeyRunRepository) GetGroupKeyRunById(tenantId, getGroupKeyRunId string) (*db.GetGroupKeyRunModel, error) { + return s.client.GetGroupKeyRun.FindUnique( + db.GetGroupKeyRun.ID.Equals(getGroupKeyRunId), + ).With( + db.GetGroupKeyRun.Ticker.Fetch(), + db.GetGroupKeyRun.WorkflowRun.Fetch().With( + db.WorkflowRun.WorkflowVersion.Fetch().With( + db.WorkflowVersion.Concurrency.Fetch().With( + db.WorkflowConcurrency.GetConcurrencyGroup.Fetch(), + ), + ), + ), + ).Exec(context.Background()) +} diff --git a/internal/repository/prisma/repository.go b/internal/repository/prisma/repository.go index 99ea6b195..6359c87d2 100644 --- a/internal/repository/prisma/repository.go +++ b/internal/repository/prisma/repository.go @@ -10,20 +10,21 @@ import ( ) type prismaRepository struct { - apiToken repository.APITokenRepository - event repository.EventRepository - tenant repository.TenantRepository - tenantInvite repository.TenantInviteRepository - workflow repository.WorkflowRepository - workflowRun repository.WorkflowRunRepository - jobRun repository.JobRunRepository - stepRun repository.StepRunRepository - step repository.StepRepository - dispatcher repository.DispatcherRepository - worker repository.WorkerRepository - ticker repository.TickerRepository - userSession repository.UserSessionRepository - user repository.UserRepository + apiToken repository.APITokenRepository + event repository.EventRepository + tenant repository.TenantRepository + tenantInvite repository.TenantInviteRepository + workflow repository.WorkflowRepository + workflowRun repository.WorkflowRunRepository + jobRun repository.JobRunRepository + stepRun repository.StepRunRepository + getGroupKeyRun repository.GetGroupKeyRunRepository + step repository.StepRepository + dispatcher repository.DispatcherRepository + worker repository.WorkerRepository + ticker repository.TickerRepository + userSession repository.UserSessionRepository + user repository.UserRepository } type PrismaRepositoryOpt func(*PrismaRepositoryOpts) @@ -62,20 +63,21 @@ func NewPrismaRepository(client *db.PrismaClient, pool *pgxpool.Pool, fs ...Pris opts.l = &newLogger return &prismaRepository{ - apiToken: NewAPITokenRepository(client, opts.v), - event: NewEventRepository(client, pool, opts.v, opts.l), - tenant: NewTenantRepository(client, opts.v), - tenantInvite: NewTenantInviteRepository(client, opts.v), - workflow: NewWorkflowRepository(client, pool, opts.v, opts.l), - workflowRun: NewWorkflowRunRepository(client, pool, opts.v, opts.l), - jobRun: NewJobRunRepository(client, pool, opts.v, opts.l), - stepRun: NewStepRunRepository(client, pool, opts.v, opts.l), - step: NewStepRepository(client, opts.v), - dispatcher: NewDispatcherRepository(client, pool, opts.v, opts.l), - worker: NewWorkerRepository(client, opts.v), - ticker: NewTickerRepository(client, pool, opts.v, opts.l), - userSession: NewUserSessionRepository(client, opts.v), - user: NewUserRepository(client, opts.v), + apiToken: NewAPITokenRepository(client, opts.v), + event: NewEventRepository(client, pool, opts.v, opts.l), + tenant: NewTenantRepository(client, opts.v), + tenantInvite: NewTenantInviteRepository(client, opts.v), + workflow: NewWorkflowRepository(client, pool, opts.v, opts.l), + workflowRun: NewWorkflowRunRepository(client, pool, opts.v, opts.l), + jobRun: NewJobRunRepository(client, pool, opts.v, opts.l), + stepRun: NewStepRunRepository(client, pool, opts.v, opts.l), + getGroupKeyRun: NewGetGroupKeyRunRepository(client, pool, opts.v, opts.l), + step: NewStepRepository(client, opts.v), + dispatcher: NewDispatcherRepository(client, pool, opts.v, opts.l), + worker: NewWorkerRepository(client, opts.v), + ticker: NewTickerRepository(client, pool, opts.v, opts.l), + userSession: NewUserSessionRepository(client, opts.v), + user: NewUserRepository(client, opts.v), } } @@ -111,6 +113,10 @@ func (r *prismaRepository) StepRun() repository.StepRunRepository { return r.stepRun } +func (r *prismaRepository) GetGroupKeyRun() repository.GetGroupKeyRunRepository { + return r.getGroupKeyRun +} + func (r *prismaRepository) Step() repository.StepRepository { return r.step } diff --git a/internal/repository/prisma/sqlchelpers/int.go b/internal/repository/prisma/sqlchelpers/int.go new file mode 100644 index 000000000..50cd8e3fb --- /dev/null +++ b/internal/repository/prisma/sqlchelpers/int.go @@ -0,0 +1,10 @@ +package sqlchelpers + +import "github.com/jackc/pgx/v5/pgtype" + +func ToInt(i int32) pgtype.Int4 { + return pgtype.Int4{ + Valid: true, + Int32: i, + } +} diff --git a/internal/repository/prisma/step_run.go b/internal/repository/prisma/step_run.go index f8974b2ce..de8f5072c 100644 --- a/internal/repository/prisma/step_run.go +++ b/internal/repository/prisma/step_run.go @@ -109,6 +109,12 @@ func (s *stepRunRepository) ListStepRuns(tenantId string, opts *repository.ListS params = append(params, db.StepRun.JobRunID.Equals(*opts.JobRunId)) } + if opts.WorkflowRunId != nil { + params = append(params, db.StepRun.JobRun.Where( + db.JobRun.WorkflowRunID.Equals(*opts.WorkflowRunId), + )) + } + return s.client.StepRun.FindMany( params..., ).With( diff --git a/internal/repository/prisma/ticker.go b/internal/repository/prisma/ticker.go index e9630e10f..e485af6fc 100644 --- a/internal/repository/prisma/ticker.go +++ b/internal/repository/prisma/ticker.go @@ -102,6 +102,16 @@ func (t *tickerRepository) AddStepRun(tickerId, stepRunId string) (*db.TickerMod ).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), diff --git a/internal/repository/prisma/worker.go b/internal/repository/prisma/worker.go index b17b44388..533d60a8d 100644 --- a/internal/repository/prisma/worker.go +++ b/internal/repository/prisma/worker.go @@ -336,3 +336,23 @@ func (w *workerRepository) AddStepRun(tenantId, workerId, stepRunId string) erro return err } + +func (w *workerRepository) AddGetGroupKeyRun(tenantId, workerId, getGroupKeyRunId string) error { + tx1 := w.client.Worker.FindUnique( + db.Worker.ID.Equals(workerId), + ).Update( + db.Worker.GroupKeyRuns.Link( + db.GetGroupKeyRun.ID.Equals(getGroupKeyRunId), + ), + ).Tx() + + tx2 := w.client.GetGroupKeyRun.FindUnique( + db.GetGroupKeyRun.ID.Equals(getGroupKeyRunId), + ).Update( + db.GetGroupKeyRun.Status.Set(db.StepRunStatusAssigned), + ).Tx() + + err := w.client.Prisma.Transaction(tx1, tx2).Exec(context.Background()) + + return err +} diff --git a/internal/repository/prisma/workflow.go b/internal/repository/prisma/workflow.go index cd2b92bb6..ea020dcc6 100644 --- a/internal/repository/prisma/workflow.go +++ b/internal/repository/prisma/workflow.go @@ -474,6 +474,56 @@ func (r *workflowRepository) createWorkflowVersionTxs(ctx context.Context, tx pg return "", err } + // create concurrency group + if opts.Concurrency != nil { + // upsert the action + action, err := r.queries.UpsertAction( + context.Background(), + tx, + dbsqlc.UpsertActionParams{ + Action: opts.Concurrency.Action, + Tenantid: tenantId, + }, + ) + + if err != nil { + return "", fmt.Errorf("could not upsert action: %w", err) + } + + params := dbsqlc.CreateWorkflowConcurrencyParams{ + ID: sqlchelpers.UUIDFromStr(uuid.New().String()), + Workflowversionid: sqlcWorkflowVersion.ID, + Getconcurrencygroupid: action.ID, + } + + if opts.Concurrency.MaxRuns != nil { + params.MaxRuns = sqlchelpers.ToInt(*opts.Concurrency.MaxRuns) + } + + var ls dbsqlc.ConcurrencyLimitStrategy + + if opts.Concurrency.LimitStrategy != nil && *opts.Concurrency.LimitStrategy != "" { + ls = dbsqlc.ConcurrencyLimitStrategy(*opts.Concurrency.LimitStrategy) + } else { + ls = dbsqlc.ConcurrencyLimitStrategyCANCELINPROGRESS + } + + params.LimitStrategy = dbsqlc.NullConcurrencyLimitStrategy{ + Valid: true, + ConcurrencyLimitStrategy: ls, + } + + _, err = r.queries.CreateWorkflowConcurrency( + context.Background(), + tx, + params, + ) + + if err != nil { + return "", fmt.Errorf("could not create concurrency group: %w", err) + } + } + // create the workflow jobs for _, jobOpts := range opts.Jobs { jobId := uuid.New().String() @@ -519,7 +569,7 @@ func (r *workflowRepository) createWorkflowVersionTxs(ctx context.Context, tx pg } // upsert the action - err := r.queries.UpsertAction( + _, err := r.queries.UpsertAction( context.Background(), tx, dbsqlc.UpsertActionParams{ @@ -668,6 +718,9 @@ func defaultWorkflowVersionPopulator() []db.WorkflowVersionRelationWith { db.WorkflowTriggerCronRef.Ticker.Fetch(), ), ), + db.WorkflowVersion.Concurrency.Fetch().With( + db.WorkflowConcurrency.GetConcurrencyGroup.Fetch(), + ), db.WorkflowVersion.Jobs.Fetch().With( db.Job.Steps.Fetch().With( db.Step.Action.Fetch(), diff --git a/internal/repository/prisma/workflow_run.go b/internal/repository/prisma/workflow_run.go index d241b29ae..df41373d2 100644 --- a/internal/repository/prisma/workflow_run.go +++ b/internal/repository/prisma/workflow_run.go @@ -75,6 +75,13 @@ func (w *workflowRunRepository) ListWorkflowRuns(tenantId string, opts *reposito countParams.WorkflowId = pgWorkflowId } + if opts.WorkflowVersionId != nil { + pgWorkflowVersionId := sqlchelpers.UUIDFromStr(*opts.WorkflowVersionId) + + queryParams.WorkflowVersionId = pgWorkflowVersionId + countParams.WorkflowVersionId = pgWorkflowVersionId + } + if opts.EventId != nil { pgEventId := sqlchelpers.UUIDFromStr(*opts.EventId) @@ -82,9 +89,34 @@ func (w *workflowRunRepository) ListWorkflowRuns(tenantId string, opts *reposito countParams.EventId = pgEventId } + if opts.GroupKey != nil { + queryParams.GroupKey = sqlchelpers.TextFromStr(*opts.GroupKey) + countParams.GroupKey = sqlchelpers.TextFromStr(*opts.GroupKey) + } + + if opts.Status != nil { + var status dbsqlc.NullWorkflowRunStatus + + if err := status.Scan(string(*opts.Status)); err != nil { + return nil, err + } + + queryParams.Status = status + countParams.Status = status + } + orderByField := "createdAt" + + if opts.OrderBy != nil { + orderByField = *opts.OrderBy + } + orderByDirection := "DESC" + if opts.OrderDirection != nil { + orderByDirection = *opts.OrderDirection + } + queryParams.Orderby = orderByField + " " + orderByDirection tx, err := w.pool.Begin(context.Background()) @@ -198,12 +230,31 @@ func (w *workflowRunRepository) CreateNewWorkflowRun(ctx context.Context, tenant return nil, err } + requeueAfter := time.Now().UTC().Add(5 * time.Second) + + if opts.GetGroupKeyRun != nil { + params := dbsqlc.CreateGetGroupKeyRunParams{ + Tenantid: pgTenantId, + Workflowrunid: sqlcWorkflowRun.ID, + Input: opts.GetGroupKeyRun.Input, + Requeueafter: sqlchelpers.TimestampFromTime(requeueAfter), + } + + _, err = w.queries.CreateGetGroupKeyRun( + tx1Ctx, + tx, + params, + ) + + if err != nil { + return nil, err + } + } + // create the child jobs for _, jobOpts := range opts.JobRuns { jobRunId := uuid.New().String() - requeueAfter := time.Now().UTC().Add(5 * time.Second) - if jobOpts.RequeueAfter != nil { requeueAfter = *jobOpts.RequeueAfter } @@ -321,7 +372,11 @@ func defaultWorkflowRunPopulator() []db.WorkflowRunRelationWith { return []db.WorkflowRunRelationWith{ db.WorkflowRun.WorkflowVersion.Fetch().With( db.WorkflowVersion.Workflow.Fetch(), + db.WorkflowVersion.Concurrency.Fetch().With( + db.WorkflowConcurrency.GetConcurrencyGroup.Fetch(), + ), ), + db.WorkflowRun.GetGroupKeyRun.Fetch(), db.WorkflowRun.TriggeredBy.Fetch().With( db.WorkflowRunTriggeredBy.Event.Fetch(), db.WorkflowRunTriggeredBy.Cron.Fetch(), diff --git a/internal/repository/repository.go b/internal/repository/repository.go index 9be2b2d2f..448ca637f 100644 --- a/internal/repository/repository.go +++ b/internal/repository/repository.go @@ -9,6 +9,7 @@ type Repository interface { WorkflowRun() WorkflowRunRepository JobRun() JobRunRepository StepRun() StepRunRepository + GetGroupKeyRun() GetGroupKeyRunRepository Step() StepRepository Dispatcher() DispatcherRepository Ticker() TickerRepository diff --git a/internal/repository/step_run.go b/internal/repository/step_run.go index 769190621..850758e77 100644 --- a/internal/repository/step_run.go +++ b/internal/repository/step_run.go @@ -21,6 +21,8 @@ type ListStepRunsOpts struct { JobRunId *string + WorkflowRunId *string + Status *db.StepRunStatus } diff --git a/internal/repository/ticker.go b/internal/repository/ticker.go index 43eb9c404..c2aada6f5 100644 --- a/internal/repository/ticker.go +++ b/internal/repository/ticker.go @@ -42,6 +42,9 @@ type TickerRepository interface { // AddStepRun assigns a step run to a ticker. AddStepRun(tickerId, stepRunId string) (*db.TickerModel, error) + // AddGetGroupKeyRun assigns a get group key run to a ticker. + AddGetGroupKeyRun(tickerId, getGroupKeyRunId string) (*db.TickerModel, error) + // AddCron assigns a cron to a ticker. AddCron(tickerId string, cron *db.WorkflowTriggerCronRefModel) (*db.TickerModel, error) diff --git a/internal/repository/worker.go b/internal/repository/worker.go index 24c65aff2..76de25652 100644 --- a/internal/repository/worker.go +++ b/internal/repository/worker.go @@ -66,4 +66,7 @@ type WorkerRepository interface { // AddStepRun assigns a step run to a worker. AddStepRun(tenantId, workerId, stepRunId string) error + + // AddGetGroupKeyRun assigns a get group key run to a worker. + AddGetGroupKeyRun(tenantId, workerId, getGroupKeyRunId string) error } diff --git a/internal/repository/workflow.go b/internal/repository/workflow.go index 05caca931..5f7ba5761 100644 --- a/internal/repository/workflow.go +++ b/internal/repository/workflow.go @@ -33,6 +33,20 @@ type CreateWorkflowVersionOpts struct { // (required) the workflow jobs Jobs []CreateWorkflowJobOpts `validate:"required,min=1,dive"` + + // (optional) the workflow concurrency groups + Concurrency *CreateWorkflowConcurrencyOpts `json:"concurrency,omitempty" validator:"omitnil"` +} + +type CreateWorkflowConcurrencyOpts struct { + // (required) the action id for getting the concurrency group + Action string `validate:"required,actionId"` + + // (optional) the maximum number of concurrent workflow runs, default 1 + MaxRuns *int32 + + // (optional) the strategy to use when the concurrency limit is reached, default CANCEL_IN_PROGRESS + LimitStrategy *string `validate:"omitnil,oneof=CANCEL_IN_PROGRESS DROP_NEWEST QUEUE_NEWEST"` } func (o *CreateWorkflowVersionOpts) Checksum() (string, error) { diff --git a/internal/repository/workflow_run.go b/internal/repository/workflow_run.go index 15f61072d..78f9659ad 100644 --- a/internal/repository/workflow_run.go +++ b/internal/repository/workflow_run.go @@ -26,10 +26,23 @@ type CreateWorkflowRunOpts struct { // (required) the workflow jobs JobRuns []CreateWorkflowJobRunOpts `validate:"required,min=1,dive"` + + GetGroupKeyRun *CreateGroupKeyRunOpts `validate:"omitempty"` +} + +type CreateGroupKeyRunOpts struct { + // (optional) the input data + Input []byte } func GetCreateWorkflowRunOptsFromEvent(event *db.EventModel, workflowVersion *db.WorkflowVersionModel) (*CreateWorkflowRunOpts, error) { eventId := event.ID + + opts := &CreateWorkflowRunOpts{ + WorkflowVersionId: workflowVersion.ID, + TriggeringEventId: &eventId, + } + data := event.InnerEvent.Data var jobRunData []byte @@ -38,11 +51,12 @@ func GetCreateWorkflowRunOptsFromEvent(event *db.EventModel, workflowVersion *db if data != nil { jobRunData = []byte(json.RawMessage(*data)) - } - opts := &CreateWorkflowRunOpts{ - WorkflowVersionId: workflowVersion.ID, - TriggeringEventId: &eventId, + if _, hasConcurrency := workflowVersion.Concurrency(); hasConcurrency { + opts.GetGroupKeyRun = &CreateGroupKeyRunOpts{ + Input: jobRunData, + } + } } opts.JobRuns, err = getJobsFromWorkflowVersion(workflowVersion, datautils.TriggeredByEvent, jobRunData) @@ -65,6 +79,11 @@ func GetCreateWorkflowRunOptsFromCron(cron, cronParentId string, workflowVersion } func GetCreateWorkflowRunOptsFromSchedule(scheduledTrigger *db.WorkflowTriggerScheduledRefModel, workflowVersion *db.WorkflowVersionModel) (*CreateWorkflowRunOpts, error) { + opts := &CreateWorkflowRunOpts{ + WorkflowVersionId: workflowVersion.ID, + ScheduledWorkflowId: &scheduledTrigger.ID, + } + data := scheduledTrigger.InnerWorkflowTriggerScheduledRef.Input var jobRunData []byte @@ -72,11 +91,12 @@ func GetCreateWorkflowRunOptsFromSchedule(scheduledTrigger *db.WorkflowTriggerSc if data != nil { jobRunData = []byte(json.RawMessage(*data)) - } - opts := &CreateWorkflowRunOpts{ - WorkflowVersionId: workflowVersion.ID, - ScheduledWorkflowId: &scheduledTrigger.ID, + if _, hasConcurrency := workflowVersion.Concurrency(); hasConcurrency { + opts.GetGroupKeyRun = &CreateGroupKeyRunOpts{ + Input: jobRunData, + } + } } opts.JobRuns, err = getJobsFromWorkflowVersion(workflowVersion, datautils.TriggeredBySchedule, jobRunData) @@ -131,17 +151,32 @@ type CreateWorkflowStepRunOpts struct { } type ListWorkflowRunsOpts struct { - // (optional) the workflow version id + // (optional) the workflow id WorkflowId *string `validate:"omitempty,uuid"` + // (optional) the workflow version id + WorkflowVersionId *string `validate:"omitempty,uuid"` + // (optional) the event id that triggered the workflow run EventId *string `validate:"omitempty,uuid"` + // (optional) the group key for the workflow run + GroupKey *string + + // (optional) the status of the workflow run + Status *db.WorkflowRunStatus + // (optional) number of events to skip Offset *int // (optional) number of events to return Limit *int + + // (optional) the order by field + OrderBy *string `validate:"omitempty,oneof=createdAt"` + + // (optional) the order direction + OrderDirection *string `validate:"omitempty,oneof=ASC DESC"` } type ListWorkflowRunsResult struct { diff --git a/internal/services/admin/contracts/workflows.pb.go b/internal/services/admin/contracts/workflows.pb.go index 42643a0df..60969ecc2 100644 --- a/internal/services/admin/contracts/workflows.pb.go +++ b/internal/services/admin/contracts/workflows.pb.go @@ -22,6 +22,55 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +type ConcurrencyLimitStrategy int32 + +const ( + ConcurrencyLimitStrategy_CANCEL_IN_PROGRESS ConcurrencyLimitStrategy = 0 + ConcurrencyLimitStrategy_DROP_NEWEST ConcurrencyLimitStrategy = 1 + ConcurrencyLimitStrategy_QUEUE_NEWEST ConcurrencyLimitStrategy = 2 +) + +// Enum value maps for ConcurrencyLimitStrategy. +var ( + ConcurrencyLimitStrategy_name = map[int32]string{ + 0: "CANCEL_IN_PROGRESS", + 1: "DROP_NEWEST", + 2: "QUEUE_NEWEST", + } + ConcurrencyLimitStrategy_value = map[string]int32{ + "CANCEL_IN_PROGRESS": 0, + "DROP_NEWEST": 1, + "QUEUE_NEWEST": 2, + } +) + +func (x ConcurrencyLimitStrategy) Enum() *ConcurrencyLimitStrategy { + p := new(ConcurrencyLimitStrategy) + *p = x + return p +} + +func (x ConcurrencyLimitStrategy) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ConcurrencyLimitStrategy) Descriptor() protoreflect.EnumDescriptor { + return file_workflows_proto_enumTypes[0].Descriptor() +} + +func (ConcurrencyLimitStrategy) Type() protoreflect.EnumType { + return &file_workflows_proto_enumTypes[0] +} + +func (x ConcurrencyLimitStrategy) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ConcurrencyLimitStrategy.Descriptor instead. +func (ConcurrencyLimitStrategy) EnumDescriptor() ([]byte, []int) { + return file_workflows_proto_rawDescGZIP(), []int{0} +} + type PutWorkflowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -82,6 +131,7 @@ type CreateWorkflowVersionOpts struct { CronTriggers []string `protobuf:"bytes,5,rep,name=cron_triggers,json=cronTriggers,proto3" json:"cron_triggers,omitempty"` // (optional) cron triggers for the workflow ScheduledTriggers []*timestamppb.Timestamp `protobuf:"bytes,6,rep,name=scheduled_triggers,json=scheduledTriggers,proto3" json:"scheduled_triggers,omitempty"` // (optional) scheduled triggers for the workflow Jobs []*CreateWorkflowJobOpts `protobuf:"bytes,7,rep,name=jobs,proto3" json:"jobs,omitempty"` // (required) the workflow jobs + Concurrency *WorkflowConcurrencyOpts `protobuf:"bytes,8,opt,name=concurrency,proto3" json:"concurrency,omitempty"` // (optional) the workflow concurrency options } func (x *CreateWorkflowVersionOpts) Reset() { @@ -165,6 +215,76 @@ func (x *CreateWorkflowVersionOpts) GetJobs() []*CreateWorkflowJobOpts { return nil } +func (x *CreateWorkflowVersionOpts) GetConcurrency() *WorkflowConcurrencyOpts { + if x != nil { + return x.Concurrency + } + return nil +} + +type WorkflowConcurrencyOpts struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Action string `protobuf:"bytes,1,opt,name=action,proto3" json:"action,omitempty"` // (required) the action id for getting the concurrency group + MaxRuns int32 `protobuf:"varint,2,opt,name=max_runs,json=maxRuns,proto3" json:"max_runs,omitempty"` // (optional) the maximum number of concurrent workflow runs, default 1 + LimitStrategy ConcurrencyLimitStrategy `protobuf:"varint,3,opt,name=limit_strategy,json=limitStrategy,proto3,enum=ConcurrencyLimitStrategy" json:"limit_strategy,omitempty"` // (optional) the strategy to use when the concurrency limit is reached, default CANCEL_IN_PROGRESS +} + +func (x *WorkflowConcurrencyOpts) Reset() { + *x = WorkflowConcurrencyOpts{} + if protoimpl.UnsafeEnabled { + mi := &file_workflows_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkflowConcurrencyOpts) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkflowConcurrencyOpts) ProtoMessage() {} + +func (x *WorkflowConcurrencyOpts) ProtoReflect() protoreflect.Message { + mi := &file_workflows_proto_msgTypes[2] + 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 WorkflowConcurrencyOpts.ProtoReflect.Descriptor instead. +func (*WorkflowConcurrencyOpts) Descriptor() ([]byte, []int) { + return file_workflows_proto_rawDescGZIP(), []int{2} +} + +func (x *WorkflowConcurrencyOpts) GetAction() string { + if x != nil { + return x.Action + } + return "" +} + +func (x *WorkflowConcurrencyOpts) GetMaxRuns() int32 { + if x != nil { + return x.MaxRuns + } + return 0 +} + +func (x *WorkflowConcurrencyOpts) GetLimitStrategy() ConcurrencyLimitStrategy { + if x != nil { + return x.LimitStrategy + } + return ConcurrencyLimitStrategy_CANCEL_IN_PROGRESS +} + // CreateWorkflowJobOpts represents options to create a workflow job. type CreateWorkflowJobOpts struct { state protoimpl.MessageState @@ -180,7 +300,7 @@ type CreateWorkflowJobOpts struct { func (x *CreateWorkflowJobOpts) Reset() { *x = CreateWorkflowJobOpts{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[2] + mi := &file_workflows_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -193,7 +313,7 @@ func (x *CreateWorkflowJobOpts) String() string { func (*CreateWorkflowJobOpts) ProtoMessage() {} func (x *CreateWorkflowJobOpts) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[2] + mi := &file_workflows_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -206,7 +326,7 @@ func (x *CreateWorkflowJobOpts) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateWorkflowJobOpts.ProtoReflect.Descriptor instead. func (*CreateWorkflowJobOpts) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{2} + return file_workflows_proto_rawDescGZIP(), []int{3} } func (x *CreateWorkflowJobOpts) GetName() string { @@ -253,7 +373,7 @@ type CreateWorkflowStepOpts struct { func (x *CreateWorkflowStepOpts) Reset() { *x = CreateWorkflowStepOpts{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[3] + mi := &file_workflows_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -266,7 +386,7 @@ func (x *CreateWorkflowStepOpts) String() string { func (*CreateWorkflowStepOpts) ProtoMessage() {} func (x *CreateWorkflowStepOpts) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[3] + mi := &file_workflows_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -279,7 +399,7 @@ func (x *CreateWorkflowStepOpts) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateWorkflowStepOpts.ProtoReflect.Descriptor instead. func (*CreateWorkflowStepOpts) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{3} + return file_workflows_proto_rawDescGZIP(), []int{4} } func (x *CreateWorkflowStepOpts) GetReadableId() string { @@ -327,7 +447,7 @@ type ListWorkflowsRequest struct { func (x *ListWorkflowsRequest) Reset() { *x = ListWorkflowsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[4] + mi := &file_workflows_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -340,7 +460,7 @@ func (x *ListWorkflowsRequest) String() string { func (*ListWorkflowsRequest) ProtoMessage() {} func (x *ListWorkflowsRequest) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[4] + mi := &file_workflows_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -353,7 +473,7 @@ func (x *ListWorkflowsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListWorkflowsRequest.ProtoReflect.Descriptor instead. func (*ListWorkflowsRequest) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{4} + return file_workflows_proto_rawDescGZIP(), []int{5} } type ScheduleWorkflowRequest struct { @@ -370,7 +490,7 @@ type ScheduleWorkflowRequest struct { func (x *ScheduleWorkflowRequest) Reset() { *x = ScheduleWorkflowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[5] + mi := &file_workflows_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -383,7 +503,7 @@ func (x *ScheduleWorkflowRequest) String() string { func (*ScheduleWorkflowRequest) ProtoMessage() {} func (x *ScheduleWorkflowRequest) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[5] + mi := &file_workflows_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -396,7 +516,7 @@ func (x *ScheduleWorkflowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ScheduleWorkflowRequest.ProtoReflect.Descriptor instead. func (*ScheduleWorkflowRequest) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{5} + return file_workflows_proto_rawDescGZIP(), []int{6} } func (x *ScheduleWorkflowRequest) GetWorkflowId() string { @@ -432,7 +552,7 @@ type ListWorkflowsResponse struct { func (x *ListWorkflowsResponse) Reset() { *x = ListWorkflowsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[6] + mi := &file_workflows_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -445,7 +565,7 @@ func (x *ListWorkflowsResponse) String() string { func (*ListWorkflowsResponse) ProtoMessage() {} func (x *ListWorkflowsResponse) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[6] + mi := &file_workflows_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -458,7 +578,7 @@ func (x *ListWorkflowsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListWorkflowsResponse.ProtoReflect.Descriptor instead. func (*ListWorkflowsResponse) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{6} + return file_workflows_proto_rawDescGZIP(), []int{7} } func (x *ListWorkflowsResponse) GetWorkflows() []*Workflow { @@ -480,7 +600,7 @@ type ListWorkflowsForEventRequest struct { func (x *ListWorkflowsForEventRequest) Reset() { *x = ListWorkflowsForEventRequest{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[7] + mi := &file_workflows_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -493,7 +613,7 @@ func (x *ListWorkflowsForEventRequest) String() string { func (*ListWorkflowsForEventRequest) ProtoMessage() {} func (x *ListWorkflowsForEventRequest) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[7] + mi := &file_workflows_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -506,7 +626,7 @@ func (x *ListWorkflowsForEventRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListWorkflowsForEventRequest.ProtoReflect.Descriptor instead. func (*ListWorkflowsForEventRequest) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{7} + return file_workflows_proto_rawDescGZIP(), []int{8} } func (x *ListWorkflowsForEventRequest) GetEventKey() string { @@ -534,7 +654,7 @@ type Workflow struct { func (x *Workflow) Reset() { *x = Workflow{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[8] + mi := &file_workflows_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -547,7 +667,7 @@ func (x *Workflow) String() string { func (*Workflow) ProtoMessage() {} func (x *Workflow) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[8] + mi := &file_workflows_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -560,7 +680,7 @@ func (x *Workflow) ProtoReflect() protoreflect.Message { // Deprecated: Use Workflow.ProtoReflect.Descriptor instead. func (*Workflow) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{8} + return file_workflows_proto_rawDescGZIP(), []int{9} } func (x *Workflow) GetId() string { @@ -631,7 +751,7 @@ type WorkflowVersion struct { func (x *WorkflowVersion) Reset() { *x = WorkflowVersion{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[9] + mi := &file_workflows_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -644,7 +764,7 @@ func (x *WorkflowVersion) String() string { func (*WorkflowVersion) ProtoMessage() {} func (x *WorkflowVersion) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[9] + mi := &file_workflows_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -657,7 +777,7 @@ func (x *WorkflowVersion) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowVersion.ProtoReflect.Descriptor instead. func (*WorkflowVersion) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{9} + return file_workflows_proto_rawDescGZIP(), []int{10} } func (x *WorkflowVersion) GetId() string { @@ -734,7 +854,7 @@ type WorkflowTriggers struct { func (x *WorkflowTriggers) Reset() { *x = WorkflowTriggers{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[10] + mi := &file_workflows_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -747,7 +867,7 @@ func (x *WorkflowTriggers) String() string { func (*WorkflowTriggers) ProtoMessage() {} func (x *WorkflowTriggers) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[10] + mi := &file_workflows_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -760,7 +880,7 @@ func (x *WorkflowTriggers) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowTriggers.ProtoReflect.Descriptor instead. func (*WorkflowTriggers) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{10} + return file_workflows_proto_rawDescGZIP(), []int{11} } func (x *WorkflowTriggers) GetId() string { @@ -825,7 +945,7 @@ type WorkflowTriggerEventRef struct { func (x *WorkflowTriggerEventRef) Reset() { *x = WorkflowTriggerEventRef{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[11] + mi := &file_workflows_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -838,7 +958,7 @@ func (x *WorkflowTriggerEventRef) String() string { func (*WorkflowTriggerEventRef) ProtoMessage() {} func (x *WorkflowTriggerEventRef) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[11] + mi := &file_workflows_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -851,7 +971,7 @@ func (x *WorkflowTriggerEventRef) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowTriggerEventRef.ProtoReflect.Descriptor instead. func (*WorkflowTriggerEventRef) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{11} + return file_workflows_proto_rawDescGZIP(), []int{12} } func (x *WorkflowTriggerEventRef) GetParentId() string { @@ -881,7 +1001,7 @@ type WorkflowTriggerCronRef struct { func (x *WorkflowTriggerCronRef) Reset() { *x = WorkflowTriggerCronRef{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[12] + mi := &file_workflows_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -894,7 +1014,7 @@ func (x *WorkflowTriggerCronRef) String() string { func (*WorkflowTriggerCronRef) ProtoMessage() {} func (x *WorkflowTriggerCronRef) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[12] + mi := &file_workflows_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -907,7 +1027,7 @@ func (x *WorkflowTriggerCronRef) ProtoReflect() protoreflect.Message { // Deprecated: Use WorkflowTriggerCronRef.ProtoReflect.Descriptor instead. func (*WorkflowTriggerCronRef) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{12} + return file_workflows_proto_rawDescGZIP(), []int{13} } func (x *WorkflowTriggerCronRef) GetParentId() string { @@ -944,7 +1064,7 @@ type Job struct { func (x *Job) Reset() { *x = Job{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[13] + mi := &file_workflows_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -957,7 +1077,7 @@ func (x *Job) String() string { func (*Job) ProtoMessage() {} func (x *Job) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[13] + mi := &file_workflows_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -970,7 +1090,7 @@ func (x *Job) ProtoReflect() protoreflect.Message { // Deprecated: Use Job.ProtoReflect.Descriptor instead. func (*Job) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{13} + return file_workflows_proto_rawDescGZIP(), []int{14} } func (x *Job) GetId() string { @@ -1057,7 +1177,7 @@ type Step struct { func (x *Step) Reset() { *x = Step{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[14] + mi := &file_workflows_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1070,7 +1190,7 @@ func (x *Step) String() string { func (*Step) ProtoMessage() {} func (x *Step) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[14] + mi := &file_workflows_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1083,7 +1203,7 @@ func (x *Step) ProtoReflect() protoreflect.Message { // Deprecated: Use Step.ProtoReflect.Descriptor instead. func (*Step) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{14} + return file_workflows_proto_rawDescGZIP(), []int{15} } func (x *Step) GetId() string { @@ -1167,7 +1287,7 @@ type DeleteWorkflowRequest struct { func (x *DeleteWorkflowRequest) Reset() { *x = DeleteWorkflowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[15] + mi := &file_workflows_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1180,7 +1300,7 @@ func (x *DeleteWorkflowRequest) String() string { func (*DeleteWorkflowRequest) ProtoMessage() {} func (x *DeleteWorkflowRequest) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[15] + mi := &file_workflows_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1193,7 +1313,7 @@ func (x *DeleteWorkflowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteWorkflowRequest.ProtoReflect.Descriptor instead. func (*DeleteWorkflowRequest) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{15} + return file_workflows_proto_rawDescGZIP(), []int{16} } func (x *DeleteWorkflowRequest) GetWorkflowId() string { @@ -1214,7 +1334,7 @@ type GetWorkflowByNameRequest struct { func (x *GetWorkflowByNameRequest) Reset() { *x = GetWorkflowByNameRequest{} if protoimpl.UnsafeEnabled { - mi := &file_workflows_proto_msgTypes[16] + mi := &file_workflows_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1227,7 +1347,7 @@ func (x *GetWorkflowByNameRequest) String() string { func (*GetWorkflowByNameRequest) ProtoMessage() {} func (x *GetWorkflowByNameRequest) ProtoReflect() protoreflect.Message { - mi := &file_workflows_proto_msgTypes[16] + mi := &file_workflows_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1240,7 +1360,7 @@ func (x *GetWorkflowByNameRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetWorkflowByNameRequest.ProtoReflect.Descriptor instead. func (*GetWorkflowByNameRequest) Descriptor() ([]byte, []int) { - return file_workflows_proto_rawDescGZIP(), []int{16} + return file_workflows_proto_rawDescGZIP(), []int{17} } func (x *GetWorkflowByNameRequest) GetName() string { @@ -1262,7 +1382,7 @@ var file_workflows_proto_rawDesc = []byte{ 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2e, 0x0a, 0x04, 0x6f, 0x70, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4f, 0x70, - 0x74, 0x73, 0x52, 0x04, 0x6f, 0x70, 0x74, 0x73, 0x22, 0xae, 0x02, 0x0a, 0x19, 0x43, 0x72, 0x65, + 0x74, 0x73, 0x52, 0x04, 0x6f, 0x70, 0x74, 0x73, 0x22, 0xea, 0x02, 0x0a, 0x19, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x4f, 0x70, 0x74, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, @@ -1281,199 +1401,217 @@ var file_workflows_proto_rawDesc = []byte{ 0x64, 0x75, 0x6c, 0x65, 0x64, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x12, 0x2a, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4f, - 0x70, 0x74, 0x73, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0x96, 0x01, 0x0a, 0x15, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4f, - 0x70, 0x74, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, - 0x65, 0x6f, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, - 0x6f, 0x75, 0x74, 0x12, 0x2d, 0x0a, 0x05, 0x73, 0x74, 0x65, 0x70, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x65, 0x70, 0x4f, 0x70, 0x74, 0x73, 0x52, 0x05, 0x73, 0x74, 0x65, - 0x70, 0x73, 0x22, 0x9d, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x65, 0x70, 0x4f, 0x70, 0x74, 0x73, 0x12, 0x1f, 0x0a, - 0x0b, 0x72, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16, - 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, + 0x70, 0x74, 0x73, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x12, 0x3a, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, + 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, + 0x65, 0x6e, 0x63, 0x79, 0x4f, 0x70, 0x74, 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x6e, 0x63, 0x75, 0x72, + 0x72, 0x65, 0x6e, 0x63, 0x79, 0x22, 0x8e, 0x01, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x4f, 0x70, 0x74, + 0x73, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, 0x78, + 0x5f, 0x72, 0x75, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6d, 0x61, 0x78, + 0x52, 0x75, 0x6e, 0x73, 0x12, 0x40, 0x0a, 0x0e, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x5f, 0x73, 0x74, + 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x19, 0x2e, 0x43, + 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x53, + 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x0d, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x53, 0x74, + 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x22, 0x96, 0x01, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x4a, 0x6f, 0x62, 0x4f, 0x70, 0x74, 0x73, + 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, - 0x12, 0x16, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x72, 0x65, - 0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x72, 0x65, 0x6e, - 0x74, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x8a, 0x01, 0x0a, 0x17, 0x53, - 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, - 0x75, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 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, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, - 0x73, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x22, 0x40, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x27, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x09, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x09, - 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x3b, 0x0a, 0x1c, 0x4c, 0x69, 0x73, - 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x46, 0x6f, 0x72, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x65, - 0x6e, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x79, 0x22, 0xaf, 0x02, 0x0a, 0x08, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x02, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, - 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, + 0x12, 0x2d, 0x0a, 0x05, 0x73, 0x74, 0x65, 0x70, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x17, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x53, 0x74, 0x65, 0x70, 0x4f, 0x70, 0x74, 0x73, 0x52, 0x05, 0x73, 0x74, 0x65, 0x70, 0x73, 0x22, + 0x9d, 0x01, 0x0a, 0x16, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x53, 0x74, 0x65, 0x70, 0x4f, 0x70, 0x74, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, + 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x72, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x16, 0x0a, + 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x22, + 0x16, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x8a, 0x01, 0x0a, 0x17, 0x53, 0x63, 0x68, 0x65, + 0x64, 0x75, 0x6c, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, + 0x73, 0x18, 0x02, 0x20, 0x03, 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, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, - 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, + 0x61, 0x6d, 0x70, 0x52, 0x09, 0x73, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x14, + 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x22, 0x40, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, + 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x09, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x09, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x3b, 0x0a, 0x1c, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x46, 0x6f, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x4b, 0x65, 0x79, 0x22, 0xaf, 0x02, 0x0a, 0x08, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, + 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, + 0x74, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, + 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xb1, 0x02, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, + 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, + 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x72, 0x64, + 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x12, + 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, + 0x12, 0x2d, 0x0a, 0x08, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, + 0x67, 0x67, 0x65, 0x72, 0x73, 0x52, 0x08, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x12, + 0x18, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x04, 0x2e, + 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0xc6, 0x02, 0x0a, 0x10, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x12, 0x0e, + 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x39, + 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, - 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, - 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, - 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x0b, 0x64, 0x65, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0b, 0x64, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x0a, 0x08, 0x76, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x08, - 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xb1, 0x02, 0x0a, 0x0f, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x0a, - 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, - 0x41, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x14, 0x0a, 0x05, - 0x6f, 0x72, 0x64, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x6f, 0x72, 0x64, - 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, - 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x49, 0x64, 0x12, 0x2d, 0x0a, 0x08, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x52, 0x08, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, - 0x72, 0x73, 0x12, 0x18, 0x0a, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x04, 0x2e, 0x4a, 0x6f, 0x62, 0x52, 0x04, 0x6a, 0x6f, 0x62, 0x73, 0x22, 0xc6, 0x02, 0x0a, - 0x10, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, - 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, - 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, - 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x2e, 0x0a, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, - 0x6e, 0x74, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x07, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, - 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x52, 0x06, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2d, 0x0a, 0x05, 0x63, 0x72, 0x6f, 0x6e, 0x73, 0x18, - 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x72, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x52, 0x05, - 0x63, 0x72, 0x6f, 0x6e, 0x73, 0x22, 0x53, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, - 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1b, 0x0a, - 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x79, 0x22, 0x49, 0x0a, 0x16, 0x57, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x72, 0x6f, - 0x6e, 0x52, 0x65, 0x66, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, - 0x64, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x72, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x63, 0x72, 0x6f, 0x6e, 0x22, 0x81, 0x03, 0x0a, 0x03, 0x4a, 0x6f, 0x62, 0x12, 0x0e, 0x0a, - 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x39, 0x0a, - 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x64, 0x41, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, - 0x12, 0x2e, 0x0a, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x76, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x77, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x3e, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, - 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1b, 0x0a, 0x05, 0x73, 0x74, 0x65, 0x70, 0x73, 0x18, 0x09, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x05, 0x2e, 0x53, 0x74, 0x65, 0x70, 0x52, 0x05, 0x73, 0x74, 0x65, 0x70, - 0x73, 0x12, 0x36, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x0a, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0x85, 0x03, 0x0a, 0x04, 0x53, 0x74, - 0x65, 0x70, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, - 0x69, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, - 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x3d, 0x0a, 0x0b, 0x72, 0x65, 0x61, 0x64, - 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, - 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x61, - 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, - 0x6e, 0x74, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x36, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x09, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, - 0x75, 0x65, 0x52, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, - 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, - 0x72, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, - 0x6e, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, - 0x6e, 0x22, 0x38, 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x22, 0x2e, 0x0a, 0x18, 0x47, - 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x32, 0x87, 0x03, 0x0a, 0x0f, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, - 0x3e, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, - 0x12, 0x15, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x34, 0x0a, 0x0b, 0x50, 0x75, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x13, - 0x2e, 0x50, 0x75, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x10, 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, - 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x18, 0x2e, 0x53, 0x63, 0x68, 0x65, - 0x64, 0x75, 0x6c, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x2e, 0x47, 0x65, 0x74, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x09, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x12, 0x4e, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x73, 0x46, 0x6f, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x46, 0x6f, 0x72, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x33, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x12, 0x16, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x09, 0x2e, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x42, 0x5a, 0x40, 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, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x2f, - 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x64, 0x41, 0x74, 0x12, 0x2e, 0x0a, 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, + 0x64, 0x12, 0x30, 0x0a, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x18, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, 0x67, + 0x67, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x52, 0x06, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x2d, 0x0a, 0x05, 0x63, 0x72, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, + 0x67, 0x67, 0x65, 0x72, 0x43, 0x72, 0x6f, 0x6e, 0x52, 0x65, 0x66, 0x52, 0x05, 0x63, 0x72, 0x6f, + 0x6e, 0x73, 0x22, 0x53, 0x0a, 0x17, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x66, 0x12, 0x1b, 0x0a, + 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x4b, 0x65, 0x79, 0x22, 0x49, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x72, 0x6f, 0x6e, 0x52, 0x65, + 0x66, 0x12, 0x1b, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x12, + 0x0a, 0x04, 0x63, 0x72, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x72, + 0x6f, 0x6e, 0x22, 0x81, 0x03, 0x0a, 0x03, 0x4a, 0x6f, 0x62, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x0a, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, 0x09, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, + 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, + 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x2e, 0x0a, + 0x13, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x12, 0x3e, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x1b, 0x0a, 0x05, 0x73, 0x74, 0x65, 0x70, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x05, 0x2e, 0x53, 0x74, 0x65, 0x70, 0x52, 0x05, 0x73, 0x74, 0x65, 0x70, 0x73, 0x12, 0x36, + 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, + 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x07, 0x74, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0x85, 0x03, 0x0a, 0x04, 0x53, 0x74, 0x65, 0x70, 0x12, + 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, + 0x39, 0x0a, 0x0a, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 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, + 0x09, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x39, 0x0a, 0x0a, 0x75, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 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, 0x09, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x3d, 0x0a, 0x0b, 0x72, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, + 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x72, 0x65, 0x61, 0x64, 0x61, 0x62, + 0x6c, 0x65, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, + 0x64, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x36, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x72, 0x65, + 0x6e, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x18, 0x0b, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x63, 0x68, 0x69, 0x6c, 0x64, 0x72, 0x65, 0x6e, 0x22, 0x38, + 0x0a, 0x15, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x22, 0x2e, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x2a, 0x55, 0x0a, 0x18, 0x43, 0x6f, 0x6e, 0x63, + 0x75, 0x72, 0x72, 0x65, 0x6e, 0x63, 0x79, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x53, 0x74, 0x72, 0x61, + 0x74, 0x65, 0x67, 0x79, 0x12, 0x16, 0x0a, 0x12, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x49, + 0x4e, 0x5f, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, + 0x44, 0x52, 0x4f, 0x50, 0x5f, 0x4e, 0x45, 0x57, 0x45, 0x53, 0x54, 0x10, 0x01, 0x12, 0x10, 0x0a, + 0x0c, 0x51, 0x55, 0x45, 0x55, 0x45, 0x5f, 0x4e, 0x45, 0x57, 0x45, 0x53, 0x54, 0x10, 0x02, 0x32, + 0x87, 0x03, 0x0a, 0x0f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x12, 0x3e, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x15, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x34, 0x0a, 0x0b, 0x50, 0x75, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x12, 0x13, 0x2e, 0x50, 0x75, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3e, 0x0a, 0x10, 0x53, 0x63, 0x68, + 0x65, 0x64, 0x75, 0x6c, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x18, 0x2e, + 0x53, 0x63, 0x68, 0x65, 0x64, 0x75, 0x6c, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x39, 0x0a, 0x11, 0x47, 0x65, 0x74, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, + 0x2e, 0x47, 0x65, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x79, 0x4e, 0x61, + 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x09, 0x2e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x4e, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x46, 0x6f, 0x72, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1d, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x46, 0x6f, 0x72, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x16, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x09, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x42, 0x42, 0x5a, 0x40, 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, 0x61, 0x64, + 0x6d, 0x69, 0x6e, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x73, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1488,73 +1626,78 @@ func file_workflows_proto_rawDescGZIP() []byte { return file_workflows_proto_rawDescData } -var file_workflows_proto_msgTypes = make([]protoimpl.MessageInfo, 17) +var file_workflows_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_workflows_proto_msgTypes = make([]protoimpl.MessageInfo, 18) var file_workflows_proto_goTypes = []interface{}{ - (*PutWorkflowRequest)(nil), // 0: PutWorkflowRequest - (*CreateWorkflowVersionOpts)(nil), // 1: CreateWorkflowVersionOpts - (*CreateWorkflowJobOpts)(nil), // 2: CreateWorkflowJobOpts - (*CreateWorkflowStepOpts)(nil), // 3: CreateWorkflowStepOpts - (*ListWorkflowsRequest)(nil), // 4: ListWorkflowsRequest - (*ScheduleWorkflowRequest)(nil), // 5: ScheduleWorkflowRequest - (*ListWorkflowsResponse)(nil), // 6: ListWorkflowsResponse - (*ListWorkflowsForEventRequest)(nil), // 7: ListWorkflowsForEventRequest - (*Workflow)(nil), // 8: Workflow - (*WorkflowVersion)(nil), // 9: WorkflowVersion - (*WorkflowTriggers)(nil), // 10: WorkflowTriggers - (*WorkflowTriggerEventRef)(nil), // 11: WorkflowTriggerEventRef - (*WorkflowTriggerCronRef)(nil), // 12: WorkflowTriggerCronRef - (*Job)(nil), // 13: Job - (*Step)(nil), // 14: Step - (*DeleteWorkflowRequest)(nil), // 15: DeleteWorkflowRequest - (*GetWorkflowByNameRequest)(nil), // 16: GetWorkflowByNameRequest - (*timestamppb.Timestamp)(nil), // 17: google.protobuf.Timestamp - (*wrapperspb.StringValue)(nil), // 18: google.protobuf.StringValue + (ConcurrencyLimitStrategy)(0), // 0: ConcurrencyLimitStrategy + (*PutWorkflowRequest)(nil), // 1: PutWorkflowRequest + (*CreateWorkflowVersionOpts)(nil), // 2: CreateWorkflowVersionOpts + (*WorkflowConcurrencyOpts)(nil), // 3: WorkflowConcurrencyOpts + (*CreateWorkflowJobOpts)(nil), // 4: CreateWorkflowJobOpts + (*CreateWorkflowStepOpts)(nil), // 5: CreateWorkflowStepOpts + (*ListWorkflowsRequest)(nil), // 6: ListWorkflowsRequest + (*ScheduleWorkflowRequest)(nil), // 7: ScheduleWorkflowRequest + (*ListWorkflowsResponse)(nil), // 8: ListWorkflowsResponse + (*ListWorkflowsForEventRequest)(nil), // 9: ListWorkflowsForEventRequest + (*Workflow)(nil), // 10: Workflow + (*WorkflowVersion)(nil), // 11: WorkflowVersion + (*WorkflowTriggers)(nil), // 12: WorkflowTriggers + (*WorkflowTriggerEventRef)(nil), // 13: WorkflowTriggerEventRef + (*WorkflowTriggerCronRef)(nil), // 14: WorkflowTriggerCronRef + (*Job)(nil), // 15: Job + (*Step)(nil), // 16: Step + (*DeleteWorkflowRequest)(nil), // 17: DeleteWorkflowRequest + (*GetWorkflowByNameRequest)(nil), // 18: GetWorkflowByNameRequest + (*timestamppb.Timestamp)(nil), // 19: google.protobuf.Timestamp + (*wrapperspb.StringValue)(nil), // 20: google.protobuf.StringValue } var file_workflows_proto_depIdxs = []int32{ - 1, // 0: PutWorkflowRequest.opts:type_name -> CreateWorkflowVersionOpts - 17, // 1: CreateWorkflowVersionOpts.scheduled_triggers:type_name -> google.protobuf.Timestamp - 2, // 2: CreateWorkflowVersionOpts.jobs:type_name -> CreateWorkflowJobOpts - 3, // 3: CreateWorkflowJobOpts.steps:type_name -> CreateWorkflowStepOpts - 17, // 4: ScheduleWorkflowRequest.schedules:type_name -> google.protobuf.Timestamp - 8, // 5: ListWorkflowsResponse.workflows:type_name -> Workflow - 17, // 6: Workflow.created_at:type_name -> google.protobuf.Timestamp - 17, // 7: Workflow.updated_at:type_name -> google.protobuf.Timestamp - 18, // 8: Workflow.description:type_name -> google.protobuf.StringValue - 9, // 9: Workflow.versions:type_name -> WorkflowVersion - 17, // 10: WorkflowVersion.created_at:type_name -> google.protobuf.Timestamp - 17, // 11: WorkflowVersion.updated_at:type_name -> google.protobuf.Timestamp - 10, // 12: WorkflowVersion.triggers:type_name -> WorkflowTriggers - 13, // 13: WorkflowVersion.jobs:type_name -> Job - 17, // 14: WorkflowTriggers.created_at:type_name -> google.protobuf.Timestamp - 17, // 15: WorkflowTriggers.updated_at:type_name -> google.protobuf.Timestamp - 11, // 16: WorkflowTriggers.events:type_name -> WorkflowTriggerEventRef - 12, // 17: WorkflowTriggers.crons:type_name -> WorkflowTriggerCronRef - 17, // 18: Job.created_at:type_name -> google.protobuf.Timestamp - 17, // 19: Job.updated_at:type_name -> google.protobuf.Timestamp - 18, // 20: Job.description:type_name -> google.protobuf.StringValue - 14, // 21: Job.steps:type_name -> Step - 18, // 22: Job.timeout:type_name -> google.protobuf.StringValue - 17, // 23: Step.created_at:type_name -> google.protobuf.Timestamp - 17, // 24: Step.updated_at:type_name -> google.protobuf.Timestamp - 18, // 25: Step.readable_id:type_name -> google.protobuf.StringValue - 18, // 26: Step.timeout:type_name -> google.protobuf.StringValue - 4, // 27: WorkflowService.ListWorkflows:input_type -> ListWorkflowsRequest - 0, // 28: WorkflowService.PutWorkflow:input_type -> PutWorkflowRequest - 5, // 29: WorkflowService.ScheduleWorkflow:input_type -> ScheduleWorkflowRequest - 16, // 30: WorkflowService.GetWorkflowByName:input_type -> GetWorkflowByNameRequest - 7, // 31: WorkflowService.ListWorkflowsForEvent:input_type -> ListWorkflowsForEventRequest - 15, // 32: WorkflowService.DeleteWorkflow:input_type -> DeleteWorkflowRequest - 6, // 33: WorkflowService.ListWorkflows:output_type -> ListWorkflowsResponse - 9, // 34: WorkflowService.PutWorkflow:output_type -> WorkflowVersion - 9, // 35: WorkflowService.ScheduleWorkflow:output_type -> WorkflowVersion - 8, // 36: WorkflowService.GetWorkflowByName:output_type -> Workflow - 6, // 37: WorkflowService.ListWorkflowsForEvent:output_type -> ListWorkflowsResponse - 8, // 38: WorkflowService.DeleteWorkflow:output_type -> Workflow - 33, // [33:39] is the sub-list for method output_type - 27, // [27:33] is the sub-list for method input_type - 27, // [27:27] is the sub-list for extension type_name - 27, // [27:27] is the sub-list for extension extendee - 0, // [0:27] is the sub-list for field type_name + 2, // 0: PutWorkflowRequest.opts:type_name -> CreateWorkflowVersionOpts + 19, // 1: CreateWorkflowVersionOpts.scheduled_triggers:type_name -> google.protobuf.Timestamp + 4, // 2: CreateWorkflowVersionOpts.jobs:type_name -> CreateWorkflowJobOpts + 3, // 3: CreateWorkflowVersionOpts.concurrency:type_name -> WorkflowConcurrencyOpts + 0, // 4: WorkflowConcurrencyOpts.limit_strategy:type_name -> ConcurrencyLimitStrategy + 5, // 5: CreateWorkflowJobOpts.steps:type_name -> CreateWorkflowStepOpts + 19, // 6: ScheduleWorkflowRequest.schedules:type_name -> google.protobuf.Timestamp + 10, // 7: ListWorkflowsResponse.workflows:type_name -> Workflow + 19, // 8: Workflow.created_at:type_name -> google.protobuf.Timestamp + 19, // 9: Workflow.updated_at:type_name -> google.protobuf.Timestamp + 20, // 10: Workflow.description:type_name -> google.protobuf.StringValue + 11, // 11: Workflow.versions:type_name -> WorkflowVersion + 19, // 12: WorkflowVersion.created_at:type_name -> google.protobuf.Timestamp + 19, // 13: WorkflowVersion.updated_at:type_name -> google.protobuf.Timestamp + 12, // 14: WorkflowVersion.triggers:type_name -> WorkflowTriggers + 15, // 15: WorkflowVersion.jobs:type_name -> Job + 19, // 16: WorkflowTriggers.created_at:type_name -> google.protobuf.Timestamp + 19, // 17: WorkflowTriggers.updated_at:type_name -> google.protobuf.Timestamp + 13, // 18: WorkflowTriggers.events:type_name -> WorkflowTriggerEventRef + 14, // 19: WorkflowTriggers.crons:type_name -> WorkflowTriggerCronRef + 19, // 20: Job.created_at:type_name -> google.protobuf.Timestamp + 19, // 21: Job.updated_at:type_name -> google.protobuf.Timestamp + 20, // 22: Job.description:type_name -> google.protobuf.StringValue + 16, // 23: Job.steps:type_name -> Step + 20, // 24: Job.timeout:type_name -> google.protobuf.StringValue + 19, // 25: Step.created_at:type_name -> google.protobuf.Timestamp + 19, // 26: Step.updated_at:type_name -> google.protobuf.Timestamp + 20, // 27: Step.readable_id:type_name -> google.protobuf.StringValue + 20, // 28: Step.timeout:type_name -> google.protobuf.StringValue + 6, // 29: WorkflowService.ListWorkflows:input_type -> ListWorkflowsRequest + 1, // 30: WorkflowService.PutWorkflow:input_type -> PutWorkflowRequest + 7, // 31: WorkflowService.ScheduleWorkflow:input_type -> ScheduleWorkflowRequest + 18, // 32: WorkflowService.GetWorkflowByName:input_type -> GetWorkflowByNameRequest + 9, // 33: WorkflowService.ListWorkflowsForEvent:input_type -> ListWorkflowsForEventRequest + 17, // 34: WorkflowService.DeleteWorkflow:input_type -> DeleteWorkflowRequest + 8, // 35: WorkflowService.ListWorkflows:output_type -> ListWorkflowsResponse + 11, // 36: WorkflowService.PutWorkflow:output_type -> WorkflowVersion + 11, // 37: WorkflowService.ScheduleWorkflow:output_type -> WorkflowVersion + 10, // 38: WorkflowService.GetWorkflowByName:output_type -> Workflow + 8, // 39: WorkflowService.ListWorkflowsForEvent:output_type -> ListWorkflowsResponse + 10, // 40: WorkflowService.DeleteWorkflow:output_type -> Workflow + 35, // [35:41] is the sub-list for method output_type + 29, // [29:35] is the sub-list for method input_type + 29, // [29:29] is the sub-list for extension type_name + 29, // [29:29] is the sub-list for extension extendee + 0, // [0:29] is the sub-list for field type_name } func init() { file_workflows_proto_init() } @@ -1588,7 +1731,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateWorkflowJobOpts); i { + switch v := v.(*WorkflowConcurrencyOpts); i { case 0: return &v.state case 1: @@ -1600,7 +1743,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateWorkflowStepOpts); i { + switch v := v.(*CreateWorkflowJobOpts); i { case 0: return &v.state case 1: @@ -1612,7 +1755,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListWorkflowsRequest); i { + switch v := v.(*CreateWorkflowStepOpts); i { case 0: return &v.state case 1: @@ -1624,7 +1767,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ScheduleWorkflowRequest); i { + switch v := v.(*ListWorkflowsRequest); i { case 0: return &v.state case 1: @@ -1636,7 +1779,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListWorkflowsResponse); i { + switch v := v.(*ScheduleWorkflowRequest); i { case 0: return &v.state case 1: @@ -1648,7 +1791,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListWorkflowsForEventRequest); i { + switch v := v.(*ListWorkflowsResponse); i { case 0: return &v.state case 1: @@ -1660,7 +1803,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Workflow); i { + switch v := v.(*ListWorkflowsForEventRequest); i { case 0: return &v.state case 1: @@ -1672,7 +1815,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkflowVersion); i { + switch v := v.(*Workflow); i { case 0: return &v.state case 1: @@ -1684,7 +1827,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkflowTriggers); i { + switch v := v.(*WorkflowVersion); i { case 0: return &v.state case 1: @@ -1696,7 +1839,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkflowTriggerEventRef); i { + switch v := v.(*WorkflowTriggers); i { case 0: return &v.state case 1: @@ -1708,7 +1851,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WorkflowTriggerCronRef); i { + switch v := v.(*WorkflowTriggerEventRef); i { case 0: return &v.state case 1: @@ -1720,7 +1863,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Job); i { + switch v := v.(*WorkflowTriggerCronRef); i { case 0: return &v.state case 1: @@ -1732,7 +1875,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Step); i { + switch v := v.(*Job); i { case 0: return &v.state case 1: @@ -1744,7 +1887,7 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteWorkflowRequest); i { + switch v := v.(*Step); i { case 0: return &v.state case 1: @@ -1756,6 +1899,18 @@ func file_workflows_proto_init() { } } file_workflows_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DeleteWorkflowRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_workflows_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetWorkflowByNameRequest); i { case 0: return &v.state @@ -1773,13 +1928,14 @@ func file_workflows_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_workflows_proto_rawDesc, - NumEnums: 0, - NumMessages: 17, + NumEnums: 1, + NumMessages: 18, NumExtensions: 0, NumServices: 1, }, GoTypes: file_workflows_proto_goTypes, DependencyIndexes: file_workflows_proto_depIdxs, + EnumInfos: file_workflows_proto_enumTypes, MessageInfos: file_workflows_proto_msgTypes, }.Build() File_workflows_proto = out.File diff --git a/internal/services/admin/server.go b/internal/services/admin/server.go index 1008027a1..01d38aa26 100644 --- a/internal/services/admin/server.go +++ b/internal/services/admin/server.go @@ -543,8 +543,28 @@ func getCreateWorkflowOpts(req *contracts.PutWorkflowRequest) (*repository.Creat scheduledTriggers = append(scheduledTriggers, trigger.AsTime()) } + var concurrency *repository.CreateWorkflowConcurrencyOpts + + if req.Opts.Concurrency != nil { + var limitStrategy *string + + if req.Opts.Concurrency.LimitStrategy.String() != "" { + limitStrategy = repository.StringPtr(req.Opts.Concurrency.LimitStrategy.String()) + } + + concurrency = &repository.CreateWorkflowConcurrencyOpts{ + Action: req.Opts.Concurrency.Action, + LimitStrategy: limitStrategy, + } + + if req.Opts.Concurrency.MaxRuns != 0 { + concurrency.MaxRuns = &req.Opts.Concurrency.MaxRuns + } + } + return &repository.CreateWorkflowVersionOpts{ Name: req.Opts.Name, + Concurrency: concurrency, Description: &req.Opts.Description, Version: &req.Opts.Version, EventTriggers: req.Opts.EventTriggers, diff --git a/internal/services/eventscontroller/controller.go b/internal/services/controllers/events/controller.go similarity index 87% rename from internal/services/eventscontroller/controller.go rename to internal/services/controllers/events/controller.go index 80a17d996..98e61444c 100644 --- a/internal/services/eventscontroller/controller.go +++ b/internal/services/controllers/events/controller.go @@ -1,9 +1,8 @@ -package eventscontroller +package events import ( "context" "fmt" - "sync" "github.com/rs/zerolog" "golang.org/x/sync/errgroup" @@ -156,11 +155,7 @@ func (ec *EventsControllerImpl) processEvent(ctx context.Context, event *db.Even } // create a new workflow run in the database - var ( - g = new(errgroup.Group) - mu = &sync.Mutex{} - jobRuns = make([]*db.JobRunModel, 0) - ) + var g = new(errgroup.Group) for _, workflow := range workflows { workflowCp := workflow @@ -179,14 +174,12 @@ func (ec *EventsControllerImpl) processEvent(ctx context.Context, event *db.Even return fmt.Errorf("could not create workflow run: %w", err) } - for _, jobRun := range workflowRun.JobRuns() { - jobRunCp := jobRun - mu.Lock() - jobRuns = append(jobRuns, &jobRunCp) - mu.Unlock() - } - - return nil + // send to workflow processing queue + return ec.tq.AddTask( + context.Background(), + taskqueue.WORKFLOW_PROCESSING_QUEUE, + tasktypes.WorkflowRunQueuedToTask(workflowRun), + ) }) } @@ -194,18 +187,5 @@ func (ec *EventsControllerImpl) processEvent(ctx context.Context, event *db.Even return err } - // send jobs to the job processing queue - for i := range jobRuns { - err = ec.tq.AddTask( - context.Background(), - taskqueue.JOB_PROCESSING_QUEUE, - tasktypes.JobRunQueuedToTask(jobRuns[i].Job(), jobRuns[i]), - ) - - if err != nil { - return fmt.Errorf("could not add event to task queue: %w", err) - } - } - return nil } diff --git a/internal/services/jobscontroller/controller.go b/internal/services/controllers/jobs/controller.go similarity index 95% rename from internal/services/jobscontroller/controller.go rename to internal/services/controllers/jobs/controller.go index ecae0996b..b41833a48 100644 --- a/internal/services/jobscontroller/controller.go +++ b/internal/services/controllers/jobs/controller.go @@ -1,4 +1,4 @@ -package jobscontroller +package jobs import ( "context" @@ -138,6 +138,8 @@ func (ec *JobsControllerImpl) handleTask(ctx context.Context, task *taskqueue.Ta return ec.handleStepRunFinished(ctx, task) case "step-run-failed": return ec.handleStepRunFailed(ctx, task) + case "step-run-cancelled": + return ec.handleStepRunCancelled(ctx, task) case "step-run-timed-out": return ec.handleStepRunTimedOut(ctx, task) case "ticker-removed": @@ -729,30 +731,6 @@ func (ec *JobsControllerImpl) handleStepRunFinished(ctx context.Context, task *t servertel.WithJobRunModel(span, jobRun) - // stepReadableId, ok := stepRun.Step().ReadableID() - - // only update the job lookup data if the step has a readable id to key - // if ok && stepReadableId != "" { - // if payload.StepOutputData != "" { - // fmt.Println("UPDATING WITH PAYLOAD", payload.StepOutputData) - // unquoted, err := strconv.Unquote(payload.StepOutputData) - - // if err != nil { - // unquoted = payload.StepOutputData - // } - - // // update the job run lookup data - // err = ec.repo.JobRun().UpdateJobRunLookupData(metadata.TenantId, stepRun.JobRunID, &repository.UpdateJobRunLookupDataOpts{ - // FieldPath: []string{"steps", stepReadableId}, - // Data: []byte(unquoted), - // }) - - // if err != nil { - // return fmt.Errorf("could not update job run lookup data: %w", err) - // } - // } - // } - // queue the next step runs nextStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(metadata.TenantId, jobRun.ID, stepRun.ID) @@ -768,14 +746,6 @@ func (ec *JobsControllerImpl) handleStepRunFinished(ctx context.Context, task *t } } - // if next, ok := stepRun.Next(); ok && next != nil { - // err := ec.queueStepRun(ctx, metadata.TenantId, next.StepID, next.ID) - - // if err != nil { - // return fmt.Errorf("could not queue next step run: %w", err) - // } - // } - // cancel the timeout task stepRunTicker, ok := stepRun.Ticker() @@ -868,12 +838,41 @@ func (ec *JobsControllerImpl) handleStepRunTimedOut(ctx context.Context, task *t return fmt.Errorf("could not decode step run started task metadata: %w", err) } - now := time.Now().UTC() + return ec.cancelStepRun(ctx, metadata.TenantId, payload.StepRunId, "TIMED_OUT") +} + +func (ec *JobsControllerImpl) handleStepRunCancelled(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "handle-step-run-cancelled") + defer span.End() + + payload := tasktypes.StepRunNotifyCancelTaskPayload{} + metadata := tasktypes.StepRunNotifyCancelTaskMetadata{} + + err := ec.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode step run notify cancel task payload: %w", err) + } + + err = ec.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode step run notify cancel task metadata: %w", err) + } + + return ec.cancelStepRun(ctx, metadata.TenantId, payload.StepRunId, payload.CancelledReason) +} + +func (ec *JobsControllerImpl) cancelStepRun(ctx context.Context, tenantId, stepRunId, reason string) error { + ctx, span := telemetry.NewSpan(ctx, "cancel-step-run") + defer span.End() // cancel current step run - stepRun, err := ec.repo.StepRun().UpdateStepRun(metadata.TenantId, payload.StepRunId, &repository.UpdateStepRunOpts{ + now := time.Now().UTC() + + stepRun, err := ec.repo.StepRun().UpdateStepRun(tenantId, stepRunId, &repository.UpdateStepRunOpts{ CancelledAt: &now, - CancelledReason: repository.StringPtr("TIMED_OUT"), + CancelledReason: repository.StringPtr(reason), Status: repository.StepRunStatusPtr(db.StepRunStatusCancelled), }) @@ -899,20 +898,13 @@ func (ec *JobsControllerImpl) handleStepRunTimedOut(ctx context.Context, task *t err = ec.tq.AddTask( ctx, taskqueue.QueueTypeFromDispatcherID(worker.Dispatcher().ID), - stepRunCancelledTask(metadata.TenantId, payload.StepRunId, "TIMED_OUT", worker), + stepRunCancelledTask(tenantId, stepRunId, reason, worker), ) if err != nil { return fmt.Errorf("could not add job assigned task to task queue: %w", err) } - // // cancel any pending steps for this job run in the database - // err = ec.repo.StepRun().CancelPendingStepRuns(metadata.TenantId, payload.JobRunId, "PREVIOUS_STEP_TIMED_OUT") - - // if err != nil { - // return fmt.Errorf("could not cancel pending step runs: %w", err) - // } - return nil } diff --git a/internal/services/controllers/workflows/controller.go b/internal/services/controllers/workflows/controller.go new file mode 100644 index 000000000..4051bcfa2 --- /dev/null +++ b/internal/services/controllers/workflows/controller.go @@ -0,0 +1,321 @@ +package workflows + +import ( + "context" + "fmt" + "time" + + "github.com/rs/zerolog" + "golang.org/x/sync/errgroup" + + "github.com/hatchet-dev/hatchet/internal/datautils" + "github.com/hatchet-dev/hatchet/internal/logger" + "github.com/hatchet-dev/hatchet/internal/repository" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" + "github.com/hatchet-dev/hatchet/internal/services/shared/tasktypes" + "github.com/hatchet-dev/hatchet/internal/taskqueue" + "github.com/hatchet-dev/hatchet/internal/telemetry" +) + +type WorkflowsController interface { + Start(ctx context.Context) error +} + +type WorkflowsControllerImpl struct { + tq taskqueue.TaskQueue + l *zerolog.Logger + repo repository.Repository + dv datautils.DataDecoderValidator +} + +type WorkflowsControllerOpt func(*WorkflowsControllerOpts) + +type WorkflowsControllerOpts struct { + tq taskqueue.TaskQueue + l *zerolog.Logger + repo repository.Repository + dv datautils.DataDecoderValidator +} + +func defaultWorkflowsControllerOpts() *WorkflowsControllerOpts { + logger := logger.NewDefaultLogger("workflows-controller") + return &WorkflowsControllerOpts{ + l: &logger, + dv: datautils.NewDataDecoderValidator(), + } +} + +func WithTaskQueue(tq taskqueue.TaskQueue) WorkflowsControllerOpt { + return func(opts *WorkflowsControllerOpts) { + opts.tq = tq + } +} + +func WithLogger(l *zerolog.Logger) WorkflowsControllerOpt { + return func(opts *WorkflowsControllerOpts) { + opts.l = l + } +} + +func WithRepository(r repository.Repository) WorkflowsControllerOpt { + return func(opts *WorkflowsControllerOpts) { + opts.repo = r + } +} + +func WithDataDecoderValidator(dv datautils.DataDecoderValidator) WorkflowsControllerOpt { + return func(opts *WorkflowsControllerOpts) { + opts.dv = dv + } +} + +func New(fs ...WorkflowsControllerOpt) (*WorkflowsControllerImpl, error) { + opts := defaultWorkflowsControllerOpts() + + for _, f := range fs { + f(opts) + } + + if opts.tq == nil { + return nil, fmt.Errorf("task queue is required. use WithTaskQueue") + } + + if opts.repo == nil { + return nil, fmt.Errorf("repository is required. use WithRepository") + } + + newLogger := opts.l.With().Str("service", "workflows-controller").Logger() + opts.l = &newLogger + + return &WorkflowsControllerImpl{ + tq: opts.tq, + l: opts.l, + repo: opts.repo, + dv: opts.dv, + }, nil +} + +func (wc *WorkflowsControllerImpl) Start(ctx context.Context) error { + wc.l.Debug().Msg("starting workflows controller") + + taskChan, err := wc.tq.Subscribe(ctx, taskqueue.WORKFLOW_PROCESSING_QUEUE) + + if err != nil { + return err + } + + // TODO: close when ctx is done + for task := range taskChan { + go func(task *taskqueue.Task) { + err = wc.handleTask(ctx, task) + + if err != nil { + wc.l.Error().Err(err).Msg("could not handle job task") + } + }(task) + } + + return nil +} + +func (wc *WorkflowsControllerImpl) handleTask(ctx context.Context, task *taskqueue.Task) error { + switch task.ID { + case "workflow-run-queued": + return wc.handleWorkflowRunQueued(ctx, task) + case "group-key-action-requeue-ticker": + return wc.handleGroupKeyActionRequeue(ctx, task) + case "get-group-key-run-started": + return wc.handleGroupKeyRunStarted(ctx, task) + case "get-group-key-run-finished": + return wc.handleGroupKeyRunFinished(ctx, task) + case "get-group-key-run-failed": + return wc.handleGroupKeyRunFailed(ctx, task) + case "workflow-run-finished": + // return ec.handleStepRunStarted(ctx, task) + } + + return fmt.Errorf("unknown task: %s in queue %s", task.ID, string(task.Queue)) +} + +func (ec *WorkflowsControllerImpl) handleGroupKeyRunStarted(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "get-group-key-run-started") + defer span.End() + + payload := tasktypes.GetGroupKeyRunStartedTaskPayload{} + metadata := tasktypes.GetGroupKeyRunStartedTaskMetadata{} + + err := ec.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode step run started task payload: %w", err) + } + + err = ec.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode step run started task metadata: %w", err) + } + + // update the get group key run in the database + startedAt, err := time.Parse(time.RFC3339, payload.StartedAt) + + if err != nil { + return fmt.Errorf("could not parse started at: %w", err) + } + + _, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{ + StartedAt: &startedAt, + Status: repository.StepRunStatusPtr(db.StepRunStatusRunning), + }) + + return err +} + +func (wc *WorkflowsControllerImpl) handleGroupKeyRunFinished(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "handle-step-run-finished") + defer span.End() + + payload := tasktypes.GetGroupKeyRunFinishedTaskPayload{} + metadata := tasktypes.GetGroupKeyRunFinishedTaskMetadata{} + + err := wc.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode step run started task payload: %w", err) + } + + err = wc.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode step run started task metadata: %w", err) + } + + // update the step run in the database + finishedAt, err := time.Parse(time.RFC3339, payload.FinishedAt) + + if err != nil { + return fmt.Errorf("could not parse started at: %w", err) + } + + groupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{ + FinishedAt: &finishedAt, + Status: repository.StepRunStatusPtr(db.StepRunStatusSucceeded), + Output: &payload.GroupKey, + }) + + if err != nil { + return fmt.Errorf("could not update step run: %w", err) + } + + errGroup := new(errgroup.Group) + + errGroup.Go(func() error { + workflowVersion, err := wc.repo.Workflow().GetWorkflowVersionById(metadata.TenantId, groupKeyRun.WorkflowRun().WorkflowVersionID) + + if err != nil { + return fmt.Errorf("could not get workflow version: %w", err) + } + + concurrency, _ := workflowVersion.Concurrency() + + switch concurrency.LimitStrategy { + case db.ConcurrencyLimitStrategyCancelInProgress: + err = wc.queueByCancelInProgress(ctx, metadata.TenantId, payload.GroupKey, workflowVersion) + default: + return fmt.Errorf("unimplemented concurrency limit strategy: %s", concurrency.LimitStrategy) + } + + return err + }) + + // cancel the timeout task + errGroup.Go(func() error { + groupKeyRunTicker, ok := groupKeyRun.Ticker() + + if ok { + err = wc.tq.AddTask( + ctx, + taskqueue.QueueTypeFromTickerID(groupKeyRunTicker.ID), + cancelGetGroupKeyRunTimeoutTask(groupKeyRunTicker, groupKeyRun), + ) + + if err != nil { + return fmt.Errorf("could not add cancel step run timeout task to task queue: %w", err) + } + } + + return nil + }) + + return errGroup.Wait() +} + +func (wc *WorkflowsControllerImpl) handleGroupKeyRunFailed(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "handle-group-key-run-failed") + defer span.End() + + payload := tasktypes.GetGroupKeyRunFailedTaskPayload{} + metadata := tasktypes.GetGroupKeyRunFailedTaskMetadata{} + + err := wc.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode step run started task payload: %w", err) + } + + err = wc.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode step run started task metadata: %w", err) + } + + // update the group key run in the database + failedAt, err := time.Parse(time.RFC3339, payload.FailedAt) + if err != nil { + return fmt.Errorf("could not parse started at: %w", err) + } + + groupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{ + FinishedAt: &failedAt, + Error: &payload.Error, + Status: repository.StepRunStatusPtr(db.StepRunStatusFailed), + }) + + if err != nil { + return fmt.Errorf("could not update step run: %w", err) + } + + // cancel the ticker for the step run + getGroupKeyRunTicker, ok := groupKeyRun.Ticker() + + if ok { + err = wc.tq.AddTask( + ctx, + taskqueue.QueueTypeFromTickerID(getGroupKeyRunTicker.ID), + cancelGetGroupKeyRunTimeoutTask(getGroupKeyRunTicker, groupKeyRun), + ) + + if err != nil { + return fmt.Errorf("could not add cancel step run timeout task to task queue: %w", err) + } + } + + return nil +} + +func cancelGetGroupKeyRunTimeoutTask(ticker *db.TickerModel, getGroupKeyRun *db.GetGroupKeyRunModel) *taskqueue.Task { + payload, _ := datautils.ToJSONMap(tasktypes.CancelGetGroupKeyRunTimeoutTaskPayload{ + GetGroupKeyRunId: getGroupKeyRun.ID, + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.CancelGetGroupKeyRunTimeoutTaskMetadata{ + TenantId: getGroupKeyRun.TenantID, + }) + + return &taskqueue.Task{ + ID: "cancel-get-group-key-run-timeout", + Queue: taskqueue.QueueTypeFromTickerID(ticker.ID), + Payload: payload, + Metadata: metadata, + } +} diff --git a/internal/services/controllers/workflows/queue.go b/internal/services/controllers/workflows/queue.go new file mode 100644 index 000000000..a2f7d3265 --- /dev/null +++ b/internal/services/controllers/workflows/queue.go @@ -0,0 +1,508 @@ +package workflows + +import ( + "context" + "fmt" + "time" + + "github.com/hashicorp/go-multierror" + "golang.org/x/sync/errgroup" + + "github.com/hatchet-dev/hatchet/internal/datautils" + "github.com/hatchet-dev/hatchet/internal/repository" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" + "github.com/hatchet-dev/hatchet/internal/services/shared/defaults" + "github.com/hatchet-dev/hatchet/internal/services/shared/tasktypes" + "github.com/hatchet-dev/hatchet/internal/taskqueue" + "github.com/hatchet-dev/hatchet/internal/telemetry" + "github.com/hatchet-dev/hatchet/internal/telemetry/servertel" +) + +func (wc *WorkflowsControllerImpl) handleWorkflowRunQueued(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "handle-workflow-run-queued") + defer span.End() + + payload := tasktypes.WorkflowRunQueuedTaskPayload{} + metadata := tasktypes.WorkflowRunQueuedTaskMetadata{} + + err := wc.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode job task payload: %w", err) + } + + err = wc.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode job task metadata: %w", err) + } + + // get the workflow run in the database + workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId) + + if err != nil { + return fmt.Errorf("could not get job run: %w", err) + } + + servertel.WithWorkflowRunModel(span, workflowRun) + + wc.l.Info().Msgf("starting workflow run %s", workflowRun.ID) + + // determine if we should start this workflow run or we need to limit its concurrency + // if the workflow has concurrency settings, then we need to check if we can start it + if _, hasConcurrency := workflowRun.WorkflowVersion().Concurrency(); hasConcurrency { + wc.l.Info().Msgf("workflow %s has concurrency settings", workflowRun.ID) + + groupKeyRun, ok := workflowRun.GetGroupKeyRun() + + if !ok { + return fmt.Errorf("could not get group key run") + } + + err = wc.scheduleGetGroupAction(ctx, groupKeyRun) + + if err != nil { + return fmt.Errorf("could not trigger get group action: %w", err) + } + + return nil + } + + err = wc.queueWorkflowRunJobs(ctx, workflowRun) + + if err != nil { + return fmt.Errorf("could not start workflow run: %w", err) + } + + return nil +} + +func (wc *WorkflowsControllerImpl) scheduleGetGroupAction( + ctx context.Context, + getGroupKeyRun *db.GetGroupKeyRunModel, +) error { + ctx, span := telemetry.NewSpan(ctx, "trigger-get-group-action") + defer span.End() + + getGroupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(getGroupKeyRun.TenantID, getGroupKeyRun.ID, &repository.UpdateGetGroupKeyRunOpts{ + Status: repository.StepRunStatusPtr(db.StepRunStatusPendingAssignment), + }) + + if err != nil { + return fmt.Errorf("could not update get group key run: %w", err) + } + + workflowRun := getGroupKeyRun.WorkflowRun() + concurrency, hasConcurrency := workflowRun.WorkflowVersion().Concurrency() + + if !hasConcurrency { + return fmt.Errorf("workflow run does not have concurrency settings") + } + + tenantId := workflowRun.TenantID + + // Assign the get group action to a worker. + // + // 1. Get a list of workers that can run this step. If there are no workers available, then return an error. + // 2. Pick a worker to run the step and get the dispatcher currently connected to this worker. + // 3. Update the step run's designated worker. + // + // After creating the worker, send a task to the taskqueue, which will be picked up by the dispatcher. + after := time.Now().UTC().Add(-6 * time.Second) + + getAction, ok := concurrency.GetConcurrencyGroup() + + if !ok { + return fmt.Errorf("could not get concurrency group") + } + + workers, err := wc.repo.Worker().ListWorkers(workflowRun.TenantID, &repository.ListWorkersOpts{ + Action: &getAction.ActionID, + LastHeartbeatAfter: &after, + }) + + if err != nil { + return fmt.Errorf("could not list workers for step: %w", err) + } + + if len(workers) == 0 { + wc.l.Debug().Msgf("no workers available for action %s, requeueing", getAction.ActionID) + return nil + } + + // pick the worker with the least jobs currently assigned (this heuristic can and should change) + selectedWorker := workers[0] + + for _, worker := range workers { + if worker.StepRunCount < selectedWorker.StepRunCount { + selectedWorker = worker + } + } + + telemetry.WithAttributes(span, servertel.WorkerId(selectedWorker.Worker.ID)) + + // update the job run's designated worker + err = wc.repo.Worker().AddGetGroupKeyRun(tenantId, selectedWorker.Worker.ID, getGroupKeyRun.ID) + + if err != nil { + return fmt.Errorf("could not add step run to worker: %w", err) + } + + // pick a ticker to use for timeout + tickers, err := wc.getValidTickers() + + if err != nil { + return err + } + + ticker := &tickers[0] + + ticker, err = wc.repo.Ticker().AddGetGroupKeyRun(ticker.ID, getGroupKeyRun.ID) + + if err != nil { + return fmt.Errorf("could not add step run to ticker: %w", err) + } + + scheduleTimeoutTask, err := scheduleGetGroupKeyRunTimeoutTask(ticker, getGroupKeyRun) + + if err != nil { + return fmt.Errorf("could not schedule step run timeout task: %w", err) + } + + // send a task to the dispatcher + err = wc.tq.AddTask( + ctx, + taskqueue.QueueTypeFromDispatcherID(selectedWorker.Worker.Dispatcher().ID), + getGroupActionTask(workflowRun.TenantID, workflowRun.ID, selectedWorker.Worker), + ) + + if err != nil { + return fmt.Errorf("could not add job assigned task to task queue: %w", err) + } + + // send a task to the ticker + err = wc.tq.AddTask( + ctx, + taskqueue.QueueTypeFromTickerID(ticker.ID), + scheduleTimeoutTask, + ) + + if err != nil { + return fmt.Errorf("could not add schedule step run timeout task to task queue: %w", err) + } + + return nil +} + +func (wc *WorkflowsControllerImpl) queueWorkflowRunJobs(ctx context.Context, workflowRun *db.WorkflowRunModel) error { + ctx, span := telemetry.NewSpan(ctx, "process-event") + defer span.End() + + jobRuns := workflowRun.JobRuns() + + var err error + + for i := range jobRuns { + err := wc.tq.AddTask( + context.Background(), + taskqueue.JOB_PROCESSING_QUEUE, + tasktypes.JobRunQueuedToTask(jobRuns[i].Job(), &jobRuns[i]), + ) + + if err != nil { + err = multierror.Append(err, fmt.Errorf("could not add job run to task queue: %w", err)) + } + } + + return err +} + +func (wc *WorkflowsControllerImpl) handleGroupKeyActionRequeue(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "handle-group-key-action-requeue-ticker") + defer span.End() + + payload := tasktypes.GroupKeyActionRequeueTaskPayload{} + metadata := tasktypes.GroupKeyActionRequeueTaskMetadata{} + + err := wc.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode job task payload: %w", err) + } + + err = wc.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode job task metadata: %w", err) + } + + getGroupKeyRuns, err := wc.repo.GetGroupKeyRun().ListGetGroupKeyRuns(payload.TenantId, &repository.ListGetGroupKeyRunsOpts{ + Requeuable: repository.BoolPtr(true), + }) + + if err != nil { + return fmt.Errorf("could not list step runs: %w", err) + } + + g := new(errgroup.Group) + + for _, getGroupKeyRun := range getGroupKeyRuns { + getGroupKeyRunCp := getGroupKeyRun + + // wrap in func to get defer on the span to avoid leaking spans + g.Go(func() error { + + ctx, span := telemetry.NewSpan(ctx, "handle-step-run-requeue-step-run") + defer span.End() + + wc.l.Debug().Msgf("requeueing step run %s", getGroupKeyRunCp.ID) + + now := time.Now().UTC().UTC() + + // if the current time is after the scheduleTimeoutAt, then mark this as timed out + if getGroupKeyRunCp.CreatedAt.Add(30 * time.Second).Before(now) { + _, err = wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(payload.TenantId, getGroupKeyRunCp.ID, &repository.UpdateGetGroupKeyRunOpts{ + CancelledAt: &now, + CancelledReason: repository.StringPtr("SCHEDULING_TIMED_OUT"), + Status: repository.StepRunStatusPtr(db.StepRunStatusCancelled), + }) + + if err != nil { + return fmt.Errorf("could not update step run %s: %w", getGroupKeyRunCp.ID, err) + } + + return nil + } + + requeueAfter := time.Now().UTC().Add(time.Second * 5) + + getGroupKeyRunP, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(payload.TenantId, getGroupKeyRunCp.ID, &repository.UpdateGetGroupKeyRunOpts{ + RequeueAfter: &requeueAfter, + }) + + if err != nil { + return fmt.Errorf("could not update get group key run %s: %w", getGroupKeyRunP.ID, err) + } + + return wc.scheduleGetGroupAction(ctx, getGroupKeyRunP) + }) + } + + return nil +} + +func (ec *WorkflowsControllerImpl) getValidTickers() ([]db.TickerModel, error) { + within := time.Now().UTC().Add(-6 * time.Second) + + tickers, err := ec.repo.Ticker().ListTickers(&repository.ListTickerOpts{ + LatestHeartbeatAt: &within, + Active: repository.BoolPtr(true), + }) + + if err != nil { + return nil, fmt.Errorf("could not list tickers: %w", err) + } + + if len(tickers) == 0 { + return nil, fmt.Errorf("no tickers available") + } + + return tickers, nil +} + +func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context, tenantId, groupKey string, workflowVersion *db.WorkflowVersionModel) error { + ctx, span := telemetry.NewSpan(ctx, "cancel-by-cancel-in-progress") + defer span.End() + + wc.l.Info().Msgf("handling queue with strategy CANCEL_IN_PROGRESS for %s", groupKey) + + concurrency, hasConcurrency := workflowVersion.Concurrency() + + if !hasConcurrency { + return nil + } + + // list all workflow runs that are running for this group key + running := db.WorkflowRunStatusRunning + + runningWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(tenantId, &repository.ListWorkflowRunsOpts{ + WorkflowVersionId: &concurrency.WorkflowVersionID, + GroupKey: &groupKey, + Status: &running, + // order from oldest to newest + OrderBy: repository.StringPtr("createdAt"), + OrderDirection: repository.StringPtr("ASC"), + }) + + if err != nil { + return fmt.Errorf("could not list running workflow runs: %w", err) + } + + // get workflow runs which are queued for this group key + queued := db.WorkflowRunStatusQueued + + queuedWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(tenantId, &repository.ListWorkflowRunsOpts{ + WorkflowVersionId: &concurrency.WorkflowVersionID, + GroupKey: &groupKey, + Status: &queued, + // order from oldest to newest + OrderBy: repository.StringPtr("createdAt"), + OrderDirection: repository.StringPtr("ASC"), + Limit: &concurrency.MaxRuns, + }) + + if err != nil { + return fmt.Errorf("could not list queued workflow runs: %w", err) + } + + // cancel up to maxRuns - queued runs + maxRuns := concurrency.MaxRuns + maxToQueue := min(maxRuns, len(queuedWorkflowRuns.Rows)) + errGroup := new(errgroup.Group) + + for i := range runningWorkflowRuns.Rows { + // in this strategy we need to make room for all of the queued runs + if i >= len(queuedWorkflowRuns.Rows) { + break + } + + row := runningWorkflowRuns.Rows[i] + + errGroup.Go(func() error { + return wc.cancelWorkflowRun(tenantId, row.WorkflowRun.ID) + }) + } + + if err := errGroup.Wait(); err != nil { + return fmt.Errorf("could not cancel workflow runs: %w", err) + } + + errGroup = new(errgroup.Group) + + for i := range queuedWorkflowRuns.Rows { + if i >= maxToQueue { + break + } + + row := queuedWorkflowRuns.Rows[i] + + errGroup.Go(func() error { + workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(tenantId, row.WorkflowRun.ID) + + if err != nil { + return fmt.Errorf("could not get workflow run: %w", err) + } + + return wc.queueWorkflowRunJobs(ctx, workflowRun) + }) + } + + if err := errGroup.Wait(); err != nil { + return fmt.Errorf("could not queue workflow runs: %w", err) + } + + return nil +} + +func (wc *WorkflowsControllerImpl) cancelWorkflowRun(tenantId, workflowRunId string) error { + // get the workflow run in the database + workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(tenantId, workflowRunId) + + if err != nil { + return fmt.Errorf("could not get workflow run: %w", err) + } + + // cancel all running step runs + stepRuns, err := wc.repo.StepRun().ListStepRuns(tenantId, &repository.ListStepRunsOpts{ + WorkflowRunId: &workflowRun.ID, + Status: repository.StepRunStatusPtr(db.StepRunStatusRunning), + }) + + if err != nil { + return fmt.Errorf("could not list step runs: %w", err) + } + + errGroup := new(errgroup.Group) + + for i := range stepRuns { + stepRunCp := stepRuns[i] + errGroup.Go(func() error { + return wc.tq.AddTask( + context.Background(), + taskqueue.JOB_PROCESSING_QUEUE, + getStepRunNotifyCancelTask(tenantId, stepRunCp.ID, "CANCELLED_BY_CONCURRENCY_LIMIT"), + ) + }) + } + + return errGroup.Wait() +} + +func getGroupActionTask(tenantId, workflowRunId string, worker *db.WorkerModel) *taskqueue.Task { + dispatcher := worker.Dispatcher() + + payload, _ := datautils.ToJSONMap(tasktypes.GroupKeyActionAssignedTaskPayload{ + WorkflowRunId: workflowRunId, + WorkerId: worker.ID, + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.GroupKeyActionAssignedTaskMetadata{ + TenantId: tenantId, + DispatcherId: dispatcher.ID, + }) + + return &taskqueue.Task{ + ID: "group-key-action-assigned", + Queue: taskqueue.QueueTypeFromDispatcherID(dispatcher.ID), + Payload: payload, + Metadata: metadata, + } +} + +func getStepRunNotifyCancelTask(tenantId, stepRunId, reason string) *taskqueue.Task { + payload, _ := datautils.ToJSONMap(tasktypes.StepRunNotifyCancelTaskPayload{ + StepRunId: stepRunId, + CancelledReason: reason, + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.StepRunNotifyCancelTaskMetadata{ + TenantId: tenantId, + }) + + return &taskqueue.Task{ + ID: "step-run-cancelled", + Queue: taskqueue.JOB_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + } +} + +func scheduleGetGroupKeyRunTimeoutTask(ticker *db.TickerModel, getGroupKeyRun *db.GetGroupKeyRunModel) (*taskqueue.Task, error) { + durationStr := defaults.DefaultStepRunTimeout + + // get a duration + duration, err := time.ParseDuration(durationStr) + + if err != nil { + return nil, fmt.Errorf("could not parse duration: %w", err) + } + + timeoutAt := time.Now().UTC().Add(duration) + + payload, _ := datautils.ToJSONMap(tasktypes.ScheduleGetGroupKeyRunTimeoutTaskPayload{ + GetGroupKeyRunId: getGroupKeyRun.ID, + WorkflowRunId: getGroupKeyRun.WorkflowRunID, + TimeoutAt: timeoutAt.Format(time.RFC3339), + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.ScheduleGetGroupKeyRunTimeoutTaskMetadata{ + TenantId: getGroupKeyRun.TenantID, + }) + + return &taskqueue.Task{ + ID: "schedule-get-group-key-run-timeout", + Queue: taskqueue.QueueTypeFromTickerID(ticker.ID), + Payload: payload, + Metadata: metadata, + }, nil +} diff --git a/internal/services/dispatcher/contracts/dispatcher.pb.go b/internal/services/dispatcher/contracts/dispatcher.pb.go index 6c8161afe..b1ddeea82 100644 --- a/internal/services/dispatcher/contracts/dispatcher.pb.go +++ b/internal/services/dispatcher/contracts/dispatcher.pb.go @@ -24,8 +24,9 @@ const ( type ActionType int32 const ( - ActionType_START_STEP_RUN ActionType = 0 - ActionType_CANCEL_STEP_RUN ActionType = 1 + ActionType_START_STEP_RUN ActionType = 0 + ActionType_CANCEL_STEP_RUN ActionType = 1 + ActionType_START_GET_GROUP_KEY ActionType = 2 ) // Enum value maps for ActionType. @@ -33,10 +34,12 @@ var ( ActionType_name = map[int32]string{ 0: "START_STEP_RUN", 1: "CANCEL_STEP_RUN", + 2: "START_GET_GROUP_KEY", } ActionType_value = map[string]int32{ - "START_STEP_RUN": 0, - "CANCEL_STEP_RUN": 1, + "START_STEP_RUN": 0, + "CANCEL_STEP_RUN": 1, + "START_GET_GROUP_KEY": 2, } ) @@ -67,24 +70,76 @@ func (ActionType) EnumDescriptor() ([]byte, []int) { return file_dispatcher_proto_rawDescGZIP(), []int{0} } -type ActionEventType int32 +type GroupKeyActionEventType int32 const ( - ActionEventType_STEP_EVENT_TYPE_UNKNOWN ActionEventType = 0 - ActionEventType_STEP_EVENT_TYPE_STARTED ActionEventType = 1 - ActionEventType_STEP_EVENT_TYPE_COMPLETED ActionEventType = 2 - ActionEventType_STEP_EVENT_TYPE_FAILED ActionEventType = 3 + GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_UNKNOWN GroupKeyActionEventType = 0 + GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_STARTED GroupKeyActionEventType = 1 + GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_COMPLETED GroupKeyActionEventType = 2 + GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_FAILED GroupKeyActionEventType = 3 ) -// Enum value maps for ActionEventType. +// Enum value maps for GroupKeyActionEventType. var ( - ActionEventType_name = map[int32]string{ + GroupKeyActionEventType_name = map[int32]string{ + 0: "GROUP_KEY_EVENT_TYPE_UNKNOWN", + 1: "GROUP_KEY_EVENT_TYPE_STARTED", + 2: "GROUP_KEY_EVENT_TYPE_COMPLETED", + 3: "GROUP_KEY_EVENT_TYPE_FAILED", + } + GroupKeyActionEventType_value = map[string]int32{ + "GROUP_KEY_EVENT_TYPE_UNKNOWN": 0, + "GROUP_KEY_EVENT_TYPE_STARTED": 1, + "GROUP_KEY_EVENT_TYPE_COMPLETED": 2, + "GROUP_KEY_EVENT_TYPE_FAILED": 3, + } +) + +func (x GroupKeyActionEventType) Enum() *GroupKeyActionEventType { + p := new(GroupKeyActionEventType) + *p = x + return p +} + +func (x GroupKeyActionEventType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (GroupKeyActionEventType) Descriptor() protoreflect.EnumDescriptor { + return file_dispatcher_proto_enumTypes[1].Descriptor() +} + +func (GroupKeyActionEventType) Type() protoreflect.EnumType { + return &file_dispatcher_proto_enumTypes[1] +} + +func (x GroupKeyActionEventType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use GroupKeyActionEventType.Descriptor instead. +func (GroupKeyActionEventType) EnumDescriptor() ([]byte, []int) { + return file_dispatcher_proto_rawDescGZIP(), []int{1} +} + +type StepActionEventType int32 + +const ( + StepActionEventType_STEP_EVENT_TYPE_UNKNOWN StepActionEventType = 0 + StepActionEventType_STEP_EVENT_TYPE_STARTED StepActionEventType = 1 + StepActionEventType_STEP_EVENT_TYPE_COMPLETED StepActionEventType = 2 + StepActionEventType_STEP_EVENT_TYPE_FAILED StepActionEventType = 3 +) + +// Enum value maps for StepActionEventType. +var ( + StepActionEventType_name = map[int32]string{ 0: "STEP_EVENT_TYPE_UNKNOWN", 1: "STEP_EVENT_TYPE_STARTED", 2: "STEP_EVENT_TYPE_COMPLETED", 3: "STEP_EVENT_TYPE_FAILED", } - ActionEventType_value = map[string]int32{ + StepActionEventType_value = map[string]int32{ "STEP_EVENT_TYPE_UNKNOWN": 0, "STEP_EVENT_TYPE_STARTED": 1, "STEP_EVENT_TYPE_COMPLETED": 2, @@ -92,31 +147,31 @@ var ( } ) -func (x ActionEventType) Enum() *ActionEventType { - p := new(ActionEventType) +func (x StepActionEventType) Enum() *StepActionEventType { + p := new(StepActionEventType) *p = x return p } -func (x ActionEventType) String() string { +func (x StepActionEventType) String() string { return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) } -func (ActionEventType) Descriptor() protoreflect.EnumDescriptor { - return file_dispatcher_proto_enumTypes[1].Descriptor() +func (StepActionEventType) Descriptor() protoreflect.EnumDescriptor { + return file_dispatcher_proto_enumTypes[2].Descriptor() } -func (ActionEventType) Type() protoreflect.EnumType { - return &file_dispatcher_proto_enumTypes[1] +func (StepActionEventType) Type() protoreflect.EnumType { + return &file_dispatcher_proto_enumTypes[2] } -func (x ActionEventType) Number() protoreflect.EnumNumber { +func (x StepActionEventType) Number() protoreflect.EnumNumber { return protoreflect.EnumNumber(x) } -// Deprecated: Use ActionEventType.Descriptor instead. -func (ActionEventType) EnumDescriptor() ([]byte, []int) { - return file_dispatcher_proto_rawDescGZIP(), []int{1} +// Deprecated: Use StepActionEventType.Descriptor instead. +func (StepActionEventType) EnumDescriptor() ([]byte, []int) { + return file_dispatcher_proto_rawDescGZIP(), []int{2} } type WorkerRegisterRequest struct { @@ -258,22 +313,26 @@ type AssignedAction struct { // the tenant id TenantId string `protobuf:"bytes,1,opt,name=tenantId,proto3" json:"tenantId,omitempty"` + // the workflow run id (optional) + WorkflowRunId string `protobuf:"bytes,2,opt,name=workflowRunId,proto3" json:"workflowRunId,omitempty"` + // the get group key run id (optional) + GetGroupKeyRunId string `protobuf:"bytes,3,opt,name=getGroupKeyRunId,proto3" json:"getGroupKeyRunId,omitempty"` // the job id - JobId string `protobuf:"bytes,2,opt,name=jobId,proto3" json:"jobId,omitempty"` + JobId string `protobuf:"bytes,4,opt,name=jobId,proto3" json:"jobId,omitempty"` // the job name - JobName string `protobuf:"bytes,3,opt,name=jobName,proto3" json:"jobName,omitempty"` + JobName string `protobuf:"bytes,5,opt,name=jobName,proto3" json:"jobName,omitempty"` // the job run id - JobRunId string `protobuf:"bytes,4,opt,name=jobRunId,proto3" json:"jobRunId,omitempty"` + JobRunId string `protobuf:"bytes,6,opt,name=jobRunId,proto3" json:"jobRunId,omitempty"` // the step id - StepId string `protobuf:"bytes,5,opt,name=stepId,proto3" json:"stepId,omitempty"` + StepId string `protobuf:"bytes,7,opt,name=stepId,proto3" json:"stepId,omitempty"` // the step run id - StepRunId string `protobuf:"bytes,6,opt,name=stepRunId,proto3" json:"stepRunId,omitempty"` + StepRunId string `protobuf:"bytes,8,opt,name=stepRunId,proto3" json:"stepRunId,omitempty"` // the action id - ActionId string `protobuf:"bytes,7,opt,name=actionId,proto3" json:"actionId,omitempty"` + ActionId string `protobuf:"bytes,9,opt,name=actionId,proto3" json:"actionId,omitempty"` // the action type - ActionType ActionType `protobuf:"varint,8,opt,name=actionType,proto3,enum=ActionType" json:"actionType,omitempty"` + ActionType ActionType `protobuf:"varint,10,opt,name=actionType,proto3,enum=ActionType" json:"actionType,omitempty"` // the action payload - ActionPayload string `protobuf:"bytes,9,opt,name=actionPayload,proto3" json:"actionPayload,omitempty"` + ActionPayload string `protobuf:"bytes,11,opt,name=actionPayload,proto3" json:"actionPayload,omitempty"` } func (x *AssignedAction) Reset() { @@ -315,6 +374,20 @@ func (x *AssignedAction) GetTenantId() string { return "" } +func (x *AssignedAction) GetWorkflowRunId() string { + if x != nil { + return x.WorkflowRunId + } + return "" +} + +func (x *AssignedAction) GetGetGroupKeyRunId() string { + if x != nil { + return x.GetGroupKeyRunId + } + return "" +} + func (x *AssignedAction) GetJobId() string { if x != nil { return x.JobId @@ -524,7 +597,107 @@ func (x *WorkerUnsubscribeResponse) GetWorkerId() string { return "" } -type ActionEvent struct { +type GroupKeyActionEvent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // the id of the worker + WorkerId string `protobuf:"bytes,1,opt,name=workerId,proto3" json:"workerId,omitempty"` + // the id of the job + WorkflowRunId string `protobuf:"bytes,2,opt,name=workflowRunId,proto3" json:"workflowRunId,omitempty"` + GetGroupKeyRunId string `protobuf:"bytes,3,opt,name=getGroupKeyRunId,proto3" json:"getGroupKeyRunId,omitempty"` + // the action id + ActionId string `protobuf:"bytes,4,opt,name=actionId,proto3" json:"actionId,omitempty"` + EventTimestamp *timestamppb.Timestamp `protobuf:"bytes,5,opt,name=eventTimestamp,proto3" json:"eventTimestamp,omitempty"` + // the step event type + EventType GroupKeyActionEventType `protobuf:"varint,6,opt,name=eventType,proto3,enum=GroupKeyActionEventType" json:"eventType,omitempty"` + // the event payload + EventPayload string `protobuf:"bytes,7,opt,name=eventPayload,proto3" json:"eventPayload,omitempty"` +} + +func (x *GroupKeyActionEvent) Reset() { + *x = GroupKeyActionEvent{} + if protoimpl.UnsafeEnabled { + mi := &file_dispatcher_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GroupKeyActionEvent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GroupKeyActionEvent) ProtoMessage() {} + +func (x *GroupKeyActionEvent) ProtoReflect() protoreflect.Message { + mi := &file_dispatcher_proto_msgTypes[6] + 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 GroupKeyActionEvent.ProtoReflect.Descriptor instead. +func (*GroupKeyActionEvent) Descriptor() ([]byte, []int) { + return file_dispatcher_proto_rawDescGZIP(), []int{6} +} + +func (x *GroupKeyActionEvent) GetWorkerId() string { + if x != nil { + return x.WorkerId + } + return "" +} + +func (x *GroupKeyActionEvent) GetWorkflowRunId() string { + if x != nil { + return x.WorkflowRunId + } + return "" +} + +func (x *GroupKeyActionEvent) GetGetGroupKeyRunId() string { + if x != nil { + return x.GetGroupKeyRunId + } + return "" +} + +func (x *GroupKeyActionEvent) GetActionId() string { + if x != nil { + return x.ActionId + } + return "" +} + +func (x *GroupKeyActionEvent) GetEventTimestamp() *timestamppb.Timestamp { + if x != nil { + return x.EventTimestamp + } + return nil +} + +func (x *GroupKeyActionEvent) GetEventType() GroupKeyActionEventType { + if x != nil { + return x.EventType + } + return GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_UNKNOWN +} + +func (x *GroupKeyActionEvent) GetEventPayload() string { + if x != nil { + return x.EventPayload + } + return "" +} + +type StepActionEvent struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -543,28 +716,28 @@ type ActionEvent struct { ActionId string `protobuf:"bytes,6,opt,name=actionId,proto3" json:"actionId,omitempty"` EventTimestamp *timestamppb.Timestamp `protobuf:"bytes,7,opt,name=eventTimestamp,proto3" json:"eventTimestamp,omitempty"` // the step event type - EventType ActionEventType `protobuf:"varint,8,opt,name=eventType,proto3,enum=ActionEventType" json:"eventType,omitempty"` + EventType StepActionEventType `protobuf:"varint,8,opt,name=eventType,proto3,enum=StepActionEventType" json:"eventType,omitempty"` // the event payload EventPayload string `protobuf:"bytes,9,opt,name=eventPayload,proto3" json:"eventPayload,omitempty"` } -func (x *ActionEvent) Reset() { - *x = ActionEvent{} +func (x *StepActionEvent) Reset() { + *x = StepActionEvent{} if protoimpl.UnsafeEnabled { - mi := &file_dispatcher_proto_msgTypes[6] + mi := &file_dispatcher_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *ActionEvent) String() string { +func (x *StepActionEvent) String() string { return protoimpl.X.MessageStringOf(x) } -func (*ActionEvent) ProtoMessage() {} +func (*StepActionEvent) ProtoMessage() {} -func (x *ActionEvent) ProtoReflect() protoreflect.Message { - mi := &file_dispatcher_proto_msgTypes[6] +func (x *StepActionEvent) ProtoReflect() protoreflect.Message { + mi := &file_dispatcher_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -575,68 +748,68 @@ func (x *ActionEvent) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use ActionEvent.ProtoReflect.Descriptor instead. -func (*ActionEvent) Descriptor() ([]byte, []int) { - return file_dispatcher_proto_rawDescGZIP(), []int{6} +// Deprecated: Use StepActionEvent.ProtoReflect.Descriptor instead. +func (*StepActionEvent) Descriptor() ([]byte, []int) { + return file_dispatcher_proto_rawDescGZIP(), []int{7} } -func (x *ActionEvent) GetWorkerId() string { +func (x *StepActionEvent) GetWorkerId() string { if x != nil { return x.WorkerId } return "" } -func (x *ActionEvent) GetJobId() string { +func (x *StepActionEvent) GetJobId() string { if x != nil { return x.JobId } return "" } -func (x *ActionEvent) GetJobRunId() string { +func (x *StepActionEvent) GetJobRunId() string { if x != nil { return x.JobRunId } return "" } -func (x *ActionEvent) GetStepId() string { +func (x *StepActionEvent) GetStepId() string { if x != nil { return x.StepId } return "" } -func (x *ActionEvent) GetStepRunId() string { +func (x *StepActionEvent) GetStepRunId() string { if x != nil { return x.StepRunId } return "" } -func (x *ActionEvent) GetActionId() string { +func (x *StepActionEvent) GetActionId() string { if x != nil { return x.ActionId } return "" } -func (x *ActionEvent) GetEventTimestamp() *timestamppb.Timestamp { +func (x *StepActionEvent) GetEventTimestamp() *timestamppb.Timestamp { if x != nil { return x.EventTimestamp } return nil } -func (x *ActionEvent) GetEventType() ActionEventType { +func (x *StepActionEvent) GetEventType() StepActionEventType { if x != nil { return x.EventType } - return ActionEventType_STEP_EVENT_TYPE_UNKNOWN + return StepActionEventType_STEP_EVENT_TYPE_UNKNOWN } -func (x *ActionEvent) GetEventPayload() string { +func (x *StepActionEvent) GetEventPayload() string { if x != nil { return x.EventPayload } @@ -657,7 +830,7 @@ type ActionEventResponse struct { func (x *ActionEventResponse) Reset() { *x = ActionEventResponse{} if protoimpl.UnsafeEnabled { - mi := &file_dispatcher_proto_msgTypes[7] + mi := &file_dispatcher_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -670,7 +843,7 @@ func (x *ActionEventResponse) String() string { func (*ActionEventResponse) ProtoMessage() {} func (x *ActionEventResponse) ProtoReflect() protoreflect.Message { - mi := &file_dispatcher_proto_msgTypes[7] + mi := &file_dispatcher_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -683,7 +856,7 @@ func (x *ActionEventResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ActionEventResponse.ProtoReflect.Descriptor instead. func (*ActionEventResponse) Descriptor() ([]byte, []int) { - return file_dispatcher_proto_rawDescGZIP(), []int{7} + return file_dispatcher_proto_rawDescGZIP(), []int{8} } func (x *ActionEventResponse) GetTenantId() string { @@ -720,95 +893,138 @@ var file_dispatcher_proto_rawDesc = []byte{ 0x65, 0x72, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, - 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x9d, 0x02, 0x0a, 0x0e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xef, 0x02, 0x0a, 0x0e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, - 0x74, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6a, 0x6f, 0x62, - 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, - 0x16, 0x0a, 0x06, 0x73, 0x74, 0x65, 0x70, 0x49, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x73, 0x74, 0x65, 0x70, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, - 0x75, 0x6e, 0x49, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x65, 0x70, - 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x64, 0x12, 0x2b, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x24, - 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, - 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, - 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x31, 0x0a, 0x13, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69, - 0x73, 0x74, 0x65, 0x6e, 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, 0x22, 0x36, 0x0a, 0x18, 0x57, 0x6f, 0x72, 0x6b, 0x65, - 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 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, 0x22, - 0x53, 0x0a, 0x19, 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, 0x12, 0x1a, 0x0a, 0x08, - 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, - 0x65, 0x72, 0x49, 0x64, 0x22, 0xc5, 0x02, 0x0a, 0x0b, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, - 0x76, 0x65, 0x6e, 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, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x6a, 0x6f, 0x62, 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, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x65, 0x70, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x65, 0x70, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, - 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, - 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x42, 0x0a, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, - 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x07, 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, 0x2e, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x10, 0x2e, 0x41, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, - 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x4d, 0x0a, 0x13, - 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, - 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x2a, 0x35, 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, 0x2a, 0x86, 0x01, 0x0a, 0x0f, 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, 0x32, 0x81, 0x02, 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, 0x37, - 0x0a, 0x0f, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x12, 0x0c, 0x2e, 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, 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, + 0x74, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, + 0x75, 0x6e, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x2a, 0x0a, 0x10, 0x67, 0x65, 0x74, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x10, 0x67, 0x65, 0x74, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, + 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6a, + 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6a, 0x6f, + 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, + 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, + 0x64, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x65, 0x70, 0x49, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x73, 0x74, 0x65, 0x70, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65, + 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, + 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, + 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0b, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x24, 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, + 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x31, 0x0a, 0x13, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 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, 0x22, 0x36, 0x0a, 0x18, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 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, 0x22, 0x53, 0x0a, 0x19, 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, 0x12, 0x1a, + 0x0a, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x22, 0xbf, 0x02, 0x0a, 0x13, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 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, 0x24, 0x0a, 0x0d, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, + 0x12, 0x2a, 0x0a, 0x10, 0x67, 0x65, 0x74, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x52, + 0x75, 0x6e, 0x49, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x67, 0x65, 0x74, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 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, 0x36, 0x0a, 0x09, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x18, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xcd, 0x02, 0x0a, 0x0f, 0x53, 0x74, 0x65, + 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 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, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 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, 0x16, 0x0a, 0x06, 0x73, 0x74, + 0x65, 0x70, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x74, 0x65, 0x70, + 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, + 0x12, 0x1a, 0x0a, 0x08, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x08, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x42, 0x0a, 0x0e, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x07, + 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, 0x32, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x4d, 0x0a, 0x13, 0x41, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x1a, 0x0a, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x08, 0x74, 0x65, 0x6e, 0x61, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x77, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 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, 0x32, 0xd2, 0x02, 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, 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, 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 ( @@ -823,38 +1039,44 @@ func file_dispatcher_proto_rawDescGZIP() []byte { return file_dispatcher_proto_rawDescData } -var file_dispatcher_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_dispatcher_proto_msgTypes = make([]protoimpl.MessageInfo, 8) +var file_dispatcher_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_dispatcher_proto_msgTypes = make([]protoimpl.MessageInfo, 9) var file_dispatcher_proto_goTypes = []interface{}{ (ActionType)(0), // 0: ActionType - (ActionEventType)(0), // 1: ActionEventType - (*WorkerRegisterRequest)(nil), // 2: WorkerRegisterRequest - (*WorkerRegisterResponse)(nil), // 3: WorkerRegisterResponse - (*AssignedAction)(nil), // 4: AssignedAction - (*WorkerListenRequest)(nil), // 5: WorkerListenRequest - (*WorkerUnsubscribeRequest)(nil), // 6: WorkerUnsubscribeRequest - (*WorkerUnsubscribeResponse)(nil), // 7: WorkerUnsubscribeResponse - (*ActionEvent)(nil), // 8: ActionEvent - (*ActionEventResponse)(nil), // 9: ActionEventResponse - (*timestamppb.Timestamp)(nil), // 10: google.protobuf.Timestamp + (GroupKeyActionEventType)(0), // 1: GroupKeyActionEventType + (StepActionEventType)(0), // 2: StepActionEventType + (*WorkerRegisterRequest)(nil), // 3: WorkerRegisterRequest + (*WorkerRegisterResponse)(nil), // 4: WorkerRegisterResponse + (*AssignedAction)(nil), // 5: AssignedAction + (*WorkerListenRequest)(nil), // 6: WorkerListenRequest + (*WorkerUnsubscribeRequest)(nil), // 7: WorkerUnsubscribeRequest + (*WorkerUnsubscribeResponse)(nil), // 8: WorkerUnsubscribeResponse + (*GroupKeyActionEvent)(nil), // 9: GroupKeyActionEvent + (*StepActionEvent)(nil), // 10: StepActionEvent + (*ActionEventResponse)(nil), // 11: ActionEventResponse + (*timestamppb.Timestamp)(nil), // 12: google.protobuf.Timestamp } var file_dispatcher_proto_depIdxs = []int32{ 0, // 0: AssignedAction.actionType:type_name -> ActionType - 10, // 1: ActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp - 1, // 2: ActionEvent.eventType:type_name -> ActionEventType - 2, // 3: Dispatcher.Register:input_type -> WorkerRegisterRequest - 5, // 4: Dispatcher.Listen:input_type -> WorkerListenRequest - 8, // 5: Dispatcher.SendActionEvent:input_type -> ActionEvent - 6, // 6: Dispatcher.Unsubscribe:input_type -> WorkerUnsubscribeRequest - 3, // 7: Dispatcher.Register:output_type -> WorkerRegisterResponse - 4, // 8: Dispatcher.Listen:output_type -> AssignedAction - 9, // 9: Dispatcher.SendActionEvent:output_type -> ActionEventResponse - 7, // 10: Dispatcher.Unsubscribe:output_type -> WorkerUnsubscribeResponse - 7, // [7:11] is the sub-list for method output_type - 3, // [3:7] is the sub-list for method input_type - 3, // [3:3] is the sub-list for extension type_name - 3, // [3:3] is the sub-list for extension extendee - 0, // [0:3] is the sub-list for field type_name + 12, // 1: GroupKeyActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp + 1, // 2: GroupKeyActionEvent.eventType:type_name -> GroupKeyActionEventType + 12, // 3: StepActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp + 2, // 4: StepActionEvent.eventType:type_name -> StepActionEventType + 3, // 5: Dispatcher.Register:input_type -> WorkerRegisterRequest + 6, // 6: Dispatcher.Listen:input_type -> WorkerListenRequest + 10, // 7: Dispatcher.SendStepActionEvent:input_type -> StepActionEvent + 9, // 8: Dispatcher.SendGroupKeyActionEvent:input_type -> GroupKeyActionEvent + 7, // 9: Dispatcher.Unsubscribe:input_type -> WorkerUnsubscribeRequest + 4, // 10: Dispatcher.Register:output_type -> WorkerRegisterResponse + 5, // 11: Dispatcher.Listen:output_type -> AssignedAction + 11, // 12: Dispatcher.SendStepActionEvent:output_type -> ActionEventResponse + 11, // 13: Dispatcher.SendGroupKeyActionEvent:output_type -> ActionEventResponse + 8, // 14: Dispatcher.Unsubscribe:output_type -> WorkerUnsubscribeResponse + 10, // [10:15] is the sub-list for method output_type + 5, // [5:10] is the sub-list for method input_type + 5, // [5:5] is the sub-list for extension type_name + 5, // [5:5] is the sub-list for extension extendee + 0, // [0:5] is the sub-list for field type_name } func init() { file_dispatcher_proto_init() } @@ -936,7 +1158,7 @@ func file_dispatcher_proto_init() { } } file_dispatcher_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ActionEvent); i { + switch v := v.(*GroupKeyActionEvent); i { case 0: return &v.state case 1: @@ -948,6 +1170,18 @@ func file_dispatcher_proto_init() { } } file_dispatcher_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*StepActionEvent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_dispatcher_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ActionEventResponse); i { case 0: return &v.state @@ -965,8 +1199,8 @@ func file_dispatcher_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_dispatcher_proto_rawDesc, - NumEnums: 2, - NumMessages: 8, + NumEnums: 3, + NumMessages: 9, NumExtensions: 0, NumServices: 1, }, diff --git a/internal/services/dispatcher/contracts/dispatcher_grpc.pb.go b/internal/services/dispatcher/contracts/dispatcher_grpc.pb.go index 7cdc0ca2e..02d3d15cc 100644 --- a/internal/services/dispatcher/contracts/dispatcher_grpc.pb.go +++ b/internal/services/dispatcher/contracts/dispatcher_grpc.pb.go @@ -24,7 +24,8 @@ const _ = grpc.SupportPackageIsVersion7 type DispatcherClient interface { Register(ctx context.Context, in *WorkerRegisterRequest, opts ...grpc.CallOption) (*WorkerRegisterResponse, error) Listen(ctx context.Context, in *WorkerListenRequest, opts ...grpc.CallOption) (Dispatcher_ListenClient, error) - SendActionEvent(ctx context.Context, in *ActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) + SendStepActionEvent(ctx context.Context, in *StepActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) + SendGroupKeyActionEvent(ctx context.Context, in *GroupKeyActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) Unsubscribe(ctx context.Context, in *WorkerUnsubscribeRequest, opts ...grpc.CallOption) (*WorkerUnsubscribeResponse, error) } @@ -77,9 +78,18 @@ func (x *dispatcherListenClient) Recv() (*AssignedAction, error) { return m, nil } -func (c *dispatcherClient) SendActionEvent(ctx context.Context, in *ActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) { +func (c *dispatcherClient) SendStepActionEvent(ctx context.Context, in *StepActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) { out := new(ActionEventResponse) - err := c.cc.Invoke(ctx, "/Dispatcher/SendActionEvent", in, out, opts...) + err := c.cc.Invoke(ctx, "/Dispatcher/SendStepActionEvent", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dispatcherClient) SendGroupKeyActionEvent(ctx context.Context, in *GroupKeyActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) { + out := new(ActionEventResponse) + err := c.cc.Invoke(ctx, "/Dispatcher/SendGroupKeyActionEvent", in, out, opts...) if err != nil { return nil, err } @@ -101,7 +111,8 @@ func (c *dispatcherClient) Unsubscribe(ctx context.Context, in *WorkerUnsubscrib type DispatcherServer interface { Register(context.Context, *WorkerRegisterRequest) (*WorkerRegisterResponse, error) Listen(*WorkerListenRequest, Dispatcher_ListenServer) error - SendActionEvent(context.Context, *ActionEvent) (*ActionEventResponse, error) + SendStepActionEvent(context.Context, *StepActionEvent) (*ActionEventResponse, error) + SendGroupKeyActionEvent(context.Context, *GroupKeyActionEvent) (*ActionEventResponse, error) Unsubscribe(context.Context, *WorkerUnsubscribeRequest) (*WorkerUnsubscribeResponse, error) mustEmbedUnimplementedDispatcherServer() } @@ -116,8 +127,11 @@ func (UnimplementedDispatcherServer) Register(context.Context, *WorkerRegisterRe func (UnimplementedDispatcherServer) Listen(*WorkerListenRequest, Dispatcher_ListenServer) error { return status.Errorf(codes.Unimplemented, "method Listen not implemented") } -func (UnimplementedDispatcherServer) SendActionEvent(context.Context, *ActionEvent) (*ActionEventResponse, error) { - return nil, status.Errorf(codes.Unimplemented, "method SendActionEvent not implemented") +func (UnimplementedDispatcherServer) SendStepActionEvent(context.Context, *StepActionEvent) (*ActionEventResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SendStepActionEvent not implemented") +} +func (UnimplementedDispatcherServer) SendGroupKeyActionEvent(context.Context, *GroupKeyActionEvent) (*ActionEventResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method SendGroupKeyActionEvent not implemented") } func (UnimplementedDispatcherServer) Unsubscribe(context.Context, *WorkerUnsubscribeRequest) (*WorkerUnsubscribeResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method Unsubscribe not implemented") @@ -174,20 +188,38 @@ func (x *dispatcherListenServer) Send(m *AssignedAction) error { return x.ServerStream.SendMsg(m) } -func _Dispatcher_SendActionEvent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ActionEvent) +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 { return nil, err } if interceptor == nil { - return srv.(DispatcherServer).SendActionEvent(ctx, in) + return srv.(DispatcherServer).SendStepActionEvent(ctx, in) } info := &grpc.UnaryServerInfo{ Server: srv, - FullMethod: "/Dispatcher/SendActionEvent", + FullMethod: "/Dispatcher/SendStepActionEvent", } handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(DispatcherServer).SendActionEvent(ctx, req.(*ActionEvent)) + return srv.(DispatcherServer).SendStepActionEvent(ctx, req.(*StepActionEvent)) + } + return interceptor(ctx, in, info, handler) +} + +func _Dispatcher_SendGroupKeyActionEvent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GroupKeyActionEvent) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DispatcherServer).SendGroupKeyActionEvent(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/Dispatcher/SendGroupKeyActionEvent", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DispatcherServer).SendGroupKeyActionEvent(ctx, req.(*GroupKeyActionEvent)) } return interceptor(ctx, in, info, handler) } @@ -222,8 +254,12 @@ var Dispatcher_ServiceDesc = grpc.ServiceDesc{ Handler: _Dispatcher_Register_Handler, }, { - MethodName: "SendActionEvent", - Handler: _Dispatcher_SendActionEvent_Handler, + MethodName: "SendStepActionEvent", + Handler: _Dispatcher_SendStepActionEvent_Handler, + }, + { + MethodName: "SendGroupKeyActionEvent", + Handler: _Dispatcher_SendGroupKeyActionEvent_Handler, }, { MethodName: "Unsubscribe", diff --git a/internal/services/dispatcher/dispatcher.go b/internal/services/dispatcher/dispatcher.go index 2c7b48bf7..2c498f86a 100644 --- a/internal/services/dispatcher/dispatcher.go +++ b/internal/services/dispatcher/dispatcher.go @@ -187,6 +187,8 @@ func (d *DispatcherImpl) Start(ctx context.Context) error { func (d *DispatcherImpl) handleTask(ctx context.Context, task *taskqueue.Task) error { switch task.ID { + case "group-key-action-assigned": + return d.handleGroupKeyActionAssignedTask(ctx, task) case "step-run-assigned": return d.handleStepRunAssignedTask(ctx, task) case "step-run-cancelled": @@ -196,7 +198,52 @@ func (d *DispatcherImpl) handleTask(ctx context.Context, task *taskqueue.Task) e return fmt.Errorf("unknown task: %s in queue %s", task.ID, string(task.Queue)) } -// handleJobScheduledTask signals to the connected worker channel that a job should start executing. +func (d *DispatcherImpl) handleGroupKeyActionAssignedTask(ctx context.Context, task *taskqueue.Task) error { + ctx, span := telemetry.NewSpan(ctx, "group-key-action-assigned") + defer span.End() + + payload := tasktypes.GroupKeyActionAssignedTaskPayload{} + metadata := tasktypes.GroupKeyActionAssignedTaskMetadata{} + + err := d.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode dispatcher task payload: %w", err) + } + + err = d.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode dispatcher task metadata: %w", err) + } + + // get the worker for this task + w, err := d.GetWorker(payload.WorkerId) + + if err != nil { + return fmt.Errorf("could not get worker: %w", err) + } + + telemetry.WithAttributes(span, servertel.WorkerId(payload.WorkerId)) + + // load the workflow run from the database + workflowRun, err := d.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId) + + if err != nil { + return fmt.Errorf("could not get workflow run: %w", err) + } + + servertel.WithWorkflowRunModel(span, workflowRun) + + err = w.StartGroupKeyAction(ctx, metadata.TenantId, workflowRun) + + if err != nil { + return fmt.Errorf("could not send group key action to worker: %w", err) + } + + return nil +} + func (d *DispatcherImpl) handleStepRunAssignedTask(ctx context.Context, task *taskqueue.Task) error { ctx, span := telemetry.NewSpan(ctx, "step-run-assigned") defer span.End() @@ -237,7 +284,7 @@ func (d *DispatcherImpl) handleStepRunAssignedTask(ctx context.Context, task *ta err = w.StartStepRun(ctx, metadata.TenantId, stepRun) if err != nil { - return fmt.Errorf("could not send job to worker: %w", err) + return fmt.Errorf("could not send step action to worker: %w", err) } return nil diff --git a/internal/services/dispatcher/server.go b/internal/services/dispatcher/server.go index 6a709ca18..111fb7744 100644 --- a/internal/services/dispatcher/server.go +++ b/internal/services/dispatcher/server.go @@ -5,6 +5,8 @@ import ( "fmt" "time" + "github.com/steebchen/prisma-client-go/runtime/types" + "github.com/hatchet-dev/hatchet/internal/datautils" "github.com/hatchet-dev/hatchet/internal/repository" "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" @@ -70,6 +72,67 @@ func (worker *subscribedWorker) StartStepRun( }) } +func (worker *subscribedWorker) StartGroupKeyAction( + ctx context.Context, + tenantId string, + workflowRun *db.WorkflowRunModel, +) error { + ctx, span := telemetry.NewSpan(ctx, "start-group-key-action") + defer span.End() + + inputBytes := []byte{} + + concurrency, ok := workflowRun.WorkflowVersion().Concurrency() + + if !ok { + return fmt.Errorf("could not get concurrency for workflow version %s", workflowRun.WorkflowVersionID) + } + + concurrencyFn, ok := concurrency.GetConcurrencyGroup() + + if !ok { + return fmt.Errorf("could not get concurrency group for workflow version %s", workflowRun.WorkflowVersionID) + } + + // get the input from the workflow run + triggeredBy, ok := workflowRun.TriggeredBy() + + if !ok { + return fmt.Errorf("could not get triggered by from workflow run %s", workflowRun.ID) + } + + var inputData types.JSON + + if event, ok := triggeredBy.Event(); ok { + inputData, _ = event.Data() + } else if schedule, ok := triggeredBy.Scheduled(); ok { + inputData, _ = schedule.Input() + } else if cron, ok := triggeredBy.Cron(); ok { + inputData, _ = cron.Input() + } + + inputBytes, err := inputData.MarshalJSON() + + if err != nil { + return err + } + + getGroupKeyRun, ok := workflowRun.GetGroupKeyRun() + + if !ok { + return fmt.Errorf("could not get get group key run for workflow run %s", workflowRun.ID) + } + + return worker.stream.Send(&contracts.AssignedAction{ + TenantId: tenantId, + WorkflowRunId: workflowRun.ID, + GetGroupKeyRunId: getGroupKeyRun.ID, + ActionType: contracts.ActionType_START_GET_GROUP_KEY, + ActionId: concurrencyFn.ActionID, + ActionPayload: string(inputBytes), + }) +} + func (worker *subscribedWorker) CancelStepRun( ctx context.Context, tenantId string, @@ -213,21 +276,32 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c } } -func (s *DispatcherImpl) SendActionEvent(ctx context.Context, request *contracts.ActionEvent) (*contracts.ActionEventResponse, error) { - // TODO: auth checks to make sure the worker is allowed to send an action event for this tenant - +func (s *DispatcherImpl) SendStepActionEvent(ctx context.Context, request *contracts.StepActionEvent) (*contracts.ActionEventResponse, error) { switch request.EventType { - case contracts.ActionEventType_STEP_EVENT_TYPE_STARTED: + case contracts.StepActionEventType_STEP_EVENT_TYPE_STARTED: return s.handleStepRunStarted(ctx, request) - case contracts.ActionEventType_STEP_EVENT_TYPE_COMPLETED: + case contracts.StepActionEventType_STEP_EVENT_TYPE_COMPLETED: return s.handleStepRunCompleted(ctx, request) - case contracts.ActionEventType_STEP_EVENT_TYPE_FAILED: + case contracts.StepActionEventType_STEP_EVENT_TYPE_FAILED: return s.handleStepRunFailed(ctx, request) } return nil, fmt.Errorf("unknown event type %s", request.EventType) } +func (s *DispatcherImpl) SendGroupKeyActionEvent(ctx context.Context, request *contracts.GroupKeyActionEvent) (*contracts.ActionEventResponse, error) { + switch request.EventType { + case contracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_STARTED: + return s.handleGetGroupKeyRunStarted(ctx, request) + case contracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_COMPLETED: + return s.handleGetGroupKeyRunCompleted(ctx, request) + case contracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_FAILED: + return s.handleGetGroupKeyRunFailed(ctx, request) + } + + return nil, fmt.Errorf("unknown event type %s", request.EventType) +} + func (s *DispatcherImpl) Unsubscribe(ctx context.Context, request *contracts.WorkerUnsubscribeRequest) (*contracts.WorkerUnsubscribeResponse, error) { tenant := ctx.Value("tenant").(*db.TenantModel) @@ -246,7 +320,7 @@ func (s *DispatcherImpl) Unsubscribe(ctx context.Context, request *contracts.Wor }, nil } -func (s *DispatcherImpl) handleStepRunStarted(ctx context.Context, request *contracts.ActionEvent) (*contracts.ActionEventResponse, error) { +func (s *DispatcherImpl) handleStepRunStarted(ctx context.Context, request *contracts.StepActionEvent) (*contracts.ActionEventResponse, error) { tenant := ctx.Value("tenant").(*db.TenantModel) s.l.Debug().Msgf("Received step started event for step run %s", request.StepRunId) @@ -280,7 +354,7 @@ func (s *DispatcherImpl) handleStepRunStarted(ctx context.Context, request *cont }, nil } -func (s *DispatcherImpl) handleStepRunCompleted(ctx context.Context, request *contracts.ActionEvent) (*contracts.ActionEventResponse, error) { +func (s *DispatcherImpl) handleStepRunCompleted(ctx context.Context, request *contracts.StepActionEvent) (*contracts.ActionEventResponse, error) { tenant := ctx.Value("tenant").(*db.TenantModel) s.l.Debug().Msgf("Received step completed event for step run %s", request.StepRunId) @@ -315,7 +389,7 @@ func (s *DispatcherImpl) handleStepRunCompleted(ctx context.Context, request *co }, nil } -func (s *DispatcherImpl) handleStepRunFailed(ctx context.Context, request *contracts.ActionEvent) (*contracts.ActionEventResponse, error) { +func (s *DispatcherImpl) handleStepRunFailed(ctx context.Context, request *contracts.StepActionEvent) (*contracts.ActionEventResponse, error) { tenant := ctx.Value("tenant").(*db.TenantModel) s.l.Debug().Msgf("Received step failed event for step run %s", request.StepRunId) @@ -349,3 +423,107 @@ func (s *DispatcherImpl) handleStepRunFailed(ctx context.Context, request *contr WorkerId: request.WorkerId, }, nil } + +func (s *DispatcherImpl) handleGetGroupKeyRunStarted(ctx context.Context, request *contracts.GroupKeyActionEvent) (*contracts.ActionEventResponse, error) { + tenant := ctx.Value("tenant").(*db.TenantModel) + + s.l.Debug().Msgf("Received step started event for step run %s", request.GetGroupKeyRunId) + + startedAt := request.EventTimestamp.AsTime() + + payload, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunStartedTaskPayload{ + GetGroupKeyRunId: request.GetGroupKeyRunId, + StartedAt: startedAt.Format(time.RFC3339), + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunStartedTaskMetadata{ + TenantId: tenant.ID, + }) + + // send the event to the jobs queue + err := s.tq.AddTask(ctx, taskqueue.WORKFLOW_PROCESSING_QUEUE, &taskqueue.Task{ + ID: "get-group-key-run-started", + Queue: taskqueue.WORKFLOW_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + }) + + if err != nil { + return nil, err + } + + return &contracts.ActionEventResponse{ + TenantId: tenant.ID, + WorkerId: request.WorkerId, + }, nil +} + +func (s *DispatcherImpl) handleGetGroupKeyRunCompleted(ctx context.Context, request *contracts.GroupKeyActionEvent) (*contracts.ActionEventResponse, error) { + tenant := ctx.Value("tenant").(*db.TenantModel) + + s.l.Debug().Msgf("Received step completed event for step run %s", request.GetGroupKeyRunId) + + finishedAt := request.EventTimestamp.AsTime() + + payload, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunFinishedTaskPayload{ + GetGroupKeyRunId: request.GetGroupKeyRunId, + FinishedAt: finishedAt.Format(time.RFC3339), + GroupKey: request.EventPayload, + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunFinishedTaskMetadata{ + TenantId: tenant.ID, + }) + + // send the event to the jobs queue + err := s.tq.AddTask(ctx, taskqueue.WORKFLOW_PROCESSING_QUEUE, &taskqueue.Task{ + ID: "get-group-key-run-finished", + Queue: taskqueue.WORKFLOW_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + }) + + if err != nil { + return nil, err + } + + return &contracts.ActionEventResponse{ + TenantId: tenant.ID, + WorkerId: request.WorkerId, + }, nil +} + +func (s *DispatcherImpl) handleGetGroupKeyRunFailed(ctx context.Context, request *contracts.GroupKeyActionEvent) (*contracts.ActionEventResponse, error) { + tenant := ctx.Value("tenant").(*db.TenantModel) + + s.l.Debug().Msgf("Received step failed event for step run %s", request.GetGroupKeyRunId) + + failedAt := request.EventTimestamp.AsTime() + + payload, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunFailedTaskPayload{ + GetGroupKeyRunId: request.GetGroupKeyRunId, + FailedAt: failedAt.Format(time.RFC3339), + Error: request.EventPayload, + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunFailedTaskMetadata{ + TenantId: tenant.ID, + }) + + // send the event to the jobs queue + err := s.tq.AddTask(ctx, taskqueue.WORKFLOW_PROCESSING_QUEUE, &taskqueue.Task{ + ID: "get-group-key-run-failed", + Queue: taskqueue.WORKFLOW_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + }) + + if err != nil { + return nil, err + } + + return &contracts.ActionEventResponse{ + TenantId: tenant.ID, + WorkerId: request.WorkerId, + }, nil +} diff --git a/internal/services/shared/tasktypes/group_key.go b/internal/services/shared/tasktypes/group_key.go new file mode 100644 index 000000000..8d89f8f58 --- /dev/null +++ b/internal/services/shared/tasktypes/group_key.go @@ -0,0 +1,80 @@ +package tasktypes + +import ( + "github.com/hatchet-dev/hatchet/internal/datautils" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" + "github.com/hatchet-dev/hatchet/internal/taskqueue" +) + +type GroupKeyActionAssignedTaskPayload struct { + WorkflowRunId string `json:"workflow_run_id" validate:"required,uuid"` + WorkerId string `json:"worker_id" validate:"required,uuid"` +} + +type GroupKeyActionAssignedTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` + DispatcherId string `json:"dispatcher_id" validate:"required,uuid"` +} + +type GroupKeyActionRequeueTaskPayload struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + +type GroupKeyActionRequeueTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + +func TenantToGroupKeyActionRequeueTask(tenant db.TenantModel) *taskqueue.Task { + payload, _ := datautils.ToJSONMap(GroupKeyActionRequeueTaskPayload{ + TenantId: tenant.ID, + }) + + metadata, _ := datautils.ToJSONMap(GroupKeyActionRequeueTaskMetadata{ + TenantId: tenant.ID, + }) + + return &taskqueue.Task{ + ID: "group-key-action-requeue-ticker", + Queue: taskqueue.WORKFLOW_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + } +} + +type GetGroupKeyRunStartedTaskPayload struct { + GetGroupKeyRunId string `json:"get_group_key_run_id" validate:"required,uuid"` + StartedAt string `json:"started_at" validate:"required"` +} + +type GetGroupKeyRunStartedTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + +type GetGroupKeyRunFinishedTaskPayload struct { + GetGroupKeyRunId string `json:"get_group_key_run_id" validate:"required,uuid"` + FinishedAt string `json:"finished_at" validate:"required"` + GroupKey string `json:"group_key"` +} + +type GetGroupKeyRunFinishedTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + +type GetGroupKeyRunFailedTaskPayload struct { + GetGroupKeyRunId string `json:"get_group_key_run_id" validate:"required,uuid"` + FailedAt string `json:"failed_at" validate:"required"` + Error string `json:"error" validate:"required"` +} + +type GetGroupKeyRunFailedTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + +type GetGroupKeyRunTimedOutTaskPayload struct { + GetGroupKeyRunId string `json:"get_group_key_run_id" validate:"required,uuid"` + WorkflowRunId string `json:"workflow_run_id" validate:"required,uuid"` +} + +type GetGroupKeyRunTimedOutTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} diff --git a/internal/services/shared/tasktypes/step.go b/internal/services/shared/tasktypes/step.go index 388b715eb..9eccb6e38 100644 --- a/internal/services/shared/tasktypes/step.go +++ b/internal/services/shared/tasktypes/step.go @@ -50,6 +50,15 @@ type StepRunRequeueTaskMetadata struct { TenantId string `json:"tenant_id" validate:"required,uuid"` } +type StepRunNotifyCancelTaskPayload struct { + StepRunId string `json:"step_run_id" validate:"required,uuid"` + CancelledReason string `json:"cancelled_reason" validate:"required"` +} + +type StepRunNotifyCancelTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + type StepRunStartedTaskPayload struct { StepRunId string `json:"step_run_id" validate:"required,uuid"` StartedAt string `json:"started_at" validate:"required"` diff --git a/internal/services/shared/tasktypes/ticker.go b/internal/services/shared/tasktypes/ticker.go index 83cbd9081..ed133c40f 100644 --- a/internal/services/shared/tasktypes/ticker.go +++ b/internal/services/shared/tasktypes/ticker.go @@ -18,6 +18,24 @@ type CancelStepRunTimeoutTaskMetadata struct { TenantId string `json:"tenant_id" validate:"required,uuid"` } +type ScheduleGetGroupKeyRunTimeoutTaskPayload struct { + GetGroupKeyRunId string `json:"get_group_key_run_id" validate:"required,uuid"` + WorkflowRunId string `json:"workflow_run_id" validate:"required,uuid"` + TimeoutAt string `json:"timeout_at" validate:"required"` +} + +type ScheduleGetGroupKeyRunTimeoutTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + +type CancelGetGroupKeyRunTimeoutTaskPayload struct { + GetGroupKeyRunId string `json:"get_group_key_run_id" validate:"required,uuid"` +} + +type CancelGetGroupKeyRunTimeoutTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` +} + type RemoveTickerTaskPayload struct { TickerId string `json:"ticker_id" validate:"required,uuid"` } diff --git a/internal/services/shared/tasktypes/workflow.go b/internal/services/shared/tasktypes/workflow.go new file mode 100644 index 000000000..1f2cbe6da --- /dev/null +++ b/internal/services/shared/tasktypes/workflow.go @@ -0,0 +1,34 @@ +package tasktypes + +import ( + "github.com/hatchet-dev/hatchet/internal/datautils" + "github.com/hatchet-dev/hatchet/internal/repository/prisma/db" + "github.com/hatchet-dev/hatchet/internal/taskqueue" +) + +type WorkflowRunQueuedTaskPayload struct { + WorkflowRunId string `json:"workflow_run_id" validate:"required,uuid"` +} + +type WorkflowRunQueuedTaskMetadata struct { + TenantId string `json:"tenant_id" validate:"required,uuid"` + WorkflowVersionId string `json:"workflow_version_id" validate:"required,uuid"` +} + +func WorkflowRunQueuedToTask(workflowRun *db.WorkflowRunModel) *taskqueue.Task { + payload, _ := datautils.ToJSONMap(WorkflowRunQueuedTaskPayload{ + WorkflowRunId: workflowRun.ID, + }) + + metadata, _ := datautils.ToJSONMap(WorkflowRunQueuedTaskMetadata{ + WorkflowVersionId: workflowRun.WorkflowVersionID, + TenantId: workflowRun.TenantID, + }) + + return &taskqueue.Task{ + ID: "workflow-run-queued", + Queue: taskqueue.WORKFLOW_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + } +} diff --git a/internal/services/ticker/get_group_key_run_timeout.go b/internal/services/ticker/get_group_key_run_timeout.go new file mode 100644 index 000000000..36c1f4a0e --- /dev/null +++ b/internal/services/ticker/get_group_key_run_timeout.go @@ -0,0 +1,142 @@ +package ticker + +import ( + "context" + "fmt" + "time" + + "github.com/hatchet-dev/hatchet/internal/datautils" + "github.com/hatchet-dev/hatchet/internal/services/shared/tasktypes" + "github.com/hatchet-dev/hatchet/internal/taskqueue" +) + +func (t *TickerImpl) handleScheduleGetGroupKeyRunTimeout(ctx context.Context, task *taskqueue.Task) error { + t.l.Debug().Msg("ticker: scheduling get group key run timeout") + + payload := tasktypes.ScheduleGetGroupKeyRunTimeoutTaskPayload{} + metadata := tasktypes.ScheduleGetGroupKeyRunTimeoutTaskMetadata{} + + err := t.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode ticker task payload: %w", err) + } + + err = t.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode ticker task metadata: %w", err) + } + + timeoutAt, err := time.Parse(time.RFC3339, payload.TimeoutAt) + + if err != nil { + return fmt.Errorf("could not parse timeout at: %w", err) + } + + // schedule the timeout + childCtx, cancel := context.WithDeadline(context.Background(), timeoutAt) + + go func() { + <-childCtx.Done() + t.runGetGroupKeyRunTimeout(metadata.TenantId, payload.WorkflowRunId, payload.GetGroupKeyRunId) + }() + + // store the schedule in the get group key run map + t.getGroupKeyRuns.Store(payload.GetGroupKeyRunId, &timeoutCtx{ + ctx: childCtx, + cancel: cancel, + }) + + return nil +} + +func (t *TickerImpl) handleCancelGetGroupKeyRunTimeout(ctx context.Context, task *taskqueue.Task) error { + t.l.Debug().Msg("ticker: canceling get group key run timeout") + + payload := tasktypes.CancelGetGroupKeyRunTimeoutTaskPayload{} + metadata := tasktypes.CancelGetGroupKeyRunTimeoutTaskMetadata{} + + err := t.dv.DecodeAndValidate(task.Payload, &payload) + + if err != nil { + return fmt.Errorf("could not decode ticker task payload: %w", err) + } + + err = t.dv.DecodeAndValidate(task.Metadata, &metadata) + + if err != nil { + return fmt.Errorf("could not decode ticker task metadata: %w", err) + } + + // get the cancel function + childTimeoutCtxVal, ok := t.getGroupKeyRuns.Load(payload.GetGroupKeyRunId) + + if !ok { + return nil + } + + // cancel the timeout + childTimeoutCtx := childTimeoutCtxVal.(*timeoutCtx) + + childTimeoutCtx.ctx = context.WithValue(childTimeoutCtx.ctx, "cancelled", true) + + childTimeoutCtx.cancel() + + return nil +} + +func (t *TickerImpl) runGetGroupKeyRunTimeout(tenantId, workflowRunId, getGroupKeyRunId string) { + defer t.getGroupKeyRuns.Delete(getGroupKeyRunId) + + childTimeoutCtxVal, ok := t.getGroupKeyRuns.Load(getGroupKeyRunId) + + if !ok { + t.l.Debug().Msgf("ticker: could not find get group key run %s", getGroupKeyRunId) + return + } + + childTimeoutCtx := childTimeoutCtxVal.(*timeoutCtx) + + var isCancelled bool + + if cancelledVal := childTimeoutCtx.ctx.Value("cancelled"); cancelledVal != nil { + isCancelled = cancelledVal.(bool) + } + + if isCancelled { + t.l.Debug().Msgf("ticker: timeout of %s was cancelled", getGroupKeyRunId) + return + } + + t.l.Debug().Msgf("ticker: get group key run %s timed out", getGroupKeyRunId) + + // signal the jobs controller that the group key run timed out + err := t.tq.AddTask( + context.Background(), + taskqueue.JOB_PROCESSING_QUEUE, + taskGetGroupKeyRunTimedOut(tenantId, workflowRunId, getGroupKeyRunId), + ) + + if err != nil { + t.l.Err(err).Msg("could not add get group key run requeue task") + } +} + +func taskGetGroupKeyRunTimedOut(tenantId, workflowRunId, getGroupKeyRunId string) *taskqueue.Task { + payload, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunTimedOutTaskPayload{ + GetGroupKeyRunId: getGroupKeyRunId, + WorkflowRunId: workflowRunId, + }) + + metadata, _ := datautils.ToJSONMap(tasktypes.GetGroupKeyRunTimedOutTaskMetadata{ + TenantId: tenantId, + }) + + return &taskqueue.Task{ + ID: "get-group-key-run-timed-out", + Queue: taskqueue.WORKFLOW_PROCESSING_QUEUE, + Payload: payload, + Metadata: metadata, + } +} diff --git a/internal/services/ticker/ticker.go b/internal/services/ticker/ticker.go index 350d7db51..68c0e5bd0 100644 --- a/internal/services/ticker/ticker.go +++ b/internal/services/ticker/ticker.go @@ -31,6 +31,7 @@ type TickerImpl struct { scheduledWorkflows sync.Map jobRuns sync.Map stepRuns sync.Map + getGroupKeyRuns sync.Map dv datautils.DataDecoderValidator @@ -144,6 +145,17 @@ func (t *TickerImpl) Start(ctx context.Context) error { return fmt.Errorf("could not schedule step run requeue: %w", err) } + _, err = t.s.NewJob( + gocron.DurationJob(time.Second*5), + gocron.NewTask( + t.runGetGroupKeyRunRequeue(ctx), + ), + ) + + if err != nil { + return fmt.Errorf("could not schedule get group key run requeue: %w", err) + } + _, err = t.s.NewJob( gocron.DurationJob(time.Second*5), gocron.NewTask( @@ -198,6 +210,10 @@ func (t *TickerImpl) handleTask(ctx context.Context, task *taskqueue.Task) error return t.handleScheduleStepRunTimeout(ctx, task) case "cancel-step-run-timeout": return t.handleCancelStepRunTimeout(ctx, task) + case "schedule-get-group-key-run-timeout": + return t.handleScheduleGetGroupKeyRunTimeout(ctx, task) + case "cancel-get-group-key-run-timeout": + return t.handleCancelGetGroupKeyRunTimeout(ctx, task) case "schedule-job-run-timeout": return t.handleScheduleJobRunTimeout(ctx, task) case "cancel-job-run-timeout": @@ -247,6 +263,34 @@ func (t *TickerImpl) runStepRunRequeue(ctx context.Context) func() { } } +func (t *TickerImpl) runGetGroupKeyRunRequeue(ctx context.Context) func() { + return func() { + t.l.Debug().Msgf("ticker: checking get group key run requeue") + + // list all tenants + tenants, err := t.repo.Tenant().ListTenants() + + if err != nil { + t.l.Err(err).Msg("could not list tenants") + return + } + + for i := range tenants { + t.l.Debug().Msgf("adding get group key run requeue task for tenant %s", tenants[i].ID) + + err := t.tq.AddTask( + ctx, + taskqueue.WORKFLOW_PROCESSING_QUEUE, + tasktypes.TenantToGroupKeyActionRequeueTask(tenants[i]), + ) + + if err != nil { + t.l.Err(err).Msg("could not add get group key run requeue task") + } + } + } +} + func (t *TickerImpl) runUpdateHeartbeat(ctx context.Context) func() { return func() { t.l.Debug().Msgf("ticker: updating heartbeat") diff --git a/internal/taskqueue/rabbitmq/rabbitmq.go b/internal/taskqueue/rabbitmq/rabbitmq.go index 3a6076419..7fb311c80 100644 --- a/internal/taskqueue/rabbitmq/rabbitmq.go +++ b/internal/taskqueue/rabbitmq/rabbitmq.go @@ -86,6 +86,7 @@ func New(ctx context.Context, fs ...TaskQueueImplOpt) *TaskQueueImpl { t.initQueue(sub, string(taskqueue.EVENT_PROCESSING_QUEUE)) t.initQueue(sub, string(taskqueue.JOB_PROCESSING_QUEUE)) + t.initQueue(sub, string(taskqueue.WORKFLOW_PROCESSING_QUEUE)) t.initQueue(sub, string(taskqueue.SCHEDULING_QUEUE)) break } @@ -164,76 +165,6 @@ func (t *TaskQueueImpl) publish() { } } -// func (t *TaskQueueImpl) publish() { -// for session := range t.sessions { -// var ( -// running bool -// reading = t.tasks -// pending = make(chan []byte, 1) -// confirm = make(chan amqp.Confirmation, 1) -// ) - -// pub := <-session - -// // publisher confirms for this channel/connection -// if err := pub.Channel.Confirm(false); err != nil { -// t.l.Info().Msgf("publisher confirms not supported") -// close(confirm) // confirms not supported, simulate by always nacking -// } else { -// pub.NotifyPublish(confirm) -// } - -// ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) -// defer cancel() - -// Publish: -// for { -// var ( -// body []byte -// task *taskqueue.Task -// ) - -// select { -// case confirmed, ok := <-confirm: -// if !ok { -// break Publish -// } -// if !confirmed.Ack { -// t.l.Info().Msgf("nack message %d", confirmed.DeliveryTag) -// } -// reading = t.tasks - -// case body = <-pending: -// err := pub.PublishWithContext(ctx, "", string(task.Queue), false, false, amqp.Publishing{ -// Body: body, -// }) -// // Retry failed delivery on the next session -// if err != nil { -// pending <- body -// pub.Channel.Close() -// break Publish -// } -// case task, running = <-reading: -// body, err := json.Marshal(task) - -// if err != nil { -// t.l.Error().Msgf("error marshaling task queue: %v", err) -// return -// } - -// // all messages consumed -// if !running { -// return -// } - -// // work on pending delivery until ack'd -// pending <- body -// reading = nil -// } -// } -// } -// } - func (t *TaskQueueImpl) subscribe(ctx context.Context, subId, queue string, sessions chan chan session, messages chan *taskqueue.Task, tasks chan<- *taskqueue.Task) { sessionCount := 0 diff --git a/internal/taskqueue/taskqueue.go b/internal/taskqueue/taskqueue.go index aa2425ece..c0c76cbbe 100644 --- a/internal/taskqueue/taskqueue.go +++ b/internal/taskqueue/taskqueue.go @@ -7,10 +7,11 @@ import ( type QueueType string const ( - EVENT_PROCESSING_QUEUE QueueType = "event_processing_queue" - JOB_PROCESSING_QUEUE QueueType = "job_processing_queue" - DISPATCHER_POOL_QUEUE QueueType = "dispatcher_pool_queue" - SCHEDULING_QUEUE QueueType = "scheduling_queue" + EVENT_PROCESSING_QUEUE QueueType = "event_processing_queue" + JOB_PROCESSING_QUEUE QueueType = "job_processing_queue" + WORKFLOW_PROCESSING_QUEUE QueueType = "workflow_processing_queue" + DISPATCHER_POOL_QUEUE QueueType = "dispatcher_pool_queue" + SCHEDULING_QUEUE QueueType = "scheduling_queue" ) func QueueTypeFromDispatcherID(d string) QueueType { diff --git a/pkg/client/admin.go b/pkg/client/admin.go index c00ed5ab7..5995d058e 100644 --- a/pkg/client/admin.go +++ b/pkg/client/admin.go @@ -150,6 +150,24 @@ func (a *adminClientImpl) getPutRequest(workflow *types.Workflow) (*admincontrac CronTriggers: workflow.Triggers.Cron, } + if workflow.Concurrency != nil { + opts.Concurrency = &admincontracts.WorkflowConcurrencyOpts{ + Action: workflow.Concurrency.ActionID, + } + + switch workflow.Concurrency.LimitStrategy { + case types.CancelInProgress: + opts.Concurrency.LimitStrategy = admincontracts.ConcurrencyLimitStrategy_CANCEL_IN_PROGRESS + default: + opts.Concurrency.LimitStrategy = admincontracts.ConcurrencyLimitStrategy_CANCEL_IN_PROGRESS + } + + // TODO: should be a pointer because users might want to set maxRuns temporarily for disabling + if workflow.Concurrency.MaxRuns != 0 { + opts.Concurrency.MaxRuns = workflow.Concurrency.MaxRuns + } + } + jobOpts := make([]*admincontracts.CreateWorkflowJobOpts, 0) for jobName, job := range workflow.Jobs { diff --git a/pkg/client/dispatcher.go b/pkg/client/dispatcher.go index e6e5b226d..46854457c 100644 --- a/pkg/client/dispatcher.go +++ b/pkg/client/dispatcher.go @@ -22,7 +22,9 @@ import ( type DispatcherClient interface { GetActionListener(ctx context.Context, req *GetActionListenerRequest) (WorkerActionListener, error) - SendActionEvent(ctx context.Context, in *ActionEvent) (*ActionEventResponse, error) + SendStepActionEvent(ctx context.Context, in *ActionEvent) (*ActionEventResponse, error) + + SendGroupKeyActionEvent(ctx context.Context, in *ActionEvent) (*ActionEventResponse, error) } const ( @@ -43,8 +45,9 @@ type ActionPayload func(target interface{}) error type ActionType string const ( - ActionTypeStartStepRun ActionType = "START_STEP_RUN" - ActionTypeCancelStepRun ActionType = "CANCEL_STEP_RUN" + ActionTypeStartStepRun ActionType = "START_STEP_RUN" + ActionTypeCancelStepRun ActionType = "CANCEL_STEP_RUN" + ActionTypeStartGetGroupKey ActionType = "START_GET_GROUP_KEY" ) type Action struct { @@ -54,6 +57,12 @@ type Action struct { // the tenant id TenantId string + // the workflow run id + WorkflowRunId string + + // the get group key run id + GetGroupKeyRunId string + // the job id JobId string @@ -245,6 +254,8 @@ func (a *actionListenerImpl) Actions(ctx context.Context, errCh chan<- error) (< actionType = ActionTypeStartStepRun case dispatchercontracts.ActionType_CANCEL_STEP_RUN: actionType = ActionTypeCancelStepRun + case dispatchercontracts.ActionType_START_GET_GROUP_KEY: + actionType = ActionTypeStartGetGroupKey default: a.l.Error().Msgf("Unknown action type: %s", assignedAction.ActionType) continue @@ -255,21 +266,22 @@ func (a *actionListenerImpl) Actions(ctx context.Context, errCh chan<- error) (< unquoted, err := strconv.Unquote(assignedAction.ActionPayload) if err != nil { - a.l.Err(err).Msgf("Error unquoting payload for action: %s", assignedAction.ActionType) - continue + unquoted = assignedAction.ActionPayload } ch <- &Action{ - TenantId: assignedAction.TenantId, - WorkerId: a.workerId, - JobId: assignedAction.JobId, - JobName: assignedAction.JobName, - JobRunId: assignedAction.JobRunId, - StepId: assignedAction.StepId, - StepRunId: assignedAction.StepRunId, - ActionId: assignedAction.ActionId, - ActionType: actionType, - ActionPayload: []byte(unquoted), + TenantId: assignedAction.TenantId, + WorkflowRunId: assignedAction.WorkflowRunId, + GetGroupKeyRunId: assignedAction.GetGroupKeyRunId, + WorkerId: a.workerId, + JobId: assignedAction.JobId, + JobName: assignedAction.JobName, + JobRunId: assignedAction.JobRunId, + StepId: assignedAction.StepId, + StepRunId: assignedAction.StepRunId, + ActionId: assignedAction.ActionId, + ActionType: actionType, + ActionPayload: []byte(unquoted), } } }() @@ -320,7 +332,7 @@ func (d *dispatcherClientImpl) GetActionListener(ctx context.Context, req *GetAc return d.newActionListener(ctx, req) } -func (d *dispatcherClientImpl) SendActionEvent(ctx context.Context, in *ActionEvent) (*ActionEventResponse, error) { +func (d *dispatcherClientImpl) SendStepActionEvent(ctx context.Context, in *ActionEvent) (*ActionEventResponse, error) { // validate the request if err := d.v.Validate(in); err != nil { return nil, err @@ -332,20 +344,20 @@ func (d *dispatcherClientImpl) SendActionEvent(ctx context.Context, in *ActionEv return nil, err } - var actionEventType dispatchercontracts.ActionEventType + var actionEventType dispatchercontracts.StepActionEventType switch in.EventType { case ActionEventTypeStarted: - actionEventType = dispatchercontracts.ActionEventType_STEP_EVENT_TYPE_STARTED + actionEventType = dispatchercontracts.StepActionEventType_STEP_EVENT_TYPE_STARTED case ActionEventTypeCompleted: - actionEventType = dispatchercontracts.ActionEventType_STEP_EVENT_TYPE_COMPLETED + actionEventType = dispatchercontracts.StepActionEventType_STEP_EVENT_TYPE_COMPLETED case ActionEventTypeFailed: - actionEventType = dispatchercontracts.ActionEventType_STEP_EVENT_TYPE_FAILED + actionEventType = dispatchercontracts.StepActionEventType_STEP_EVENT_TYPE_FAILED default: - actionEventType = dispatchercontracts.ActionEventType_STEP_EVENT_TYPE_UNKNOWN + actionEventType = dispatchercontracts.StepActionEventType_STEP_EVENT_TYPE_UNKNOWN } - resp, err := d.client.SendActionEvent(d.ctx.newContext(ctx), &dispatchercontracts.ActionEvent{ + resp, err := d.client.SendStepActionEvent(d.ctx.newContext(ctx), &dispatchercontracts.StepActionEvent{ WorkerId: in.WorkerId, JobId: in.JobId, JobRunId: in.JobRunId, @@ -366,3 +378,48 @@ func (d *dispatcherClientImpl) SendActionEvent(ctx context.Context, in *ActionEv WorkerId: resp.WorkerId, }, nil } + +func (d *dispatcherClientImpl) SendGroupKeyActionEvent(ctx context.Context, in *ActionEvent) (*ActionEventResponse, error) { + // validate the request + if err := d.v.Validate(in); err != nil { + return nil, err + } + + payloadBytes, err := json.Marshal(in.EventPayload) + + if err != nil { + return nil, err + } + + var actionEventType dispatchercontracts.GroupKeyActionEventType + + switch in.EventType { + case ActionEventTypeStarted: + actionEventType = dispatchercontracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_STARTED + case ActionEventTypeCompleted: + actionEventType = dispatchercontracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_COMPLETED + case ActionEventTypeFailed: + actionEventType = dispatchercontracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_FAILED + default: + actionEventType = dispatchercontracts.GroupKeyActionEventType_GROUP_KEY_EVENT_TYPE_UNKNOWN + } + + resp, err := d.client.SendGroupKeyActionEvent(d.ctx.newContext(ctx), &dispatchercontracts.GroupKeyActionEvent{ + WorkerId: in.WorkerId, + WorkflowRunId: in.WorkflowRunId, + GetGroupKeyRunId: in.GetGroupKeyRunId, + ActionId: in.ActionId, + EventTimestamp: timestamppb.New(*in.EventTimestamp), + EventType: actionEventType, + EventPayload: string(payloadBytes), + }) + + if err != nil { + return nil, err + } + + return &ActionEventResponse{ + TenantId: resp.TenantId, + WorkerId: resp.WorkerId, + }, nil +} diff --git a/pkg/client/types/file.go b/pkg/client/types/file.go index 05cc65502..11cfe7b2e 100644 --- a/pkg/client/types/file.go +++ b/pkg/client/types/file.go @@ -12,6 +12,8 @@ import ( type Workflow struct { Name string `yaml:"name,omitempty"` + Concurrency *WorkflowConcurrency `yaml:"concurrency,omitempty"` + Version string `yaml:"version,omitempty"` Description string `yaml:"description,omitempty"` @@ -21,6 +23,20 @@ type Workflow struct { Jobs map[string]WorkflowJob `yaml:"jobs"` } +type WorkflowConcurrencyLimitStrategy string + +const ( + CancelInProgress WorkflowConcurrencyLimitStrategy = "CANCEL_IN_PROGRESS" +) + +type WorkflowConcurrency struct { + ActionID string `yaml:"action,omitempty"` + + MaxRuns int32 `yaml:"maxRuns,omitempty"` + + LimitStrategy WorkflowConcurrencyLimitStrategy `yaml:"limitStrategy,omitempty"` +} + type WorkflowTriggers struct { Events []string `yaml:"events,omitempty"` Cron []string `yaml:"crons,omitempty"` diff --git a/pkg/worker/service.go b/pkg/worker/service.go index 8677532ad..8f53099e6 100644 --- a/pkg/worker/service.go +++ b/pkg/worker/service.go @@ -42,7 +42,14 @@ func (s *Service) On(t triggerConverter, workflow workflowConverter) error { return err } - err = s.worker.registerAction(s.Name, parsedAction.Verb, fn) + if parsedAction.Service != s.Name { + // check that it's concurrency, otherwise throw error + if parsedAction.Service != "concurrency" { + return fmt.Errorf("action %s does not belong to service %s", actionId, s.Name) + } + } + + err = s.worker.registerAction(parsedAction.Service, parsedAction.Verb, fn) if err != nil { return err diff --git a/pkg/worker/worker.go b/pkg/worker/worker.go index fba762fdd..2f44d6a44 100644 --- a/pkg/worker/worker.go +++ b/pkg/worker/worker.go @@ -30,15 +30,18 @@ type Action interface { MethodFn() any + ConcurrencyFn() GetWorkflowConcurrencyGroupFn + // Service returns the service that the action belongs to Service() string } type actionImpl struct { - name string - run actionFunc - method any - service string + name string + run actionFunc + runConcurrencyAction GetWorkflowConcurrencyGroupFn + method any + service string } func (j *actionImpl) Name() string { @@ -53,6 +56,10 @@ func (j *actionImpl) MethodFn() any { return j.method } +func (j *actionImpl) ConcurrencyFn() GetWorkflowConcurrencyGroupFn { + return j.runConcurrencyAction +} + func (j *actionImpl) Service() string { return j.service } @@ -68,6 +75,8 @@ type Worker struct { cancelMap sync.Map + cancelConcurrencyMap sync.Map + services sync.Map alerter errors.Alerter @@ -211,14 +220,26 @@ func (w *Worker) RegisterAction(actionId string, method any) error { } func (w *Worker) registerAction(service, verb string, method any) error { + actionId := fmt.Sprintf("%s:%s", service, verb) + + // if the service is "concurrency", then this is a special action + if service == "concurrency" { + w.actions[actionId] = &actionImpl{ + name: actionId, + runConcurrencyAction: method.(GetWorkflowConcurrencyGroupFn), + method: method, + service: service, + } + + return nil + } + actionFunc, err := getFnFromMethod(method) if err != nil { return fmt.Errorf("could not get function from method: %w", err) } - actionId := fmt.Sprintf("%s:%s", service, verb) - // if action has already been registered, ensure that the method is the same if currMethod, ok := w.actions[actionId]; ok { if reflect.ValueOf(currMethod.MethodFn()).Pointer() != reflect.ValueOf(method).Pointer() { @@ -295,18 +316,21 @@ RunWorker: } func (w *Worker) executeAction(ctx context.Context, assignedAction *client.Action) error { - if assignedAction.ActionType == client.ActionTypeStartStepRun { + switch assignedAction.ActionType { + case client.ActionTypeStartStepRun: return w.startStepRun(ctx, assignedAction) - } else if assignedAction.ActionType == client.ActionTypeCancelStepRun { + case client.ActionTypeCancelStepRun: return w.cancelStepRun(ctx, assignedAction) + case client.ActionTypeStartGetGroupKey: + return w.startGetGroupKey(ctx, assignedAction) + default: + return fmt.Errorf("unknown action type: %s", assignedAction.ActionType) } - - return fmt.Errorf("unknown action type: %s", assignedAction.ActionType) } func (w *Worker) startStepRun(ctx context.Context, assignedAction *client.Action) error { // send a message that the step run started - _, err := w.client.Dispatcher().SendActionEvent( + _, err := w.client.Dispatcher().SendStepActionEvent( ctx, w.getActionEvent(assignedAction, client.ActionEventTypeStarted), ) @@ -389,7 +413,7 @@ func (w *Worker) startStepRun(ctx context.Context, assignedAction *client.Action failureEvent.EventPayload = err.Error() - _, err := w.client.Dispatcher().SendActionEvent( + _, err := w.client.Dispatcher().SendStepActionEvent( ctx, failureEvent, ) @@ -408,7 +432,7 @@ func (w *Worker) startStepRun(ctx context.Context, assignedAction *client.Action return fmt.Errorf("could not create finished event: %w", err) } - _, err = w.client.Dispatcher().SendActionEvent( + _, err = w.client.Dispatcher().SendStepActionEvent( ctx, finishedEvent, ) @@ -422,6 +446,84 @@ func (w *Worker) startStepRun(ctx context.Context, assignedAction *client.Action }) } +func (w *Worker) startGetGroupKey(ctx context.Context, assignedAction *client.Action) error { + // send a message that the step run started + _, err := w.client.Dispatcher().SendGroupKeyActionEvent( + ctx, + w.getActionEvent(assignedAction, client.ActionEventTypeStarted), + ) + + if err != nil { + return fmt.Errorf("could not send action event: %w", err) + } + + action, ok := w.actions[assignedAction.ActionId] + + if !ok { + return fmt.Errorf("job not found") + } + + // action should be concurrency action + if action.ConcurrencyFn() == nil { + return fmt.Errorf("action %s is not a concurrency action", action.Name()) + } + + runContext, cancel := context.WithCancel(context.Background()) + + w.cancelConcurrencyMap.Store(assignedAction.WorkflowRunId, cancel) + + hCtx, err := newHatchetContext(runContext, assignedAction) + + if err != nil { + return fmt.Errorf("could not create hatchet context: %w", err) + } + + concurrencyKey, err := action.ConcurrencyFn()(hCtx) + + if err != nil { + failureEvent := w.getActionEvent(assignedAction, client.ActionEventTypeFailed) + + w.alerter.SendAlert(context.Background(), err, map[string]interface{}{ + "actionId": assignedAction.ActionId, + "workerId": assignedAction.WorkerId, + "workflowRunId": assignedAction.WorkflowRunId, + "jobName": assignedAction.JobName, + "actionType": assignedAction.ActionType, + }) + + failureEvent.EventPayload = err.Error() + + _, err := w.client.Dispatcher().SendGroupKeyActionEvent( + ctx, + failureEvent, + ) + + if err != nil { + return fmt.Errorf("could not send action event: %w", err) + } + + return err + } + + // send a message that the step run completed + finishedEvent, err := w.getGroupKeyActionFinishedEvent(assignedAction, concurrencyKey) + + if err != nil { + return fmt.Errorf("could not create finished event: %w", err) + } + + _, err = w.client.Dispatcher().SendGroupKeyActionEvent( + ctx, + finishedEvent, + ) + + if err != nil { + return fmt.Errorf("could not send action event: %w", err) + } + + return nil +} + func (w *Worker) cancelStepRun(ctx context.Context, assignedAction *client.Action) error { cancel, ok := w.cancelMap.Load(assignedAction.StepRunId) @@ -462,6 +564,14 @@ func (w *Worker) getActionFinishedEvent(action *client.Action, output any) (*cli return event, nil } +func (w *Worker) getGroupKeyActionFinishedEvent(action *client.Action, output string) (*client.ActionEvent, error) { + event := w.getActionEvent(action, client.ActionEventTypeCompleted) + + event.EventPayload = output + + return event, nil +} + func getHostName() string { hostName, err := os.Hostname() if err != nil { diff --git a/pkg/worker/workflow.go b/pkg/worker/workflow.go index c9e63c963..63f24b30e 100644 --- a/pkg/worker/workflow.go +++ b/pkg/worker/workflow.go @@ -122,6 +122,8 @@ type Workflow struct { Jobs []WorkflowJob } +type GetWorkflowConcurrencyGroupFn func(ctx HatchetContext) (string, error) + type WorkflowJob struct { // The name of the job Name string @@ -130,10 +132,34 @@ type WorkflowJob struct { Timeout string + Concurrency *WorkflowConcurrency + // The steps that are run in the job Steps []*WorkflowStep } +type WorkflowConcurrency struct { + fn GetWorkflowConcurrencyGroupFn + maxRuns *int32 + limitStrategy *types.WorkflowConcurrencyLimitStrategy +} + +func Concurrency(fn GetWorkflowConcurrencyGroupFn) *WorkflowConcurrency { + return &WorkflowConcurrency{ + fn: fn, + } +} + +func (c *WorkflowConcurrency) MaxRuns(maxRuns int32) *WorkflowConcurrency { + c.maxRuns = &maxRuns + return c +} + +func (c *WorkflowConcurrency) LimitStrategy(limitStrategy types.WorkflowConcurrencyLimitStrategy) *WorkflowConcurrency { + c.limitStrategy = &limitStrategy + return c +} + func (j *WorkflowJob) ToWorkflow(svcName string) types.Workflow { apiJob, err := j.ToWorkflowJob(svcName) @@ -145,10 +171,26 @@ func (j *WorkflowJob) ToWorkflow(svcName string) types.Workflow { j.Name: *apiJob, } - return types.Workflow{ + w := types.Workflow{ Name: j.Name, Jobs: jobs, } + + if j.Concurrency != nil { + w.Concurrency = &types.WorkflowConcurrency{ + ActionID: "concurrency:" + getFnName(j.Concurrency.fn), + } + + if j.Concurrency.maxRuns != nil { + w.Concurrency.MaxRuns = *j.Concurrency.maxRuns + } + + if j.Concurrency.limitStrategy != nil { + w.Concurrency.LimitStrategy = *j.Concurrency.limitStrategy + } + } + + return w } func (j *WorkflowJob) ToWorkflowJob(svcName string) (*types.WorkflowJob, error) { @@ -192,6 +234,10 @@ func (j *WorkflowJob) ToActionMap(svcName string) map[string]any { res[actionId] = step.Function } + if j.Concurrency != nil { + res["concurrency:"+getFnName(j.Concurrency.fn)] = j.Concurrency.fn + } + return res } diff --git a/prisma/migrations/20240129040510_v0_8_0/migration.sql b/prisma/migrations/20240129040510_v0_8_0/migration.sql new file mode 100644 index 000000000..da82afdbd --- /dev/null +++ b/prisma/migrations/20240129040510_v0_8_0/migration.sql @@ -0,0 +1,79 @@ +-- CreateEnum +CREATE TYPE "ConcurrencyLimitStrategy" AS ENUM ('CANCEL_IN_PROGRESS', 'DROP_NEWEST', 'QUEUE_NEWEST'); + +-- AlterEnum +ALTER TYPE "WorkflowRunStatus" ADD VALUE 'QUEUED'; + +-- AlterTable +ALTER TABLE "WorkflowRun" ADD COLUMN "concurrencyGroupId" TEXT; + +-- AlterTable +ALTER TABLE "WorkflowTriggerCronRef" ADD COLUMN "input" JSONB; + +-- CreateTable +CREATE TABLE "WorkflowConcurrency" ( + "id" UUID NOT NULL, + "createdAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "updatedAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "workflowVersionId" UUID NOT NULL, + "getConcurrencyGroupId" UUID, + "maxRuns" INTEGER NOT NULL DEFAULT 1, + "limitStrategy" "ConcurrencyLimitStrategy" NOT NULL DEFAULT 'CANCEL_IN_PROGRESS', + + CONSTRAINT "WorkflowConcurrency_pkey" PRIMARY KEY ("id") +); + +-- CreateTable +CREATE TABLE "GetGroupKeyRun" ( + "id" UUID NOT NULL, + "createdAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "updatedAt" TIMESTAMP(3) NOT NULL DEFAULT CURRENT_TIMESTAMP, + "deletedAt" TIMESTAMP(3), + "tenantId" UUID NOT NULL, + "workflowRunId" TEXT NOT NULL, + "workerId" UUID, + "tickerId" UUID, + "status" "StepRunStatus" NOT NULL DEFAULT 'PENDING', + "input" JSONB, + "output" TEXT, + "requeueAfter" TIMESTAMP(3), + "error" TEXT, + "startedAt" TIMESTAMP(3), + "finishedAt" TIMESTAMP(3), + "timeoutAt" TIMESTAMP(3), + "cancelledAt" TIMESTAMP(3), + "cancelledReason" TEXT, + "cancelledError" TEXT, + + CONSTRAINT "GetGroupKeyRun_pkey" PRIMARY KEY ("id") +); + +-- CreateIndex +CREATE UNIQUE INDEX "WorkflowConcurrency_id_key" ON "WorkflowConcurrency"("id"); + +-- CreateIndex +CREATE UNIQUE INDEX "WorkflowConcurrency_workflowVersionId_key" ON "WorkflowConcurrency"("workflowVersionId"); + +-- CreateIndex +CREATE UNIQUE INDEX "GetGroupKeyRun_id_key" ON "GetGroupKeyRun"("id"); + +-- CreateIndex +CREATE UNIQUE INDEX "GetGroupKeyRun_tenantId_workflowRunId_key" ON "GetGroupKeyRun"("tenantId", "workflowRunId"); + +-- AddForeignKey +ALTER TABLE "WorkflowConcurrency" ADD CONSTRAINT "WorkflowConcurrency_workflowVersionId_fkey" FOREIGN KEY ("workflowVersionId") REFERENCES "WorkflowVersion"("id") ON DELETE CASCADE ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "WorkflowConcurrency" ADD CONSTRAINT "WorkflowConcurrency_getConcurrencyGroupId_fkey" FOREIGN KEY ("getConcurrencyGroupId") REFERENCES "Action"("id") ON DELETE SET NULL ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_tenantId_fkey" FOREIGN KEY ("tenantId") REFERENCES "Tenant"("id") ON DELETE CASCADE ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_tenantId_workflowRunId_fkey" FOREIGN KEY ("tenantId", "workflowRunId") REFERENCES "WorkflowRun"("tenantId", "id") ON DELETE CASCADE ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_workerId_fkey" FOREIGN KEY ("workerId") REFERENCES "Worker"("id") ON DELETE SET NULL ON UPDATE CASCADE; + +-- AddForeignKey +ALTER TABLE "GetGroupKeyRun" ADD CONSTRAINT "GetGroupKeyRun_tickerId_fkey" FOREIGN KEY ("tickerId") REFERENCES "Ticker"("id") ON DELETE SET NULL ON UPDATE CASCADE; diff --git a/prisma/schema.prisma b/prisma/schema.prisma index 46ef5e383..61141aa61 100644 --- a/prisma/schema.prisma +++ b/prisma/schema.prisma @@ -118,6 +118,7 @@ model Tenant { services Service[] invites TenantInviteLink[] apiTokens APIToken[] + groupKeyRuns GetGroupKeyRun[] } enum TenantMemberRole { @@ -288,6 +289,9 @@ model WorkflowVersion { // the declared triggers for the job triggers WorkflowTriggers? + // concurrency limits for the workflow + concurrency WorkflowConcurrency? + // the declared jobs jobs Job[] @@ -298,6 +302,39 @@ model WorkflowVersion { scheduled WorkflowTriggerScheduledRef[] } +enum ConcurrencyLimitStrategy { + // Cancel the existing runs and start a new one + CANCEL_IN_PROGRESS + + // Don't create a new run if concurrency limit has been reached + DROP_NEWEST + + // Queue new runs and start them when falling below the concurrency limit + QUEUE_NEWEST +} + +model WorkflowConcurrency { + // base fields + id String @id @unique @default(uuid()) @db.Uuid + createdAt DateTime @default(now()) + updatedAt DateTime @default(now()) @updatedAt + + // the parent workflow + workflow WorkflowVersion @relation(fields: [workflowVersionId], references: [id], onDelete: Cascade, onUpdate: Cascade) + workflowVersionId String @unique @db.Uuid + + // An action which gets the concurrency group for the WorkflowRun. If this is not set, the concurrency group is + // set to a checksum of the workflow input. + getConcurrencyGroup Action? @relation(fields: [getConcurrencyGroupId], references: [id]) + getConcurrencyGroupId String? @db.Uuid + + // the maximum number of concurrent workflow runs + maxRuns Int @default(1) + + // the strategy to use when the concurrency limit is reached + limitStrategy ConcurrencyLimitStrategy @default(CANCEL_IN_PROGRESS) +} + model WorkflowTriggers { // base fields id String @id @unique @default(uuid()) @db.Uuid @@ -345,6 +382,9 @@ model WorkflowTriggerCronRef { triggered WorkflowRunTriggeredBy[] + // the input parameters to the scheduled workflow + input Json? + // cron references must be unique per workflow @@unique([parentId, cron]) } @@ -421,6 +461,9 @@ model Action { // the action's workers workers Worker[] + // the action's concurrency rules + concurrency WorkflowConcurrency[] + // actions are unique per tenant @@unique([tenantId, actionId]) } @@ -464,6 +507,7 @@ model Step { enum WorkflowRunStatus { PENDING + QUEUED RUNNING SUCCEEDED FAILED @@ -484,6 +528,10 @@ model WorkflowRun { workflowVersion WorkflowVersion @relation(fields: [workflowVersionId], references: [id], onDelete: Cascade, onUpdate: Cascade) workflowVersionId String @db.Uuid + concurrencyGroupId String? + + getGroupKeyRun GetGroupKeyRun? + status WorkflowRunStatus @default(PENDING) jobRuns JobRun[] @@ -503,6 +551,65 @@ model WorkflowRun { @@unique([tenantId, id]) } +model GetGroupKeyRun { + // base fields + id String @id @unique @default(uuid()) @db.Uuid + createdAt DateTime @default(now()) + updatedAt DateTime @default(now()) @updatedAt + deletedAt DateTime? + + // the parent tenant + tenant Tenant @relation(fields: [tenantId], references: [id], onDelete: Cascade, onUpdate: Cascade) + tenantId String @db.Uuid + + // the parent workflow run + workflowRun WorkflowRun @relation(fields: [tenantId, workflowRunId], references: [tenantId, id], onDelete: Cascade, onUpdate: Cascade) + workflowRunId String + + // the worker assigned to this group key run + worker Worker? @relation(fields: [workerId], references: [id]) + workerId String? @db.Uuid + + // the assigned ticker + ticker Ticker? @relation(fields: [tickerId], references: [id]) + tickerId String? @db.Uuid + + // the run status + status StepRunStatus @default(PENDING) + + // the group key run input + input Json? + + // the group key as output + output String? + + // when the step should be requeued + requeueAfter DateTime? + + // the run error + error String? + + // the run started at + startedAt DateTime? + + // the run finished at + finishedAt DateTime? + + // the run timeout at + timeoutAt DateTime? + + // the run cancelled at + cancelledAt DateTime? + + // the reason for why the run was cancelled + cancelledReason String? + + // errors while cancelling the run + cancelledError String? + + @@unique([tenantId, workflowRunId]) +} + model WorkflowRunTriggeredBy { id String @id @unique @default(uuid()) @db.Uuid createdAt DateTime @default(now()) @@ -729,10 +836,11 @@ model Ticker { // whether this ticker is active or not isActive Boolean @default(true) - jobRuns JobRun[] - stepRuns StepRun[] - crons WorkflowTriggerCronRef[] - scheduled WorkflowTriggerScheduledRef[] + jobRuns JobRun[] + stepRuns StepRun[] + crons WorkflowTriggerCronRef[] + scheduled WorkflowTriggerScheduledRef[] + groupKeyRuns GetGroupKeyRun[] } enum WorkerStatus { @@ -771,6 +879,9 @@ model Worker { // the jobs the worker has run stepRuns StepRun[] + + // the runs which retrieve the group keys + groupKeyRuns GetGroupKeyRun[] } model Service { diff --git a/python-sdk/examples/concurrency-limit/event_test.py b/python-sdk/examples/concurrency-limit/event_test.py new file mode 100644 index 000000000..3c8fa7a06 --- /dev/null +++ b/python-sdk/examples/concurrency-limit/event_test.py @@ -0,0 +1,13 @@ +from hatchet_sdk import new_client +from dotenv import load_dotenv + +load_dotenv() + +client = new_client() + +client.event.push( + "concurrency-test", + { + "test": "test" + } +) \ No newline at end of file diff --git a/python-sdk/examples/concurrency-limit/worker.py b/python-sdk/examples/concurrency-limit/worker.py new file mode 100644 index 000000000..9250e56bc --- /dev/null +++ b/python-sdk/examples/concurrency-limit/worker.py @@ -0,0 +1,32 @@ +from hatchet_sdk import Hatchet +from dotenv import load_dotenv + +load_dotenv() + +hatchet = Hatchet(debug=True) + +@hatchet.workflow(on_events=["concurrency-test"]) +class ConcurrencyDemoWorkflow: + def __init__(self): + self.my_value = "test" + + @hatchet.concurrency(max_runs=5) + def concurrency(self, context) -> str: + return "concurrency-key" + + @hatchet.step() + def step1(self, context): + print("executed step1") + pass + + @hatchet.step(parents=["step1"],timeout='4s') + def step2(self, context): + print("started step2") + context.sleep(1) + print("finished step2") + +workflow = ConcurrencyDemoWorkflow() +worker = hatchet.worker('concurrency-demo-worker', max_threads=4) +worker.register_workflow(workflow) + +worker.start() \ No newline at end of file diff --git a/python-sdk/hatchet_sdk/clients/dispatcher.py b/python-sdk/hatchet_sdk/clients/dispatcher.py index d042e638d..cd5cf2f89 100644 --- a/python-sdk/hatchet_sdk/clients/dispatcher.py +++ b/python-sdk/hatchet_sdk/clients/dispatcher.py @@ -1,5 +1,5 @@ # relative imports -from ..dispatcher_pb2 import ActionEvent, ActionEventResponse, ActionType, AssignedAction, WorkerListenRequest, WorkerRegisterRequest, WorkerUnsubscribeRequest, WorkerRegisterResponse +from ..dispatcher_pb2 import GroupKeyActionEvent, StepActionEvent, ActionEventResponse, ActionType, AssignedAction, WorkerListenRequest, WorkerRegisterRequest, WorkerUnsubscribeRequest, WorkerRegisterResponse from ..dispatcher_pb2_grpc import DispatcherStub import time @@ -21,7 +21,7 @@ class DispatcherClient: def get_action_listener(self, ctx, req): raise NotImplementedError - def send_action_event(self, ctx, in_): + def send_step_action_event(self, ctx, in_): raise NotImplementedError DEFAULT_ACTION_LISTENER_RETRY_INTERVAL = 1 # seconds @@ -36,8 +36,10 @@ class GetActionListenerRequest: self.actions = actions class Action: - def __init__(self, worker_id: str, tenant_id: str, job_id: str, job_name: str, job_run_id: str, step_id: str, step_run_id: str, action_id: str, action_payload: str, action_type: ActionType): + def __init__(self, worker_id: str, tenant_id: str, workflow_run_id: str, get_group_key_run_id: str, job_id: str, job_name: str, job_run_id: str, step_id: str, step_run_id: str, action_id: str, action_payload: str, action_type: ActionType): self.worker_id = worker_id + self.workflow_run_id = workflow_run_id + self.get_group_key_run_id = get_group_key_run_id self.tenant_id = tenant_id self.job_id = job_id self.job_name = job_name @@ -55,9 +57,9 @@ class WorkerActionListener: def unregister(self): raise NotImplementedError -# enum for START_STEP_RUN and CANCEL_STEP_RUN START_STEP_RUN = 0 CANCEL_STEP_RUN = 1 +START_GET_GROUP_KEY = 2 class ActionListenerImpl(WorkerActionListener): def __init__(self, client : DispatcherStub, token, worker_id): @@ -88,6 +90,8 @@ class ActionListenerImpl(WorkerActionListener): action = Action( tenant_id=assigned_action.tenantId, worker_id=self.worker_id, + workflow_run_id=assigned_action.workflowRunId, + get_group_key_run_id=assigned_action.getGroupKeyRunId, job_id=assigned_action.jobId, job_name=assigned_action.jobName, job_run_id=assigned_action.jobRunId, @@ -132,6 +136,8 @@ class ActionListenerImpl(WorkerActionListener): return START_STEP_RUN elif action_type == ActionType.CANCEL_STEP_RUN: return CANCEL_STEP_RUN + elif action_type == ActionType.START_GET_GROUP_KEY: + return START_GET_GROUP_KEY else: # self.logger.error(f"Unknown action type: {action_type}") return None @@ -180,8 +186,13 @@ class DispatcherClientImpl(DispatcherClient): return ActionListenerImpl(self.client, self.token, response.workerId) - def send_action_event(self, in_: ActionEvent): - response : ActionEventResponse = self.client.SendActionEvent(in_, metadata=get_metadata(self.token),) + def send_step_action_event(self, in_: StepActionEvent): + response : ActionEventResponse = self.client.SendStepActionEvent(in_, metadata=get_metadata(self.token),) + + return response + + def send_group_key_action_event(self, in_: GroupKeyActionEvent): + response : ActionEventResponse = self.client.SendGroupKeyActionEvent(in_, metadata=get_metadata(self.token),) return response diff --git a/python-sdk/hatchet_sdk/dispatcher_pb2.py b/python-sdk/hatchet_sdk/dispatcher_pb2.py index 826e0d7e2..a765d9deb 100644 --- a/python-sdk/hatchet_sdk/dispatcher_pb2.py +++ b/python-sdk/hatchet_sdk/dispatcher_pb2.py @@ -15,7 +15,7 @@ _sym_db = _symbol_database.Default() from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x10\x64ispatcher.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"N\n\x15WorkerRegisterRequest\x12\x12\n\nworkerName\x18\x01 \x01(\t\x12\x0f\n\x07\x61\x63tions\x18\x02 \x03(\t\x12\x10\n\x08services\x18\x03 \x03(\t\"P\n\x16WorkerRegisterResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\x12\x12\n\nworkerName\x18\x03 \x01(\t\"\xc1\x01\n\x0e\x41ssignedAction\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\r\n\x05jobId\x18\x02 \x01(\t\x12\x0f\n\x07jobName\x18\x03 \x01(\t\x12\x10\n\x08jobRunId\x18\x04 \x01(\t\x12\x0e\n\x06stepId\x18\x05 \x01(\t\x12\x11\n\tstepRunId\x18\x06 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\x07 \x01(\t\x12\x1f\n\nactionType\x18\x08 \x01(\x0e\x32\x0b.ActionType\x12\x15\n\ractionPayload\x18\t \x01(\t\"\'\n\x13WorkerListenRequest\x12\x10\n\x08workerId\x18\x01 \x01(\t\",\n\x18WorkerUnsubscribeRequest\x12\x10\n\x08workerId\x18\x01 \x01(\t\"?\n\x19WorkerUnsubscribeResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\"\xe4\x01\n\x0b\x41\x63tionEvent\x12\x10\n\x08workerId\x18\x01 \x01(\t\x12\r\n\x05jobId\x18\x02 \x01(\t\x12\x10\n\x08jobRunId\x18\x03 \x01(\t\x12\x0e\n\x06stepId\x18\x04 \x01(\t\x12\x11\n\tstepRunId\x18\x05 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\x06 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12#\n\teventType\x18\x08 \x01(\x0e\x32\x10.ActionEventType\x12\x14\n\x0c\x65ventPayload\x18\t \x01(\t\"9\n\x13\x41\x63tionEventResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t*5\n\nActionType\x12\x12\n\x0eSTART_STEP_RUN\x10\x00\x12\x13\n\x0f\x43\x41NCEL_STEP_RUN\x10\x01*\x86\x01\n\x0f\x41\x63tionEventType\x12\x1b\n\x17STEP_EVENT_TYPE_UNKNOWN\x10\x00\x12\x1b\n\x17STEP_EVENT_TYPE_STARTED\x10\x01\x12\x1d\n\x19STEP_EVENT_TYPE_COMPLETED\x10\x02\x12\x1a\n\x16STEP_EVENT_TYPE_FAILED\x10\x03\x32\x81\x02\n\nDispatcher\x12=\n\x08Register\x12\x16.WorkerRegisterRequest\x1a\x17.WorkerRegisterResponse\"\x00\x12\x33\n\x06Listen\x12\x14.WorkerListenRequest\x1a\x0f.AssignedAction\"\x00\x30\x01\x12\x37\n\x0fSendActionEvent\x12\x0c.ActionEvent\x1a\x14.ActionEventResponse\"\x00\x12\x46\n\x0bUnsubscribe\x12\x19.WorkerUnsubscribeRequest\x1a\x1a.WorkerUnsubscribeResponse\"\x00\x42GZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contractsb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x10\x64ispatcher.proto\x1a\x1fgoogle/protobuf/timestamp.proto\"N\n\x15WorkerRegisterRequest\x12\x12\n\nworkerName\x18\x01 \x01(\t\x12\x0f\n\x07\x61\x63tions\x18\x02 \x03(\t\x12\x10\n\x08services\x18\x03 \x03(\t\"P\n\x16WorkerRegisterResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\x12\x12\n\nworkerName\x18\x03 \x01(\t\"\xf2\x01\n\x0e\x41ssignedAction\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x15\n\rworkflowRunId\x18\x02 \x01(\t\x12\x18\n\x10getGroupKeyRunId\x18\x03 \x01(\t\x12\r\n\x05jobId\x18\x04 \x01(\t\x12\x0f\n\x07jobName\x18\x05 \x01(\t\x12\x10\n\x08jobRunId\x18\x06 \x01(\t\x12\x0e\n\x06stepId\x18\x07 \x01(\t\x12\x11\n\tstepRunId\x18\x08 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\t \x01(\t\x12\x1f\n\nactionType\x18\n \x01(\x0e\x32\x0b.ActionType\x12\x15\n\ractionPayload\x18\x0b \x01(\t\"\'\n\x13WorkerListenRequest\x12\x10\n\x08workerId\x18\x01 \x01(\t\",\n\x18WorkerUnsubscribeRequest\x12\x10\n\x08workerId\x18\x01 \x01(\t\"?\n\x19WorkerUnsubscribeResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t\"\xe1\x01\n\x13GroupKeyActionEvent\x12\x10\n\x08workerId\x18\x01 \x01(\t\x12\x15\n\rworkflowRunId\x18\x02 \x01(\t\x12\x18\n\x10getGroupKeyRunId\x18\x03 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\x04 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12+\n\teventType\x18\x06 \x01(\x0e\x32\x18.GroupKeyActionEventType\x12\x14\n\x0c\x65ventPayload\x18\x07 \x01(\t\"\xec\x01\n\x0fStepActionEvent\x12\x10\n\x08workerId\x18\x01 \x01(\t\x12\r\n\x05jobId\x18\x02 \x01(\t\x12\x10\n\x08jobRunId\x18\x03 \x01(\t\x12\x0e\n\x06stepId\x18\x04 \x01(\t\x12\x11\n\tstepRunId\x18\x05 \x01(\t\x12\x10\n\x08\x61\x63tionId\x18\x06 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x07 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\'\n\teventType\x18\x08 \x01(\x0e\x32\x14.StepActionEventType\x12\x14\n\x0c\x65ventPayload\x18\t \x01(\t\"9\n\x13\x41\x63tionEventResponse\x12\x10\n\x08tenantId\x18\x01 \x01(\t\x12\x10\n\x08workerId\x18\x02 \x01(\t*N\n\nActionType\x12\x12\n\x0eSTART_STEP_RUN\x10\x00\x12\x13\n\x0f\x43\x41NCEL_STEP_RUN\x10\x01\x12\x17\n\x13START_GET_GROUP_KEY\x10\x02*\xa2\x01\n\x17GroupKeyActionEventType\x12 \n\x1cGROUP_KEY_EVENT_TYPE_UNKNOWN\x10\x00\x12 \n\x1cGROUP_KEY_EVENT_TYPE_STARTED\x10\x01\x12\"\n\x1eGROUP_KEY_EVENT_TYPE_COMPLETED\x10\x02\x12\x1f\n\x1bGROUP_KEY_EVENT_TYPE_FAILED\x10\x03*\x8a\x01\n\x13StepActionEventType\x12\x1b\n\x17STEP_EVENT_TYPE_UNKNOWN\x10\x00\x12\x1b\n\x17STEP_EVENT_TYPE_STARTED\x10\x01\x12\x1d\n\x19STEP_EVENT_TYPE_COMPLETED\x10\x02\x12\x1a\n\x16STEP_EVENT_TYPE_FAILED\x10\x03\x32\xd2\x02\n\nDispatcher\x12=\n\x08Register\x12\x16.WorkerRegisterRequest\x1a\x17.WorkerRegisterResponse\"\x00\x12\x33\n\x06Listen\x12\x14.WorkerListenRequest\x1a\x0f.AssignedAction\"\x00\x30\x01\x12?\n\x13SendStepActionEvent\x12\x10.StepActionEvent\x1a\x14.ActionEventResponse\"\x00\x12G\n\x17SendGroupKeyActionEvent\x12\x14.GroupKeyActionEvent\x1a\x14.ActionEventResponse\"\x00\x12\x46\n\x0bUnsubscribe\x12\x19.WorkerUnsubscribeRequest\x1a\x1a.WorkerUnsubscribeResponse\"\x00\x42GZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contractsb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -23,26 +23,30 @@ _builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'dispatcher_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: _globals['DESCRIPTOR']._options = None _globals['DESCRIPTOR']._serialized_options = b'ZEgithub.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts' - _globals['_ACTIONTYPE']._serialized_start=853 - _globals['_ACTIONTYPE']._serialized_end=906 - _globals['_ACTIONEVENTTYPE']._serialized_start=909 - _globals['_ACTIONEVENTTYPE']._serialized_end=1043 + _globals['_ACTIONTYPE']._serialized_start=1138 + _globals['_ACTIONTYPE']._serialized_end=1216 + _globals['_GROUPKEYACTIONEVENTTYPE']._serialized_start=1219 + _globals['_GROUPKEYACTIONEVENTTYPE']._serialized_end=1381 + _globals['_STEPACTIONEVENTTYPE']._serialized_start=1384 + _globals['_STEPACTIONEVENTTYPE']._serialized_end=1522 _globals['_WORKERREGISTERREQUEST']._serialized_start=53 _globals['_WORKERREGISTERREQUEST']._serialized_end=131 _globals['_WORKERREGISTERRESPONSE']._serialized_start=133 _globals['_WORKERREGISTERRESPONSE']._serialized_end=213 _globals['_ASSIGNEDACTION']._serialized_start=216 - _globals['_ASSIGNEDACTION']._serialized_end=409 - _globals['_WORKERLISTENREQUEST']._serialized_start=411 - _globals['_WORKERLISTENREQUEST']._serialized_end=450 - _globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_start=452 - _globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_end=496 - _globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_start=498 - _globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_end=561 - _globals['_ACTIONEVENT']._serialized_start=564 - _globals['_ACTIONEVENT']._serialized_end=792 - _globals['_ACTIONEVENTRESPONSE']._serialized_start=794 - _globals['_ACTIONEVENTRESPONSE']._serialized_end=851 - _globals['_DISPATCHER']._serialized_start=1046 - _globals['_DISPATCHER']._serialized_end=1303 + _globals['_ASSIGNEDACTION']._serialized_end=458 + _globals['_WORKERLISTENREQUEST']._serialized_start=460 + _globals['_WORKERLISTENREQUEST']._serialized_end=499 + _globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_start=501 + _globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_end=545 + _globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_start=547 + _globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_end=610 + _globals['_GROUPKEYACTIONEVENT']._serialized_start=613 + _globals['_GROUPKEYACTIONEVENT']._serialized_end=838 + _globals['_STEPACTIONEVENT']._serialized_start=841 + _globals['_STEPACTIONEVENT']._serialized_end=1077 + _globals['_ACTIONEVENTRESPONSE']._serialized_start=1079 + _globals['_ACTIONEVENTRESPONSE']._serialized_end=1136 + _globals['_DISPATCHER']._serialized_start=1525 + _globals['_DISPATCHER']._serialized_end=1863 # @@protoc_insertion_point(module_scope) diff --git a/python-sdk/hatchet_sdk/dispatcher_pb2.pyi b/python-sdk/hatchet_sdk/dispatcher_pb2.pyi index 05299d4ae..12404a118 100644 --- a/python-sdk/hatchet_sdk/dispatcher_pb2.pyi +++ b/python-sdk/hatchet_sdk/dispatcher_pb2.pyi @@ -11,19 +11,32 @@ class ActionType(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () START_STEP_RUN: _ClassVar[ActionType] CANCEL_STEP_RUN: _ClassVar[ActionType] + START_GET_GROUP_KEY: _ClassVar[ActionType] -class ActionEventType(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): +class GroupKeyActionEventType(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): __slots__ = () - STEP_EVENT_TYPE_UNKNOWN: _ClassVar[ActionEventType] - STEP_EVENT_TYPE_STARTED: _ClassVar[ActionEventType] - STEP_EVENT_TYPE_COMPLETED: _ClassVar[ActionEventType] - STEP_EVENT_TYPE_FAILED: _ClassVar[ActionEventType] + GROUP_KEY_EVENT_TYPE_UNKNOWN: _ClassVar[GroupKeyActionEventType] + GROUP_KEY_EVENT_TYPE_STARTED: _ClassVar[GroupKeyActionEventType] + GROUP_KEY_EVENT_TYPE_COMPLETED: _ClassVar[GroupKeyActionEventType] + GROUP_KEY_EVENT_TYPE_FAILED: _ClassVar[GroupKeyActionEventType] + +class StepActionEventType(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): + __slots__ = () + STEP_EVENT_TYPE_UNKNOWN: _ClassVar[StepActionEventType] + STEP_EVENT_TYPE_STARTED: _ClassVar[StepActionEventType] + STEP_EVENT_TYPE_COMPLETED: _ClassVar[StepActionEventType] + STEP_EVENT_TYPE_FAILED: _ClassVar[StepActionEventType] START_STEP_RUN: ActionType CANCEL_STEP_RUN: ActionType -STEP_EVENT_TYPE_UNKNOWN: ActionEventType -STEP_EVENT_TYPE_STARTED: ActionEventType -STEP_EVENT_TYPE_COMPLETED: ActionEventType -STEP_EVENT_TYPE_FAILED: ActionEventType +START_GET_GROUP_KEY: ActionType +GROUP_KEY_EVENT_TYPE_UNKNOWN: GroupKeyActionEventType +GROUP_KEY_EVENT_TYPE_STARTED: GroupKeyActionEventType +GROUP_KEY_EVENT_TYPE_COMPLETED: GroupKeyActionEventType +GROUP_KEY_EVENT_TYPE_FAILED: GroupKeyActionEventType +STEP_EVENT_TYPE_UNKNOWN: StepActionEventType +STEP_EVENT_TYPE_STARTED: StepActionEventType +STEP_EVENT_TYPE_COMPLETED: StepActionEventType +STEP_EVENT_TYPE_FAILED: StepActionEventType class WorkerRegisterRequest(_message.Message): __slots__ = ("workerName", "actions", "services") @@ -46,8 +59,10 @@ class WorkerRegisterResponse(_message.Message): def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ..., workerName: _Optional[str] = ...) -> None: ... class AssignedAction(_message.Message): - __slots__ = ("tenantId", "jobId", "jobName", "jobRunId", "stepId", "stepRunId", "actionId", "actionType", "actionPayload") + __slots__ = ("tenantId", "workflowRunId", "getGroupKeyRunId", "jobId", "jobName", "jobRunId", "stepId", "stepRunId", "actionId", "actionType", "actionPayload") TENANTID_FIELD_NUMBER: _ClassVar[int] + WORKFLOWRUNID_FIELD_NUMBER: _ClassVar[int] + GETGROUPKEYRUNID_FIELD_NUMBER: _ClassVar[int] JOBID_FIELD_NUMBER: _ClassVar[int] JOBNAME_FIELD_NUMBER: _ClassVar[int] JOBRUNID_FIELD_NUMBER: _ClassVar[int] @@ -57,6 +72,8 @@ class AssignedAction(_message.Message): ACTIONTYPE_FIELD_NUMBER: _ClassVar[int] ACTIONPAYLOAD_FIELD_NUMBER: _ClassVar[int] tenantId: str + workflowRunId: str + getGroupKeyRunId: str jobId: str jobName: str jobRunId: str @@ -65,7 +82,7 @@ class AssignedAction(_message.Message): actionId: str actionType: ActionType actionPayload: str - def __init__(self, tenantId: _Optional[str] = ..., jobId: _Optional[str] = ..., jobName: _Optional[str] = ..., jobRunId: _Optional[str] = ..., stepId: _Optional[str] = ..., stepRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., actionType: _Optional[_Union[ActionType, str]] = ..., actionPayload: _Optional[str] = ...) -> None: ... + def __init__(self, tenantId: _Optional[str] = ..., workflowRunId: _Optional[str] = ..., getGroupKeyRunId: _Optional[str] = ..., jobId: _Optional[str] = ..., jobName: _Optional[str] = ..., jobRunId: _Optional[str] = ..., stepId: _Optional[str] = ..., stepRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., actionType: _Optional[_Union[ActionType, str]] = ..., actionPayload: _Optional[str] = ...) -> None: ... class WorkerListenRequest(_message.Message): __slots__ = ("workerId",) @@ -87,7 +104,25 @@ class WorkerUnsubscribeResponse(_message.Message): workerId: str def __init__(self, tenantId: _Optional[str] = ..., workerId: _Optional[str] = ...) -> None: ... -class ActionEvent(_message.Message): +class GroupKeyActionEvent(_message.Message): + __slots__ = ("workerId", "workflowRunId", "getGroupKeyRunId", "actionId", "eventTimestamp", "eventType", "eventPayload") + WORKERID_FIELD_NUMBER: _ClassVar[int] + WORKFLOWRUNID_FIELD_NUMBER: _ClassVar[int] + GETGROUPKEYRUNID_FIELD_NUMBER: _ClassVar[int] + ACTIONID_FIELD_NUMBER: _ClassVar[int] + EVENTTIMESTAMP_FIELD_NUMBER: _ClassVar[int] + EVENTTYPE_FIELD_NUMBER: _ClassVar[int] + EVENTPAYLOAD_FIELD_NUMBER: _ClassVar[int] + workerId: str + workflowRunId: str + getGroupKeyRunId: str + actionId: str + eventTimestamp: _timestamp_pb2.Timestamp + eventType: GroupKeyActionEventType + eventPayload: str + def __init__(self, workerId: _Optional[str] = ..., workflowRunId: _Optional[str] = ..., getGroupKeyRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., eventTimestamp: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., eventType: _Optional[_Union[GroupKeyActionEventType, str]] = ..., eventPayload: _Optional[str] = ...) -> None: ... + +class StepActionEvent(_message.Message): __slots__ = ("workerId", "jobId", "jobRunId", "stepId", "stepRunId", "actionId", "eventTimestamp", "eventType", "eventPayload") WORKERID_FIELD_NUMBER: _ClassVar[int] JOBID_FIELD_NUMBER: _ClassVar[int] @@ -105,9 +140,9 @@ class ActionEvent(_message.Message): stepRunId: str actionId: str eventTimestamp: _timestamp_pb2.Timestamp - eventType: ActionEventType + eventType: StepActionEventType eventPayload: str - def __init__(self, workerId: _Optional[str] = ..., jobId: _Optional[str] = ..., jobRunId: _Optional[str] = ..., stepId: _Optional[str] = ..., stepRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., eventTimestamp: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., eventType: _Optional[_Union[ActionEventType, str]] = ..., eventPayload: _Optional[str] = ...) -> None: ... + def __init__(self, workerId: _Optional[str] = ..., jobId: _Optional[str] = ..., jobRunId: _Optional[str] = ..., stepId: _Optional[str] = ..., stepRunId: _Optional[str] = ..., actionId: _Optional[str] = ..., eventTimestamp: _Optional[_Union[_timestamp_pb2.Timestamp, _Mapping]] = ..., eventType: _Optional[_Union[StepActionEventType, str]] = ..., eventPayload: _Optional[str] = ...) -> None: ... class ActionEventResponse(_message.Message): __slots__ = ("tenantId", "workerId") diff --git a/python-sdk/hatchet_sdk/dispatcher_pb2_grpc.py b/python-sdk/hatchet_sdk/dispatcher_pb2_grpc.py index 2f6a27672..182dba619 100644 --- a/python-sdk/hatchet_sdk/dispatcher_pb2_grpc.py +++ b/python-sdk/hatchet_sdk/dispatcher_pb2_grpc.py @@ -23,9 +23,14 @@ class DispatcherStub(object): request_serializer=dispatcher__pb2.WorkerListenRequest.SerializeToString, response_deserializer=dispatcher__pb2.AssignedAction.FromString, ) - self.SendActionEvent = channel.unary_unary( - '/Dispatcher/SendActionEvent', - request_serializer=dispatcher__pb2.ActionEvent.SerializeToString, + self.SendStepActionEvent = channel.unary_unary( + '/Dispatcher/SendStepActionEvent', + request_serializer=dispatcher__pb2.StepActionEvent.SerializeToString, + response_deserializer=dispatcher__pb2.ActionEventResponse.FromString, + ) + self.SendGroupKeyActionEvent = channel.unary_unary( + '/Dispatcher/SendGroupKeyActionEvent', + request_serializer=dispatcher__pb2.GroupKeyActionEvent.SerializeToString, response_deserializer=dispatcher__pb2.ActionEventResponse.FromString, ) self.Unsubscribe = channel.unary_unary( @@ -50,7 +55,13 @@ class DispatcherServicer(object): context.set_details('Method not implemented!') raise NotImplementedError('Method not implemented!') - def SendActionEvent(self, request, context): + def SendStepActionEvent(self, request, context): + """Missing associated documentation comment in .proto file.""" + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + + def SendGroupKeyActionEvent(self, request, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) context.set_details('Method not implemented!') @@ -75,9 +86,14 @@ def add_DispatcherServicer_to_server(servicer, server): request_deserializer=dispatcher__pb2.WorkerListenRequest.FromString, response_serializer=dispatcher__pb2.AssignedAction.SerializeToString, ), - 'SendActionEvent': grpc.unary_unary_rpc_method_handler( - servicer.SendActionEvent, - request_deserializer=dispatcher__pb2.ActionEvent.FromString, + 'SendStepActionEvent': grpc.unary_unary_rpc_method_handler( + servicer.SendStepActionEvent, + request_deserializer=dispatcher__pb2.StepActionEvent.FromString, + response_serializer=dispatcher__pb2.ActionEventResponse.SerializeToString, + ), + 'SendGroupKeyActionEvent': grpc.unary_unary_rpc_method_handler( + servicer.SendGroupKeyActionEvent, + request_deserializer=dispatcher__pb2.GroupKeyActionEvent.FromString, response_serializer=dispatcher__pb2.ActionEventResponse.SerializeToString, ), 'Unsubscribe': grpc.unary_unary_rpc_method_handler( @@ -130,7 +146,7 @@ class Dispatcher(object): insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def SendActionEvent(request, + def SendStepActionEvent(request, target, options=(), channel_credentials=None, @@ -140,8 +156,25 @@ class Dispatcher(object): wait_for_ready=None, timeout=None, metadata=None): - return grpc.experimental.unary_unary(request, target, '/Dispatcher/SendActionEvent', - dispatcher__pb2.ActionEvent.SerializeToString, + return grpc.experimental.unary_unary(request, target, '/Dispatcher/SendStepActionEvent', + dispatcher__pb2.StepActionEvent.SerializeToString, + dispatcher__pb2.ActionEventResponse.FromString, + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + + @staticmethod + def SendGroupKeyActionEvent(request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/Dispatcher/SendGroupKeyActionEvent', + dispatcher__pb2.GroupKeyActionEvent.SerializeToString, dispatcher__pb2.ActionEventResponse.FromString, options, channel_credentials, insecure, call_credentials, compression, wait_for_ready, timeout, metadata) diff --git a/python-sdk/hatchet_sdk/hatchet.py b/python-sdk/hatchet_sdk/hatchet.py index 0e1db1b7b..1ca98e796 100644 --- a/python-sdk/hatchet_sdk/hatchet.py +++ b/python-sdk/hatchet_sdk/hatchet.py @@ -12,6 +12,15 @@ class Hatchet: if not debug: logger.disable("hatchet_sdk") + def concurrency(self, name : str='', max_runs : int = 1): + def inner(func): + func._concurrency_fn_name = name or func.__name__ + func._concurrency_max_runs = max_runs + + return func + + return inner + def workflow(self, name : str='', on_events : list=[], on_crons : list=[], version : str=''): def inner(cls): cls.on_events = on_events diff --git a/python-sdk/hatchet_sdk/worker.py b/python-sdk/hatchet_sdk/worker.py index f9f7cebb9..cf9e97822 100644 --- a/python-sdk/hatchet_sdk/worker.py +++ b/python-sdk/hatchet_sdk/worker.py @@ -10,7 +10,7 @@ import grpc from typing import Any, Callable, Dict from .workflow import WorkflowMeta from .clients.dispatcher import GetActionListenerRequest, ActionListenerImpl, Action -from .dispatcher_pb2 import STEP_EVENT_TYPE_FAILED, ActionType, ActionEvent, ActionEventType, STEP_EVENT_TYPE_COMPLETED, STEP_EVENT_TYPE_STARTED +from .dispatcher_pb2 import ActionType, StepActionEvent, StepActionEventType, GroupKeyActionEvent, GroupKeyActionEventType, STEP_EVENT_TYPE_COMPLETED, STEP_EVENT_TYPE_STARTED, STEP_EVENT_TYPE_FAILED, GROUP_KEY_EVENT_TYPE_STARTED, GROUP_KEY_EVENT_TYPE_COMPLETED, GROUP_KEY_EVENT_TYPE_FAILED from .client import new_client from concurrent.futures import ThreadPoolExecutor, Future from google.protobuf.timestamp_pb2 import Timestamp @@ -53,24 +53,24 @@ class Worker: errored = True # This except is coming from the application itself, so we want to send that to the Hatchet instance - event = self.get_action_event(action, STEP_EVENT_TYPE_FAILED) + event = self.get_step_action_event(action, STEP_EVENT_TYPE_FAILED) event.eventPayload = str(e) try: - self.client.dispatcher.send_action_event(event) + self.client.dispatcher.send_step_action_event(event) except Exception as e: logger.error(f"Could not send action event: {e}") if not errored: # Create an action event try: - event = self.get_action_finished_event(action, output) + event = self.get_step_action_finished_event(action, output) except Exception as e: logger.error(f"Could not get action finished event: {e}") raise e # Send the action event to the dispatcher - self.client.dispatcher.send_action_event(event) + self.client.dispatcher.send_step_action_event(event) # Remove the future from the dictionary if action.step_run_id in self.futures: @@ -100,12 +100,86 @@ class Worker: # send an event that the step run has started try: - event = self.get_action_event(action, STEP_EVENT_TYPE_STARTED) + event = self.get_step_action_event(action, STEP_EVENT_TYPE_STARTED) except Exception as e: logger.error(f"Could not create action event: {e}") # Send the action event to the dispatcher - self.client.dispatcher.send_action_event(event) + self.client.dispatcher.send_step_action_event(event) + + def handle_start_group_key_run(self, action : Action): + action_name = action.action_id # Assuming action object has 'name' attribute + context = Context(action.action_payload) # Assuming action object has 'context' attribute + + self.contexts[action.get_group_key_run_id] = context + + # Find the corresponding action function from the registry + action_func = self.action_registry.get(action_name) + + if action_func: + def callback(future : Future): + errored = False + + # Get the output from the future + try: + output = future.result() + except Exception as e: + errored = True + + # This except is coming from the application itself, so we want to send that to the Hatchet instance + event = self.get_group_key_action_event(action, GROUP_KEY_EVENT_TYPE_FAILED) + event.eventPayload = str(e) + + try: + self.client.dispatcher.send_group_key_action_event(event) + except Exception as e: + logger.error(f"Could not send action event: {e}") + + if not errored: + # Create an action event + try: + event = self.get_group_key_action_finished_event(action, output) + except Exception as e: + logger.error(f"Could not get action finished event: {e}") + raise e + + # Send the action event to the dispatcher + self.client.dispatcher.send_group_key_action_event(event) + + # Remove the future from the dictionary + if action.get_group_key_run_id in self.futures: + del self.futures[action.get_group_key_run_id] + + # Submit the action to the thread pool + def wrapped_action_func(context): + # store the thread id + self.threads[action.get_group_key_run_id] = current_thread() + + try: + res = action_func(context) + return res + except Exception as e: + logger.error(f"Could not execute action: {e}") + raise e + finally: + if action.get_group_key_run_id in self.threads: + # remove the thread id + logger.debug(f"Removing step run id {action.get_group_key_run_id} from threads") + + del self.threads[action.get_group_key_run_id] + + future = self.thread_pool.submit(wrapped_action_func, context) + future.add_done_callback(callback) + self.futures[action.get_group_key_run_id] = future + + # send an event that the step run has started + try: + event = self.get_group_key_action_event(action, GROUP_KEY_EVENT_TYPE_STARTED) + except Exception as e: + logger.error(f"Could not create action event: {e}") + + # Send the action event to the dispatcher + self.client.dispatcher.send_group_key_action_event(event) def force_kill_thread(self, thread): """Terminate a python threading.Thread.""" @@ -136,42 +210,37 @@ class Worker: except Exception as e: logger.exception(f"Failed to terminate thread: {e}") - def handle_cancel_step_run(self, action : Action): - step_run_id = action.step_run_id - + def handle_cancel_action(self, run_id: str): # call cancel to signal the context to stop - context = self.contexts.get(step_run_id) + context = self.contexts.get(run_id) context.cancel() - future = self.futures.get(step_run_id) + future = self.futures.get(run_id) if future: future.cancel() - if step_run_id in self.futures: - del self.futures[step_run_id] + if run_id in self.futures: + del self.futures[run_id] # grace period of 1 second time.sleep(1) # check if thread is still running, if so, kill it - if step_run_id in self.threads: - thread = self.threads[step_run_id] + if run_id in self.threads: + thread = self.threads[run_id] if thread: self.force_kill_thread(thread) - if step_run_id in self.threads: - del self.threads[step_run_id] + if run_id in self.threads: + del self.threads[run_id] - def get_action_event(self, action : Action, event_type : ActionEventType) -> ActionEvent: - # timestamp - # eventTimestamp = datetime.datetime.now(datetime.timezone.utc) - # eventTimestamp = eventTimestamp.isoformat() + def get_step_action_event(self, action : Action, event_type : StepActionEventType) -> StepActionEvent: eventTimestamp = Timestamp() eventTimestamp.GetCurrentTime() - return ActionEvent( + return StepActionEvent( workerId=action.worker_id, jobId=action.job_id, jobRunId=action.job_run_id, @@ -182,9 +251,9 @@ class Worker: eventType=event_type, ) - def get_action_finished_event(self, action : Action, output : Any) -> ActionEvent: + def get_step_action_finished_event(self, action : Action, output : Any) -> StepActionEvent: try: - event = self.get_action_event(action, STEP_EVENT_TYPE_COMPLETED) + event = self.get_step_action_event(action, STEP_EVENT_TYPE_COMPLETED) except Exception as e: logger.error(f"Could not create action finished event: {e}") raise e @@ -198,6 +267,30 @@ class Worker: return event + def get_group_key_action_event(self, action : Action, event_type : GroupKeyActionEventType) -> GroupKeyActionEvent: + eventTimestamp = Timestamp() + eventTimestamp.GetCurrentTime() + + return GroupKeyActionEvent( + workerId=action.worker_id, + workflowRunId=action.workflow_run_id, + getGroupKeyRunId=action.get_group_key_run_id, + actionId=action.action_id, + eventTimestamp=eventTimestamp, + eventType=event_type, + ) + + def get_group_key_action_finished_event(self, action : Action, output : str) -> StepActionEvent: + try: + event = self.get_group_key_action_event(action, GROUP_KEY_EVENT_TYPE_COMPLETED) + except Exception as e: + logger.error(f"Could not create action finished event: {e}") + raise e + + event.eventPayload = output + + return event + def register_workflow(self, workflow : WorkflowMeta): def create_action_function(action_func): def action_function(context): @@ -247,8 +340,10 @@ class Worker: self.handle_start_step_run(action) elif action.action_type == ActionType.CANCEL_STEP_RUN: self.thread_pool.submit(self.handle_cancel_step_run, action) - - pass # Replace this with your actual processing code + elif action.action_type == ActionType.START_GET_GROUP_KEY: + self.handle_start_group_key_run(action) + else: + logger.error(f"Unknown action type: {action.action_type}") except grpc.RpcError as rpc_error: logger.error(f"Could not start worker: {rpc_error}") diff --git a/python-sdk/hatchet_sdk/workflow.py b/python-sdk/hatchet_sdk/workflow.py index d6d39146a..0a124ef2e 100644 --- a/python-sdk/hatchet_sdk/workflow.py +++ b/python-sdk/hatchet_sdk/workflow.py @@ -1,5 +1,5 @@ from .client import new_client -from .workflows_pb2 import CreateWorkflowVersionOpts, CreateWorkflowJobOpts, CreateWorkflowStepOpts +from .workflows_pb2 import CreateWorkflowVersionOpts, CreateWorkflowJobOpts, CreateWorkflowStepOpts, WorkflowConcurrencyOpts from typing import Callable, List, Tuple, Any stepsType = List[Tuple[str, Callable[..., Any]]] @@ -8,6 +8,7 @@ class WorkflowMeta(type): def __new__(cls, name, bases, attrs): serviceName = "default" + concurrencyActions: stepsType = [(name.lower() + "-" + func_name, attrs.pop(func_name)) for func_name, func in list(attrs.items()) if hasattr(func, '_concurrency_fn_name')] steps: stepsType = [(name.lower() + "-" + func_name, attrs.pop(func_name)) for func_name, func in list(attrs.items()) if hasattr(func, '_step_name')] # Define __init__ and get_step_order methods @@ -18,7 +19,10 @@ class WorkflowMeta(type): original_init(self, *args, **kwargs) # Call original __init__ def get_actions(self) -> stepsType: - return [(serviceName + ":" + func_name, func) for func_name, func in steps] + func_actions = [(serviceName + ":" + func_name, func) for func_name, func in steps] + concurrency_actions = [(serviceName + ":" + func_name, func) for func_name, func in concurrencyActions] + + return func_actions + concurrency_actions # Add these methods and steps to class attributes attrs['__init__'] = __init__ @@ -48,6 +52,17 @@ class WorkflowMeta(type): for func_name, func in attrs.items() if hasattr(func, '_step_name') ] + + concurrency : WorkflowConcurrencyOpts | None = None + + if len(concurrencyActions) > 0: + action = concurrencyActions[0] + + concurrency = WorkflowConcurrencyOpts( + action="default:" + action[0], + max_runs=action[1]._concurrency_max_runs, + ) + client.admin.put_workflow(CreateWorkflowVersionOpts( name=name, version=version, @@ -59,7 +74,8 @@ class WorkflowMeta(type): timeout="60s", steps=createStepOpts, ) - ] + ], + concurrency=concurrency, )) return super(WorkflowMeta, cls).__new__(cls, name, bases, attrs) diff --git a/python-sdk/hatchet_sdk/workflows_pb2.py b/python-sdk/hatchet_sdk/workflows_pb2.py index a7806f7ed..cf0234d51 100644 --- a/python-sdk/hatchet_sdk/workflows_pb2.py +++ b/python-sdk/hatchet_sdk/workflows_pb2.py @@ -16,7 +16,7 @@ from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__ from google.protobuf import wrappers_pb2 as google_dot_protobuf_dot_wrappers__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fworkflows.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\">\n\x12PutWorkflowRequest\x12(\n\x04opts\x18\x01 \x01(\x0b\x32\x1a.CreateWorkflowVersionOpts\"\xdc\x01\n\x19\x43reateWorkflowVersionOpts\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0f\n\x07version\x18\x03 \x01(\t\x12\x16\n\x0e\x65vent_triggers\x18\x04 \x03(\t\x12\x15\n\rcron_triggers\x18\x05 \x03(\t\x12\x36\n\x12scheduled_triggers\x18\x06 \x03(\x0b\x32\x1a.google.protobuf.Timestamp\x12$\n\x04jobs\x18\x07 \x03(\x0b\x32\x16.CreateWorkflowJobOpts\"s\n\x15\x43reateWorkflowJobOpts\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0f\n\x07timeout\x18\x03 \x01(\t\x12&\n\x05steps\x18\x04 \x03(\x0b\x32\x17.CreateWorkflowStepOpts\"o\n\x16\x43reateWorkflowStepOpts\x12\x13\n\x0breadable_id\x18\x01 \x01(\t\x12\x0e\n\x06\x61\x63tion\x18\x02 \x01(\t\x12\x0f\n\x07timeout\x18\x03 \x01(\t\x12\x0e\n\x06inputs\x18\x04 \x01(\t\x12\x0f\n\x07parents\x18\x05 \x03(\t\"\x16\n\x14ListWorkflowsRequest\"l\n\x17ScheduleWorkflowRequest\x12\x13\n\x0bworkflow_id\x18\x01 \x01(\t\x12-\n\tschedules\x18\x02 \x03(\x0b\x32\x1a.google.protobuf.Timestamp\x12\r\n\x05input\x18\x03 \x01(\t\"5\n\x15ListWorkflowsResponse\x12\x1c\n\tworkflows\x18\x01 \x03(\x0b\x32\t.Workflow\"1\n\x1cListWorkflowsForEventRequest\x12\x11\n\tevent_key\x18\x01 \x01(\t\"\xee\x01\n\x08Workflow\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x11\n\ttenant_id\x18\x05 \x01(\t\x12\x0c\n\x04name\x18\x06 \x01(\t\x12\x31\n\x0b\x64\x65scription\x18\x07 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\"\n\x08versions\x18\x08 \x03(\x0b\x32\x10.WorkflowVersion\"\xeb\x01\n\x0fWorkflowVersion\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07version\x18\x05 \x01(\t\x12\r\n\x05order\x18\x06 \x01(\x05\x12\x13\n\x0bworkflow_id\x18\x07 \x01(\t\x12#\n\x08triggers\x18\x08 \x01(\x0b\x32\x11.WorkflowTriggers\x12\x12\n\x04jobs\x18\t \x03(\x0b\x32\x04.Job\"\x80\x02\n\x10WorkflowTriggers\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x1b\n\x13workflow_version_id\x18\x05 \x01(\t\x12\x11\n\ttenant_id\x18\x06 \x01(\t\x12(\n\x06\x65vents\x18\x07 \x03(\x0b\x32\x18.WorkflowTriggerEventRef\x12&\n\x05\x63rons\x18\x08 \x03(\x0b\x32\x17.WorkflowTriggerCronRef\"?\n\x17WorkflowTriggerEventRef\x12\x11\n\tparent_id\x18\x01 \x01(\t\x12\x11\n\tevent_key\x18\x02 \x01(\t\"9\n\x16WorkflowTriggerCronRef\x12\x11\n\tparent_id\x18\x01 \x01(\t\x12\x0c\n\x04\x63ron\x18\x02 \x01(\t\"\xa7\x02\n\x03Job\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x11\n\ttenant_id\x18\x05 \x01(\t\x12\x1b\n\x13workflow_version_id\x18\x06 \x01(\t\x12\x0c\n\x04name\x18\x07 \x01(\t\x12\x31\n\x0b\x64\x65scription\x18\x08 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x14\n\x05steps\x18\t \x03(\x0b\x32\x05.Step\x12-\n\x07timeout\x18\n \x01(\x0b\x32\x1c.google.protobuf.StringValue\"\xaa\x02\n\x04Step\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x31\n\x0breadable_id\x18\x05 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x11\n\ttenant_id\x18\x06 \x01(\t\x12\x0e\n\x06job_id\x18\x07 \x01(\t\x12\x0e\n\x06\x61\x63tion\x18\x08 \x01(\t\x12-\n\x07timeout\x18\t \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x0f\n\x07parents\x18\n \x03(\t\x12\x10\n\x08\x63hildren\x18\x0b \x03(\t\",\n\x15\x44\x65leteWorkflowRequest\x12\x13\n\x0bworkflow_id\x18\x01 \x01(\t\"(\n\x18GetWorkflowByNameRequest\x12\x0c\n\x04name\x18\x01 \x01(\t2\x87\x03\n\x0fWorkflowService\x12>\n\rListWorkflows\x12\x15.ListWorkflowsRequest\x1a\x16.ListWorkflowsResponse\x12\x34\n\x0bPutWorkflow\x12\x13.PutWorkflowRequest\x1a\x10.WorkflowVersion\x12>\n\x10ScheduleWorkflow\x12\x18.ScheduleWorkflowRequest\x1a\x10.WorkflowVersion\x12\x39\n\x11GetWorkflowByName\x12\x19.GetWorkflowByNameRequest\x1a\t.Workflow\x12N\n\x15ListWorkflowsForEvent\x12\x1d.ListWorkflowsForEventRequest\x1a\x16.ListWorkflowsResponse\x12\x33\n\x0e\x44\x65leteWorkflow\x12\x16.DeleteWorkflowRequest\x1a\t.WorkflowBBZ@github.com/hatchet-dev/hatchet/internal/services/admin/contractsb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x0fworkflows.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a\x1egoogle/protobuf/wrappers.proto\">\n\x12PutWorkflowRequest\x12(\n\x04opts\x18\x01 \x01(\x0b\x32\x1a.CreateWorkflowVersionOpts\"\x8b\x02\n\x19\x43reateWorkflowVersionOpts\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0f\n\x07version\x18\x03 \x01(\t\x12\x16\n\x0e\x65vent_triggers\x18\x04 \x03(\t\x12\x15\n\rcron_triggers\x18\x05 \x03(\t\x12\x36\n\x12scheduled_triggers\x18\x06 \x03(\x0b\x32\x1a.google.protobuf.Timestamp\x12$\n\x04jobs\x18\x07 \x03(\x0b\x32\x16.CreateWorkflowJobOpts\x12-\n\x0b\x63oncurrency\x18\x08 \x01(\x0b\x32\x18.WorkflowConcurrencyOpts\"n\n\x17WorkflowConcurrencyOpts\x12\x0e\n\x06\x61\x63tion\x18\x01 \x01(\t\x12\x10\n\x08max_runs\x18\x02 \x01(\x05\x12\x31\n\x0elimit_strategy\x18\x03 \x01(\x0e\x32\x19.ConcurrencyLimitStrategy\"s\n\x15\x43reateWorkflowJobOpts\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0f\n\x07timeout\x18\x03 \x01(\t\x12&\n\x05steps\x18\x04 \x03(\x0b\x32\x17.CreateWorkflowStepOpts\"o\n\x16\x43reateWorkflowStepOpts\x12\x13\n\x0breadable_id\x18\x01 \x01(\t\x12\x0e\n\x06\x61\x63tion\x18\x02 \x01(\t\x12\x0f\n\x07timeout\x18\x03 \x01(\t\x12\x0e\n\x06inputs\x18\x04 \x01(\t\x12\x0f\n\x07parents\x18\x05 \x03(\t\"\x16\n\x14ListWorkflowsRequest\"l\n\x17ScheduleWorkflowRequest\x12\x13\n\x0bworkflow_id\x18\x01 \x01(\t\x12-\n\tschedules\x18\x02 \x03(\x0b\x32\x1a.google.protobuf.Timestamp\x12\r\n\x05input\x18\x03 \x01(\t\"5\n\x15ListWorkflowsResponse\x12\x1c\n\tworkflows\x18\x01 \x03(\x0b\x32\t.Workflow\"1\n\x1cListWorkflowsForEventRequest\x12\x11\n\tevent_key\x18\x01 \x01(\t\"\xee\x01\n\x08Workflow\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x11\n\ttenant_id\x18\x05 \x01(\t\x12\x0c\n\x04name\x18\x06 \x01(\t\x12\x31\n\x0b\x64\x65scription\x18\x07 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\"\n\x08versions\x18\x08 \x03(\x0b\x32\x10.WorkflowVersion\"\xeb\x01\n\x0fWorkflowVersion\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x0f\n\x07version\x18\x05 \x01(\t\x12\r\n\x05order\x18\x06 \x01(\x05\x12\x13\n\x0bworkflow_id\x18\x07 \x01(\t\x12#\n\x08triggers\x18\x08 \x01(\x0b\x32\x11.WorkflowTriggers\x12\x12\n\x04jobs\x18\t \x03(\x0b\x32\x04.Job\"\x80\x02\n\x10WorkflowTriggers\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x1b\n\x13workflow_version_id\x18\x05 \x01(\t\x12\x11\n\ttenant_id\x18\x06 \x01(\t\x12(\n\x06\x65vents\x18\x07 \x03(\x0b\x32\x18.WorkflowTriggerEventRef\x12&\n\x05\x63rons\x18\x08 \x03(\x0b\x32\x17.WorkflowTriggerCronRef\"?\n\x17WorkflowTriggerEventRef\x12\x11\n\tparent_id\x18\x01 \x01(\t\x12\x11\n\tevent_key\x18\x02 \x01(\t\"9\n\x16WorkflowTriggerCronRef\x12\x11\n\tparent_id\x18\x01 \x01(\t\x12\x0c\n\x04\x63ron\x18\x02 \x01(\t\"\xa7\x02\n\x03Job\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x11\n\ttenant_id\x18\x05 \x01(\t\x12\x1b\n\x13workflow_version_id\x18\x06 \x01(\t\x12\x0c\n\x04name\x18\x07 \x01(\t\x12\x31\n\x0b\x64\x65scription\x18\x08 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x14\n\x05steps\x18\t \x03(\x0b\x32\x05.Step\x12-\n\x07timeout\x18\n \x01(\x0b\x32\x1c.google.protobuf.StringValue\"\xaa\x02\n\x04Step\x12\n\n\x02id\x18\x01 \x01(\t\x12.\n\ncreated_at\x18\x02 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12.\n\nupdated_at\x18\x03 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x31\n\x0breadable_id\x18\x05 \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x11\n\ttenant_id\x18\x06 \x01(\t\x12\x0e\n\x06job_id\x18\x07 \x01(\t\x12\x0e\n\x06\x61\x63tion\x18\x08 \x01(\t\x12-\n\x07timeout\x18\t \x01(\x0b\x32\x1c.google.protobuf.StringValue\x12\x0f\n\x07parents\x18\n \x03(\t\x12\x10\n\x08\x63hildren\x18\x0b \x03(\t\",\n\x15\x44\x65leteWorkflowRequest\x12\x13\n\x0bworkflow_id\x18\x01 \x01(\t\"(\n\x18GetWorkflowByNameRequest\x12\x0c\n\x04name\x18\x01 \x01(\t*U\n\x18\x43oncurrencyLimitStrategy\x12\x16\n\x12\x43\x41NCEL_IN_PROGRESS\x10\x00\x12\x0f\n\x0b\x44ROP_NEWEST\x10\x01\x12\x10\n\x0cQUEUE_NEWEST\x10\x02\x32\x87\x03\n\x0fWorkflowService\x12>\n\rListWorkflows\x12\x15.ListWorkflowsRequest\x1a\x16.ListWorkflowsResponse\x12\x34\n\x0bPutWorkflow\x12\x13.PutWorkflowRequest\x1a\x10.WorkflowVersion\x12>\n\x10ScheduleWorkflow\x12\x18.ScheduleWorkflowRequest\x1a\x10.WorkflowVersion\x12\x39\n\x11GetWorkflowByName\x12\x19.GetWorkflowByNameRequest\x1a\t.Workflow\x12N\n\x15ListWorkflowsForEvent\x12\x1d.ListWorkflowsForEventRequest\x1a\x16.ListWorkflowsResponse\x12\x33\n\x0e\x44\x65leteWorkflow\x12\x16.DeleteWorkflowRequest\x1a\t.WorkflowBBZ@github.com/hatchet-dev/hatchet/internal/services/admin/contractsb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -24,40 +24,44 @@ _builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'workflows_pb2', _globals) if _descriptor._USE_C_DESCRIPTORS == False: _globals['DESCRIPTOR']._options = None _globals['DESCRIPTOR']._serialized_options = b'Z@github.com/hatchet-dev/hatchet/internal/services/admin/contracts' + _globals['_CONCURRENCYLIMITSTRATEGY']._serialized_start=2549 + _globals['_CONCURRENCYLIMITSTRATEGY']._serialized_end=2634 _globals['_PUTWORKFLOWREQUEST']._serialized_start=84 _globals['_PUTWORKFLOWREQUEST']._serialized_end=146 _globals['_CREATEWORKFLOWVERSIONOPTS']._serialized_start=149 - _globals['_CREATEWORKFLOWVERSIONOPTS']._serialized_end=369 - _globals['_CREATEWORKFLOWJOBOPTS']._serialized_start=371 - _globals['_CREATEWORKFLOWJOBOPTS']._serialized_end=486 - _globals['_CREATEWORKFLOWSTEPOPTS']._serialized_start=488 - _globals['_CREATEWORKFLOWSTEPOPTS']._serialized_end=599 - _globals['_LISTWORKFLOWSREQUEST']._serialized_start=601 - _globals['_LISTWORKFLOWSREQUEST']._serialized_end=623 - _globals['_SCHEDULEWORKFLOWREQUEST']._serialized_start=625 - _globals['_SCHEDULEWORKFLOWREQUEST']._serialized_end=733 - _globals['_LISTWORKFLOWSRESPONSE']._serialized_start=735 - _globals['_LISTWORKFLOWSRESPONSE']._serialized_end=788 - _globals['_LISTWORKFLOWSFOREVENTREQUEST']._serialized_start=790 - _globals['_LISTWORKFLOWSFOREVENTREQUEST']._serialized_end=839 - _globals['_WORKFLOW']._serialized_start=842 - _globals['_WORKFLOW']._serialized_end=1080 - _globals['_WORKFLOWVERSION']._serialized_start=1083 - _globals['_WORKFLOWVERSION']._serialized_end=1318 - _globals['_WORKFLOWTRIGGERS']._serialized_start=1321 - _globals['_WORKFLOWTRIGGERS']._serialized_end=1577 - _globals['_WORKFLOWTRIGGEREVENTREF']._serialized_start=1579 - _globals['_WORKFLOWTRIGGEREVENTREF']._serialized_end=1642 - _globals['_WORKFLOWTRIGGERCRONREF']._serialized_start=1644 - _globals['_WORKFLOWTRIGGERCRONREF']._serialized_end=1701 - _globals['_JOB']._serialized_start=1704 - _globals['_JOB']._serialized_end=1999 - _globals['_STEP']._serialized_start=2002 - _globals['_STEP']._serialized_end=2300 - _globals['_DELETEWORKFLOWREQUEST']._serialized_start=2302 - _globals['_DELETEWORKFLOWREQUEST']._serialized_end=2346 - _globals['_GETWORKFLOWBYNAMEREQUEST']._serialized_start=2348 - _globals['_GETWORKFLOWBYNAMEREQUEST']._serialized_end=2388 - _globals['_WORKFLOWSERVICE']._serialized_start=2391 - _globals['_WORKFLOWSERVICE']._serialized_end=2782 + _globals['_CREATEWORKFLOWVERSIONOPTS']._serialized_end=416 + _globals['_WORKFLOWCONCURRENCYOPTS']._serialized_start=418 + _globals['_WORKFLOWCONCURRENCYOPTS']._serialized_end=528 + _globals['_CREATEWORKFLOWJOBOPTS']._serialized_start=530 + _globals['_CREATEWORKFLOWJOBOPTS']._serialized_end=645 + _globals['_CREATEWORKFLOWSTEPOPTS']._serialized_start=647 + _globals['_CREATEWORKFLOWSTEPOPTS']._serialized_end=758 + _globals['_LISTWORKFLOWSREQUEST']._serialized_start=760 + _globals['_LISTWORKFLOWSREQUEST']._serialized_end=782 + _globals['_SCHEDULEWORKFLOWREQUEST']._serialized_start=784 + _globals['_SCHEDULEWORKFLOWREQUEST']._serialized_end=892 + _globals['_LISTWORKFLOWSRESPONSE']._serialized_start=894 + _globals['_LISTWORKFLOWSRESPONSE']._serialized_end=947 + _globals['_LISTWORKFLOWSFOREVENTREQUEST']._serialized_start=949 + _globals['_LISTWORKFLOWSFOREVENTREQUEST']._serialized_end=998 + _globals['_WORKFLOW']._serialized_start=1001 + _globals['_WORKFLOW']._serialized_end=1239 + _globals['_WORKFLOWVERSION']._serialized_start=1242 + _globals['_WORKFLOWVERSION']._serialized_end=1477 + _globals['_WORKFLOWTRIGGERS']._serialized_start=1480 + _globals['_WORKFLOWTRIGGERS']._serialized_end=1736 + _globals['_WORKFLOWTRIGGEREVENTREF']._serialized_start=1738 + _globals['_WORKFLOWTRIGGEREVENTREF']._serialized_end=1801 + _globals['_WORKFLOWTRIGGERCRONREF']._serialized_start=1803 + _globals['_WORKFLOWTRIGGERCRONREF']._serialized_end=1860 + _globals['_JOB']._serialized_start=1863 + _globals['_JOB']._serialized_end=2158 + _globals['_STEP']._serialized_start=2161 + _globals['_STEP']._serialized_end=2459 + _globals['_DELETEWORKFLOWREQUEST']._serialized_start=2461 + _globals['_DELETEWORKFLOWREQUEST']._serialized_end=2505 + _globals['_GETWORKFLOWBYNAMEREQUEST']._serialized_start=2507 + _globals['_GETWORKFLOWBYNAMEREQUEST']._serialized_end=2547 + _globals['_WORKFLOWSERVICE']._serialized_start=2637 + _globals['_WORKFLOWSERVICE']._serialized_end=3028 # @@protoc_insertion_point(module_scope) diff --git a/python-sdk/hatchet_sdk/workflows_pb2.pyi b/python-sdk/hatchet_sdk/workflows_pb2.pyi index e6dc4ad5c..3b39c45ab 100644 --- a/python-sdk/hatchet_sdk/workflows_pb2.pyi +++ b/python-sdk/hatchet_sdk/workflows_pb2.pyi @@ -1,12 +1,22 @@ from google.protobuf import timestamp_pb2 as _timestamp_pb2 from google.protobuf import wrappers_pb2 as _wrappers_pb2 from google.protobuf.internal import containers as _containers +from google.protobuf.internal import enum_type_wrapper as _enum_type_wrapper from google.protobuf import descriptor as _descriptor from google.protobuf import message as _message from typing import ClassVar as _ClassVar, Iterable as _Iterable, Mapping as _Mapping, Optional as _Optional, Union as _Union DESCRIPTOR: _descriptor.FileDescriptor +class ConcurrencyLimitStrategy(int, metaclass=_enum_type_wrapper.EnumTypeWrapper): + __slots__ = () + CANCEL_IN_PROGRESS: _ClassVar[ConcurrencyLimitStrategy] + DROP_NEWEST: _ClassVar[ConcurrencyLimitStrategy] + QUEUE_NEWEST: _ClassVar[ConcurrencyLimitStrategy] +CANCEL_IN_PROGRESS: ConcurrencyLimitStrategy +DROP_NEWEST: ConcurrencyLimitStrategy +QUEUE_NEWEST: ConcurrencyLimitStrategy + class PutWorkflowRequest(_message.Message): __slots__ = ("opts",) OPTS_FIELD_NUMBER: _ClassVar[int] @@ -14,7 +24,7 @@ class PutWorkflowRequest(_message.Message): def __init__(self, opts: _Optional[_Union[CreateWorkflowVersionOpts, _Mapping]] = ...) -> None: ... class CreateWorkflowVersionOpts(_message.Message): - __slots__ = ("name", "description", "version", "event_triggers", "cron_triggers", "scheduled_triggers", "jobs") + __slots__ = ("name", "description", "version", "event_triggers", "cron_triggers", "scheduled_triggers", "jobs", "concurrency") NAME_FIELD_NUMBER: _ClassVar[int] DESCRIPTION_FIELD_NUMBER: _ClassVar[int] VERSION_FIELD_NUMBER: _ClassVar[int] @@ -22,6 +32,7 @@ class CreateWorkflowVersionOpts(_message.Message): CRON_TRIGGERS_FIELD_NUMBER: _ClassVar[int] SCHEDULED_TRIGGERS_FIELD_NUMBER: _ClassVar[int] JOBS_FIELD_NUMBER: _ClassVar[int] + CONCURRENCY_FIELD_NUMBER: _ClassVar[int] name: str description: str version: str @@ -29,7 +40,18 @@ class CreateWorkflowVersionOpts(_message.Message): cron_triggers: _containers.RepeatedScalarFieldContainer[str] scheduled_triggers: _containers.RepeatedCompositeFieldContainer[_timestamp_pb2.Timestamp] jobs: _containers.RepeatedCompositeFieldContainer[CreateWorkflowJobOpts] - def __init__(self, name: _Optional[str] = ..., description: _Optional[str] = ..., version: _Optional[str] = ..., event_triggers: _Optional[_Iterable[str]] = ..., cron_triggers: _Optional[_Iterable[str]] = ..., scheduled_triggers: _Optional[_Iterable[_Union[_timestamp_pb2.Timestamp, _Mapping]]] = ..., jobs: _Optional[_Iterable[_Union[CreateWorkflowJobOpts, _Mapping]]] = ...) -> None: ... + concurrency: WorkflowConcurrencyOpts + def __init__(self, name: _Optional[str] = ..., description: _Optional[str] = ..., version: _Optional[str] = ..., event_triggers: _Optional[_Iterable[str]] = ..., cron_triggers: _Optional[_Iterable[str]] = ..., scheduled_triggers: _Optional[_Iterable[_Union[_timestamp_pb2.Timestamp, _Mapping]]] = ..., jobs: _Optional[_Iterable[_Union[CreateWorkflowJobOpts, _Mapping]]] = ..., concurrency: _Optional[_Union[WorkflowConcurrencyOpts, _Mapping]] = ...) -> None: ... + +class WorkflowConcurrencyOpts(_message.Message): + __slots__ = ("action", "max_runs", "limit_strategy") + ACTION_FIELD_NUMBER: _ClassVar[int] + MAX_RUNS_FIELD_NUMBER: _ClassVar[int] + LIMIT_STRATEGY_FIELD_NUMBER: _ClassVar[int] + action: str + max_runs: int + limit_strategy: ConcurrencyLimitStrategy + def __init__(self, action: _Optional[str] = ..., max_runs: _Optional[int] = ..., limit_strategy: _Optional[_Union[ConcurrencyLimitStrategy, str]] = ...) -> None: ... class CreateWorkflowJobOpts(_message.Message): __slots__ = ("name", "description", "timeout", "steps") diff --git a/python-sdk/pyproject.toml b/python-sdk/pyproject.toml index 8ee08157b..7b97f3eb2 100644 --- a/python-sdk/pyproject.toml +++ b/python-sdk/pyproject.toml @@ -1,6 +1,6 @@ [tool.poetry] name = "hatchet-sdk" -version = "0.6.0" +version = "0.7.0" description = "" authors = ["Alexander Belanger "] readme = "README.md"