feat: support maxRuns parameter on workers (#195)

* feat: round robin queueing

* feat: configurable max runs per worker

* fix: address PR review

* docs for max runs and group round robin
This commit is contained in:
abelanger5
2024-02-25 21:48:46 -08:00
committed by GitHub
parent 2d625fec81
commit 6ea38a99f2
46 changed files with 659 additions and 373 deletions

View File

@@ -29,6 +29,9 @@ message WorkerRegisterRequest {
// (optional) the services for this worker
repeated string services = 3;
// (optional) the max number of runs this worker can handle
optional int32 maxRuns = 4;
}
message WorkerRegisterResponse {

View File

@@ -24,11 +24,12 @@ func (t *StepRunService) StepRunUpdateRerun(ctx echo.Context, request gen.StepRu
// preflight check to make sure there's at least one worker to serve this request
action := stepRun.Step().ActionID
tenSecAgo := time.Now().Add(-10 * time.Second)
sixSecAgo := time.Now().Add(-6 * time.Second)
workers, err := t.config.Repository.Worker().ListWorkers(tenant.ID, &repository.ListWorkersOpts{
Action: &action,
LastHeartbeatAfter: &tenSecAgo,
LastHeartbeatAfter: &sixSecAgo,
Assignable: repository.BoolPtr(true),
})
if err != nil || len(workers) == 0 {

View File

@@ -1,6 +1,8 @@
package workers
import (
"time"
"github.com/labstack/echo/v4"
"github.com/hatchet-dev/hatchet/api/v1/server/oas/gen"
@@ -12,7 +14,11 @@ import (
func (t *WorkerService) WorkerList(ctx echo.Context, request gen.WorkerListRequestObject) (gen.WorkerListResponseObject, error) {
tenant := ctx.Get("tenant").(*db.TenantModel)
workers, err := t.config.Repository.Worker().ListWorkers(tenant.ID, &repository.ListWorkersOpts{})
sixSecAgo := time.Now().Add(-6 * time.Second)
workers, err := t.config.Repository.Worker().ListWorkers(tenant.ID, &repository.ListWorkersOpts{
LastHeartbeatAfter: &sixSecAgo,
})
if err != nil {
return nil, err
@@ -22,7 +28,7 @@ func (t *WorkerService) WorkerList(ctx echo.Context, request gen.WorkerListReque
for i, worker := range workers {
workerCp := worker
rows[i] = *transformers.ToWorker(workerCp.Worker)
rows[i] = *transformers.ToWorkerSqlc(&workerCp.Worker)
}
return gen.WorkerList200JSONResponse(

View File

@@ -3,6 +3,7 @@ package transformers
import (
"github.com/hatchet-dev/hatchet/api/v1/server/oas/gen"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
)
func ToWorker(worker *db.WorkerModel) *gen.Worker {
@@ -29,3 +30,16 @@ func ToWorker(worker *db.WorkerModel) *gen.Worker {
return res
}
func ToWorkerSqlc(worker *dbsqlc.Worker) *gen.Worker {
res := &gen.Worker{
Metadata: *toAPIMetadata(pgUUIDToStr(worker.ID), worker.CreatedAt.Time, worker.UpdatedAt.Time),
Name: worker.Name,
}
if !worker.LastHeartbeatAt.Time.IsZero() {
res.LastHeartbeatAt = &worker.LastHeartbeatAt.Time
}
return res
}

View File

@@ -46,6 +46,7 @@ func run(ch <-chan interface{}, events chan<- string) error {
worker.WithClient(
c,
),
worker.WithMaxRuns(1),
)
if err != nil {
return fmt.Errorf("error creating worker: %w", err)

View File

@@ -57,6 +57,10 @@ The client to use to communicate with the Hatchet instance. This is required.
The name of the worker. This is used to identify the worker in the Hatchet UI.
### `worker.WithMaxRuns`
The maximum number of runs the worker can process simultaneously.
### `worker.WithErrorAlerter`
Use this option to set up an external error alerter, such as [Sentry](https://sentry.io/).

View File

@@ -24,6 +24,6 @@ worker.start()
The `hatchet.worker()` method takes a number of options:
- `name`: The name of the worker. This is used to identify the worker in the Hatchet UI.
- `max_threads`: The maximum number of threads to use for the worker. Defaults to 200.
- `max_runs`: The maximum number of concurrent runs that the worker can run. Defaults to `None` (unlimited runs).
- `debug`: Whether to enable debug logging. Defaults to `False`.
- `handle_kill`: Whether to call `sys.exit()` when the worker receives a `SIGTERM` signal after graceful termination. Defaults to `True`.

View File

@@ -23,4 +23,14 @@ main();
## Options
### Name
The `hatchet.worker()` method takes a simple name parameter which can be used to identify the worker on the Hatchet dashboard.
### Max Runs
The `maxRuns` option can be used to limit the number of runs a worker will process before stopping. This is particularly useful for resource-intensive workers. For example, to limit the worker to only executing 1 run at a time, you can use the following code:
```ts
hatchet.worker('example-worker', 1)
```

View File

@@ -186,6 +186,8 @@ const workflow: Workflow = {
};
```
The argument `limitStrategy` to the `concurrency` configuration can be set to either `CANCEL_IN_PROGRESS` (the default, documented above), or `GROUP_ROUND_ROBIN`. See documentation for the `GROUP_ROUND_ROBIN` strategy below.
### Cancellation Signalling
When a concurrent workflow is already running, Hatchet will send a cancellation signal to the step via it's context. For now, you must handle this signal to exit the step at a logical point:
@@ -239,3 +241,36 @@ This same approach can be used for:
- Setting concurrency for a specific user session by `session_id` (i.e. multiple chat messages sent)
- Limiting data or document ingestion by setting an input hash or on-file key.
- Rudimentary fairness rules by limiting groups per tenant to a certain number of concurrent executions.
### Use-Case: Group Round Robin
You can distribute workflows fairly between tenants using the `GROUP_ROUND_ROBIN` option for `limitStrategy`. This will ensure that each distinct group gets a fair share of the concurrency limit. For example, let's say 5 workflows got queued in quick succession for keys `A`, `B`, and `C`:
```txt
A, A, A, A, A, B, B, B, B, B, C, C, C, C, C
```
If there is a maximum of 2 concurrent executions, the execution order will be:
```txt
A, B, C, A, B, C, A, B, C, A, B, C, A, B, C
```
This can be set in the `concurrency` configuration as follows:
```ts
const workflow: Workflow = {
id: 'concurrency-example-rr',
description: 'test',
on: {
event: 'concurrency:create',
},
concurrency: {
name: 'multi-tenant-fairness',
key: (ctx) => ctx.workflowInput().group,
maxRuns: 2,
limitStrategy: ConcurrencyLimitStrategy.GROUP_ROUND_ROBIN,
},
steps: [...],
};
```

1
go.mod
View File

@@ -100,6 +100,7 @@ require (
github.com/hashicorp/errwrap v1.0.0 // indirect
github.com/hashicorp/hcl v1.0.0 // indirect
github.com/invopop/jsonschema v0.12.0
github.com/jackc/pgx-zerolog v0.0.0-20230315001418-f978528409eb
github.com/magiconair/properties v1.8.7 // indirect
github.com/mattn/go-colorable v0.1.13 // indirect
github.com/mattn/go-isatty v0.0.20 // indirect

2
go.sum
View File

@@ -245,6 +245,8 @@ github.com/jackc/pgpassfile v1.0.0 h1:/6Hmqy13Ss2zCq62VdNG8tM1wchn8zjSGOBJ6icpsI
github.com/jackc/pgpassfile v1.0.0/go.mod h1:CEx0iS5ambNFdcRtxPj5JhEz+xB6uRky5eyVu/W2HEg=
github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a h1:bbPeKD0xmW/Y25WS6cokEszi5g+S0QxI/d45PkRi7Nk=
github.com/jackc/pgservicefile v0.0.0-20221227161230-091c0ba34f0a/go.mod h1:5TJZWKEWniPve33vlWYSoGYefn3gLQRzjfDlhSJ9ZKM=
github.com/jackc/pgx-zerolog v0.0.0-20230315001418-f978528409eb h1:pSv+zRVeAYjbXRFjyytFIMRBSKWVowCi7KbXSMR/+ug=
github.com/jackc/pgx-zerolog v0.0.0-20230315001418-f978528409eb/go.mod h1:CRUuPsmIajLt3dZIlJ5+O8IDSib6y8yrst8DkCthTa4=
github.com/jackc/pgx/v5 v5.5.0 h1:NxstgwndsTRy7eq9/kqYc/BZh5w2hHJV86wjvO+1xPw=
github.com/jackc/pgx/v5 v5.5.0/go.mod h1:Ig06C2Vu0t5qXC60W8sqIthScaEnFvojjj9dSljmHRA=
github.com/jackc/puddle/v2 v2.2.1 h1:RhxXJtFG022u4ibrCSMSiu5aOq1i77R3OHKNJj77OAk=

View File

@@ -18,6 +18,8 @@ type ConfigFile struct {
Seed SeedConfigFile `mapstructure:"seed" json:"seed,omitempty"`
Logger shared.LoggerConfigFile `mapstructure:"logger" json:"logger,omitempty"`
LogQueries bool `mapstructure:"logQueries" json:"logQueries,omitempty" default:"false"`
}
type SeedConfigFile struct {

View File

@@ -10,6 +10,9 @@ import (
"strings"
"github.com/jackc/pgx/v5/pgxpool"
"github.com/jackc/pgx/v5/tracelog"
pgxzero "github.com/jackc/pgx-zerolog"
"github.com/hatchet-dev/hatchet/internal/auth/cookie"
"github.com/hatchet-dev/hatchet/internal/auth/oauth"
@@ -130,6 +133,13 @@ func GetDatabaseConfigFromConfigFile(cf *database.ConfigFile) (res *database.Con
return nil, err
}
if cf.LogQueries {
config.ConnConfig.Tracer = &tracelog.TraceLog{
Logger: pgxzero.NewLogger(l),
LogLevel: tracelog.LogLevelDebug,
}
}
config.MaxConns = 20
pool, err := pgxpool.NewWithConfig(context.Background(), config)

View File

@@ -726,6 +726,7 @@ type Worker struct {
Name string `json:"name"`
Status WorkerStatus `json:"status"`
DispatcherId pgtype.UUID `json:"dispatcherId"`
MaxRuns pgtype.Int4 `json:"maxRuns"`
}
type Workflow struct {

View File

@@ -429,6 +429,7 @@ CREATE TABLE "Worker" (
"name" TEXT NOT NULL,
"status" "WorkerStatus" NOT NULL DEFAULT 'ACTIVE',
"dispatcherId" UUID NOT NULL,
"maxRuns" INTEGER,
CONSTRAINT "Worker_pkey" PRIMARY KEY ("id")
);

View File

@@ -12,6 +12,7 @@ sql:
- job_runs.sql
- tickers.sql
- dispatchers.sql
- workers.sql
schema:
- schema.sql
strict_order_by: false

View File

@@ -0,0 +1,34 @@
-- name: ListWorkersWithStepCount :many
SELECT
sqlc.embed(workers),
COUNT(runs."id") FILTER (WHERE runs."status" = 'RUNNING') AS "runningStepRuns"
FROM
"Worker" workers
LEFT JOIN
"StepRun" AS runs ON runs."workerId" = workers."id" AND runs."status" = 'RUNNING'
WHERE
workers."tenantId" = @tenantId
AND (
sqlc.narg('actionId')::text IS NULL OR
workers."id" IN (
SELECT "_ActionToWorker"."B"
FROM "_ActionToWorker"
INNER JOIN "Action" ON "Action"."id" = "_ActionToWorker"."A"
WHERE "Action"."tenantId" = @tenantId AND "Action"."actionId" = sqlc.narg('actionId')::text
)
)
AND (
sqlc.narg('lastHeartbeatAfter')::timestamp IS NULL OR
workers."lastHeartbeatAt" > sqlc.narg('lastHeartbeatAfter')::timestamp
)
AND (
sqlc.narg('assignable')::boolean IS NULL OR
workers."maxRuns" IS NULL OR
(sqlc.narg('assignable')::boolean AND workers."maxRuns" > (
SELECT COUNT(*)
FROM "StepRun"
WHERE runs."workerId" = workers."id" AND runs."status" = 'RUNNING'
))
)
GROUP BY
workers."id";

View File

@@ -0,0 +1,97 @@
// Code generated by sqlc. DO NOT EDIT.
// versions:
// sqlc v1.24.0
// source: workers.sql
package dbsqlc
import (
"context"
"github.com/jackc/pgx/v5/pgtype"
)
const listWorkersWithStepCount = `-- name: ListWorkersWithStepCount :many
SELECT
workers.id, workers."createdAt", workers."updatedAt", workers."deletedAt", workers."tenantId", workers."lastHeartbeatAt", workers.name, workers.status, workers."dispatcherId", workers."maxRuns",
COUNT(runs."id") FILTER (WHERE runs."status" = 'RUNNING') AS "runningStepRuns"
FROM
"Worker" workers
LEFT JOIN
"StepRun" AS runs ON runs."workerId" = workers."id" AND runs."status" = 'RUNNING'
WHERE
workers."tenantId" = $1
AND (
$2::text IS NULL OR
workers."id" IN (
SELECT "_ActionToWorker"."B"
FROM "_ActionToWorker"
INNER JOIN "Action" ON "Action"."id" = "_ActionToWorker"."A"
WHERE "Action"."tenantId" = $1 AND "Action"."actionId" = $2::text
)
)
AND (
$3::timestamp IS NULL OR
workers."lastHeartbeatAt" > $3::timestamp
)
AND (
$4::boolean IS NULL OR
workers."maxRuns" IS NULL OR
($4::boolean AND workers."maxRuns" > (
SELECT COUNT(*)
FROM "StepRun"
WHERE runs."workerId" = workers."id" AND runs."status" = 'RUNNING'
))
)
GROUP BY
workers."id"
`
type ListWorkersWithStepCountParams struct {
Tenantid pgtype.UUID `json:"tenantid"`
ActionId pgtype.Text `json:"actionId"`
LastHeartbeatAfter pgtype.Timestamp `json:"lastHeartbeatAfter"`
Assignable pgtype.Bool `json:"assignable"`
}
type ListWorkersWithStepCountRow struct {
Worker Worker `json:"worker"`
RunningStepRuns int64 `json:"runningStepRuns"`
}
func (q *Queries) ListWorkersWithStepCount(ctx context.Context, db DBTX, arg ListWorkersWithStepCountParams) ([]*ListWorkersWithStepCountRow, error) {
rows, err := db.Query(ctx, listWorkersWithStepCount,
arg.Tenantid,
arg.ActionId,
arg.LastHeartbeatAfter,
arg.Assignable,
)
if err != nil {
return nil, err
}
defer rows.Close()
var items []*ListWorkersWithStepCountRow
for rows.Next() {
var i ListWorkersWithStepCountRow
if err := rows.Scan(
&i.Worker.ID,
&i.Worker.CreatedAt,
&i.Worker.UpdatedAt,
&i.Worker.DeletedAt,
&i.Worker.TenantId,
&i.Worker.LastHeartbeatAt,
&i.Worker.Name,
&i.Worker.Status,
&i.Worker.DispatcherId,
&i.Worker.MaxRuns,
&i.RunningStepRuns,
); err != nil {
return nil, err
}
items = append(items, &i)
}
if err := rows.Err(); err != nil {
return nil, err
}
return items, nil
}

View File

@@ -76,7 +76,7 @@ func NewPrismaRepository(client *db.PrismaClient, pool *pgxpool.Pool, fs ...Pris
github: NewGithubRepository(client, opts.v),
step: NewStepRepository(client, opts.v),
dispatcher: NewDispatcherRepository(client, pool, opts.v, opts.l),
worker: NewWorkerRepository(client, opts.v),
worker: NewWorkerRepository(client, pool, opts.v, opts.l),
ticker: NewTickerRepository(client, pool, opts.v, opts.l),
userSession: NewUserSessionRepository(client, opts.v),
user: NewUserRepository(client, opts.v),

View File

@@ -2,27 +2,40 @@ package prisma
import (
"context"
"errors"
"fmt"
"strconv"
"strings"
"github.com/google/uuid"
"github.com/jackc/pgx/v5"
"github.com/jackc/pgx/v5/pgtype"
"github.com/jackc/pgx/v5/pgxpool"
"github.com/rs/zerolog"
"github.com/steebchen/prisma-client-go/runtime/transaction"
"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 workerRepository struct {
client *db.PrismaClient
v validator.Validator
client *db.PrismaClient
pool *pgxpool.Pool
v validator.Validator
queries *dbsqlc.Queries
l *zerolog.Logger
}
func NewWorkerRepository(client *db.PrismaClient, v validator.Validator) repository.WorkerRepository {
func NewWorkerRepository(client *db.PrismaClient, pool *pgxpool.Pool, v validator.Validator, l *zerolog.Logger) repository.WorkerRepository {
queries := dbsqlc.New()
return &workerRepository{
client: client,
v: v,
client: client,
pool: pool,
v: v,
queries: queries,
l: l,
}
}
@@ -56,103 +69,57 @@ func (w *workerRepository) ListRecentWorkerStepRuns(tenantId, workerId string) (
).Exec(context.Background())
}
func (w *workerRepository) ListWorkers(tenantId string, opts *repository.ListWorkersOpts) ([]repository.WorkerWithStepCount, error) {
if err := w.v.Validate(opts); err != nil {
func (r *workerRepository) ListWorkers(tenantId string, opts *repository.ListWorkersOpts) ([]*dbsqlc.ListWorkersWithStepCountRow, error) {
if err := r.v.Validate(opts); err != nil {
return nil, err
}
queryParams := []db.WorkerWhereParam{
db.Worker.TenantID.Equals(tenantId),
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
queryParams := dbsqlc.ListWorkersWithStepCountParams{
Tenantid: pgTenantId,
}
if opts.Action != nil {
queryParams = append(queryParams, db.Worker.Actions.Some(
db.Action.TenantID.Equals(tenantId),
db.Action.ActionID.Equals(*opts.Action),
))
queryParams.ActionId = sqlchelpers.TextFromStr(*opts.Action)
}
if opts.LastHeartbeatAfter != nil {
queryParams = append(queryParams, db.Worker.LastHeartbeatAt.After(*opts.LastHeartbeatAfter))
queryParams.LastHeartbeatAfter = sqlchelpers.TimestampFromTime(opts.LastHeartbeatAfter.UTC())
}
workers, err := w.client.Worker.FindMany(
queryParams...,
).With(
db.Worker.Dispatcher.Fetch(),
).Exec(context.Background())
if err != nil {
return nil, err
}
if len(workers) == 0 {
return []repository.WorkerWithStepCount{}, nil
}
workerIds := make([]string, len(workers))
for i, worker := range workers {
workerIds[i] = worker.ID
}
var rows []struct {
ID string `json:"id"`
Count string `json:"count"`
}
workerIdStrs := make([]string, len(workerIds))
for i, workerId := range workerIds {
// verify that the worker id is a valid uuid
if _, err := uuid.Parse(workerId); err != nil {
return nil, err
if opts.Assignable != nil {
queryParams.Assignable = pgtype.Bool{
Bool: *opts.Assignable,
Valid: true,
}
workerIdStrs[i] = fmt.Sprintf("'%s'", workerId)
}
workerIdsStr := strings.Join(workerIdStrs, ",")
// raw query to get the number of active job runs for each worker
err = w.client.Prisma.QueryRaw(
fmt.Sprintf(`
SELECT "Worker"."id" AS id, COUNT("StepRun"."id") AS count
FROM "Worker"
LEFT JOIN "StepRun" ON "StepRun"."workerId" = "Worker"."id" AND "StepRun"."status" = 'RUNNING'
WHERE "Worker"."tenantId"::text = $1 AND "Worker"."id" IN (%s)
GROUP BY "Worker"."id"
`, workerIdsStr),
tenantId, workerIds,
).Exec(context.Background(), &rows)
tx, err := r.pool.Begin(context.Background())
if err != nil {
return nil, err
}
workerMap := make(map[string]int)
defer deferRollback(context.Background(), r.l, tx.Rollback)
for _, row := range rows {
stepCount, err := strconv.ParseInt(row.Count, 10, 64)
workers, err := r.queries.ListWorkersWithStepCount(context.Background(), tx, queryParams)
if err == nil {
workerMap[row.ID] = int(stepCount)
if err != nil {
if errors.Is(err, pgx.ErrNoRows) {
workers = make([]*dbsqlc.ListWorkersWithStepCountRow, 0)
} else {
workerMap[row.ID] = 0
return nil, fmt.Errorf("could not list events: %w", err)
}
}
res := make([]repository.WorkerWithStepCount, len(workers))
err = tx.Commit(context.Background())
for i, worker := range workers {
workerCp := worker
res[i] = repository.WorkerWithStepCount{
Worker: &workerCp,
StepRunCount: workerMap[worker.ID],
}
if err != nil {
return nil, fmt.Errorf("could not commit transaction: %w", err)
}
return res, nil
return workers, nil
}
func (w *workerRepository) CreateNewWorker(tenantId string, opts *repository.CreateWorkerOpts) (*db.WorkerModel, error) {
@@ -173,6 +140,7 @@ func (w *workerRepository) CreateNewWorker(tenantId string, opts *repository.Cre
db.Dispatcher.ID.Equals(opts.DispatcherId),
),
db.Worker.ID.Set(workerId),
db.Worker.MaxRuns.SetIfPresent(opts.MaxRuns),
).Tx()
txs = append(txs, createTx)

View File

@@ -4,12 +4,16 @@ import (
"time"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
)
type CreateWorkerOpts struct {
// The id of the dispatcher
DispatcherId string `validate:"required,uuid"`
// The maximum number of runs this worker can run at a time
MaxRuns *int `validate:"omitempty,gte=1"`
// The name of the worker
Name string `validate:"required,hatchetName"`
@@ -43,11 +47,13 @@ type ListWorkersOpts struct {
Action *string `validate:"omitempty,actionId"`
LastHeartbeatAfter *time.Time
Assignable *bool
}
type WorkerRepository interface {
// ListWorkers lists workers for the tenant
ListWorkers(tenantId string, opts *ListWorkersOpts) ([]WorkerWithStepCount, error)
ListWorkers(tenantId string, opts *ListWorkersOpts) ([]*dbsqlc.ListWorkersWithStepCountRow, error)
// ListRecentWorkerStepRuns lists recent step runs for a given worker
ListRecentWorkerStepRuns(tenantId, workerId string) ([]db.StepRunModel, error)

View File

@@ -678,6 +678,7 @@ func (ec *JobsControllerImpl) scheduleStepRun(ctx context.Context, tenantId, ste
workers, err := ec.repo.Worker().ListWorkers(tenantId, &repository.ListWorkersOpts{
Action: &stepRun.Step().ActionID,
LastHeartbeatAfter: &after,
Assignable: repository.BoolPtr(true),
})
if err != nil {
@@ -693,15 +694,17 @@ func (ec *JobsControllerImpl) scheduleStepRun(ctx context.Context, tenantId, ste
selectedWorker := workers[0]
for _, worker := range workers {
if worker.StepRunCount < selectedWorker.StepRunCount {
if worker.RunningStepRuns < selectedWorker.RunningStepRuns {
selectedWorker = worker
}
}
telemetry.WithAttributes(span, servertel.WorkerId(selectedWorker.Worker.ID))
selectedWorkerId := sqlchelpers.UUIDToStr(selectedWorker.Worker.ID)
telemetry.WithAttributes(span, servertel.WorkerId(selectedWorkerId))
// update the job run's designated worker
err = ec.repo.Worker().AddStepRun(tenantId, selectedWorker.Worker.ID, stepRunId)
err = ec.repo.Worker().AddStepRun(tenantId, selectedWorkerId, stepRunId)
if err != nil {
return fmt.Errorf("could not add step run to worker: %w", err)
@@ -728,11 +731,13 @@ func (ec *JobsControllerImpl) scheduleStepRun(ctx context.Context, tenantId, ste
return fmt.Errorf("could not schedule step run timeout task: %w", err)
}
dispatcherId := sqlchelpers.UUIDToStr(selectedWorker.Worker.DispatcherId)
// send a task to the dispatcher
err = ec.tq.AddTask(
ctx,
taskqueue.QueueTypeFromDispatcherID(selectedWorker.Worker.Dispatcher().ID),
stepRunAssignedTask(tenantId, stepRunId, selectedWorker.Worker),
taskqueue.QueueTypeFromDispatcherID(dispatcherId),
stepRunAssignedTask(tenantId, stepRunId, selectedWorkerId, dispatcherId),
)
if err != nil {
@@ -1215,17 +1220,15 @@ func (ec *JobsControllerImpl) getValidTickers() ([]db.TickerModel, error) {
return tickers, nil
}
func stepRunAssignedTask(tenantId, stepRunId string, worker *db.WorkerModel) *taskqueue.Task {
dispatcher := worker.Dispatcher()
func stepRunAssignedTask(tenantId, stepRunId, workerId, dispatcherId string) *taskqueue.Task {
payload, _ := datautils.ToJSONMap(tasktypes.StepRunAssignedTaskPayload{
StepRunId: stepRunId,
WorkerId: worker.ID,
WorkerId: workerId,
})
metadata, _ := datautils.ToJSONMap(tasktypes.StepRunAssignedTaskMetadata{
TenantId: tenantId,
DispatcherId: dispatcher.ID,
DispatcherId: dispatcherId,
})
return &taskqueue.Task{

View File

@@ -169,6 +169,7 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
workers, err := wc.repo.Worker().ListWorkers(workflowRun.TenantID, &repository.ListWorkersOpts{
Action: &getAction.ActionID,
LastHeartbeatAfter: &after,
Assignable: repository.BoolPtr(true),
})
if err != nil {
@@ -184,15 +185,17 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
selectedWorker := workers[0]
for _, worker := range workers {
if worker.StepRunCount < selectedWorker.StepRunCount {
if worker.RunningStepRuns < selectedWorker.RunningStepRuns {
selectedWorker = worker
}
}
telemetry.WithAttributes(span, servertel.WorkerId(selectedWorker.Worker.ID))
selectedWorkerId := sqlchelpers.UUIDToStr(selectedWorker.Worker.ID)
telemetry.WithAttributes(span, servertel.WorkerId(selectedWorkerId))
// update the job run's designated worker
err = wc.repo.Worker().AddGetGroupKeyRun(tenantId, selectedWorker.Worker.ID, getGroupKeyRun.ID)
err = wc.repo.Worker().AddGetGroupKeyRun(tenantId, selectedWorkerId, getGroupKeyRun.ID)
if err != nil {
return fmt.Errorf("could not add step run to worker: %w", err)
@@ -219,11 +222,18 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
return fmt.Errorf("could not schedule step run timeout task: %w", err)
}
dispatcherId := sqlchelpers.UUIDToStr(selectedWorker.Worker.DispatcherId)
// send a task to the dispatcher
err = wc.tq.AddTask(
ctx,
taskqueue.QueueTypeFromDispatcherID(selectedWorker.Worker.Dispatcher().ID),
getGroupActionTask(workflowRun.TenantID, workflowRun.ID, selectedWorker.Worker),
taskqueue.QueueTypeFromDispatcherID(dispatcherId),
getGroupActionTask(
workflowRun.TenantID,
workflowRun.ID,
selectedWorkerId,
dispatcherId,
),
)
if err != nil {
@@ -534,17 +544,15 @@ func (wc *WorkflowsControllerImpl) cancelWorkflowRun(tenantId, workflowRunId str
return errGroup.Wait()
}
func getGroupActionTask(tenantId, workflowRunId string, worker *db.WorkerModel) *taskqueue.Task {
dispatcher := worker.Dispatcher()
func getGroupActionTask(tenantId, workflowRunId, workerId, dispatcherId string) *taskqueue.Task {
payload, _ := datautils.ToJSONMap(tasktypes.GroupKeyActionAssignedTaskPayload{
WorkflowRunId: workflowRunId,
WorkerId: worker.ID,
WorkerId: workerId,
})
metadata, _ := datautils.ToJSONMap(tasktypes.GroupKeyActionAssignedTaskMetadata{
TenantId: tenantId,
DispatcherId: dispatcher.ID,
DispatcherId: dispatcherId,
})
return &taskqueue.Task{

View File

@@ -292,6 +292,8 @@ type WorkerRegisterRequest struct {
Actions []string `protobuf:"bytes,2,rep,name=actions,proto3" json:"actions,omitempty"`
// (optional) the services for this worker
Services []string `protobuf:"bytes,3,rep,name=services,proto3" json:"services,omitempty"`
// (optional) the max number of runs this worker can handle
MaxRuns *int32 `protobuf:"varint,4,opt,name=maxRuns,proto3,oneof" json:"maxRuns,omitempty"`
}
func (x *WorkerRegisterRequest) Reset() {
@@ -347,6 +349,13 @@ func (x *WorkerRegisterRequest) GetServices() []string {
return nil
}
func (x *WorkerRegisterRequest) GetMaxRuns() int32 {
if x != nil && x.MaxRuns != nil {
return *x.MaxRuns
}
return 0
}
type WorkerRegisterResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -1246,216 +1255,219 @@ var file_dispatcher_proto_rawDesc = []byte{
0x0a, 0x10, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x22, 0x6d, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67,
0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1e, 0x0a, 0x0a,
0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07,
0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x61,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63,
0x65, 0x73, 0x22, 0x70, 0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69,
0x73, 0x74, 0x65, 0x72, 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, 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, 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, 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,
0x6f, 0x74, 0x6f, 0x22, 0x98, 0x01, 0x0a, 0x15, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65,
0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1e, 0x0a,
0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
0x09, 0x52, 0x0a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a,
0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07,
0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x72, 0x76, 0x69,
0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x69,
0x63, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x07, 0x6d, 0x61, 0x78, 0x52, 0x75, 0x6e, 0x73, 0x18, 0x04,
0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x07, 0x6d, 0x61, 0x78, 0x52, 0x75, 0x6e, 0x73, 0x88,
0x01, 0x01, 0x42, 0x0a, 0x0a, 0x08, 0x5f, 0x6d, 0x61, 0x78, 0x52, 0x75, 0x6e, 0x73, 0x22, 0x70,
0x0a, 0x16, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72,
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,
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, 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,
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, 0x05, 0x20, 0x01, 0x28, 0x0b,
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, 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, 0x22, 0x48, 0x0a, 0x20, 0x53, 0x75, 0x62, 0x73, 0x63,
0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77,
0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49,
0x64, 0x22, 0xba, 0x02, 0x0a, 0x0d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b,
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x0c, 0x72, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x0d, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x09,
0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x12, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54,
0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e,
0x0a, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x42,
0x0a, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70,
0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x22, 0x7f,
0x0a, 0x0d, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12,
0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01,
0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a,
0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74,
0x68, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65,
0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x22,
0x17, 0x0a, 0x15, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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,
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, 0x22, 0x48, 0x0a, 0x20, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66,
0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d,
0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x22, 0xba, 0x02,
0x0a, 0x0d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12,
0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64,
0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0d, 0x2e, 0x52, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e,
0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x12, 0x2e, 0x52, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52,
0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x72, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x42, 0x0a, 0x0e, 0x65, 0x76,
0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e,
0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22,
0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06,
0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01,
0x28, 0x08, 0x52, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x22, 0x7f, 0x0a, 0x0d, 0x4f, 0x76,
0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x73,
0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09,
0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74,
0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a,
0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61,
0x6c, 0x75, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c,
0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c,
0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4f,
0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 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, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54,
0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12,
0x1d, 0x0a, 0x19, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1a,
0x0a, 0x16, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50,
0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x65, 0x0a, 0x0c, 0x52, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x45,
0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e,
0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43,
0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10,
0x01, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x52, 0x55, 0x4e, 0x10,
0x02, 0x2a, 0xde, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76,
0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55,
0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55,
0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f,
0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f,
0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54,
0x45, 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45,
0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d,
0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x47, 0x52, 0x4f, 0x55,
0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,
0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x8a, 0x01, 0x0a, 0x13, 0x53, 0x74,
0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
0x65, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f,
0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1b,
0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50,
0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x53,
0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43,
0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x53, 0x54,
0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41,
0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x65, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52,
0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10,
0x00, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x1e, 0x0a,
0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57,
0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x02, 0x2a, 0xde, 0x01,
0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54,
0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f,
0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f,
0x57, 0x4e, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45,
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52,
0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43,
0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d,
0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f,
0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53,
0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,
0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a,
0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54,
0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d,
0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54,
0x59, 0x50, 0x45, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12,
0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e,
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54,
0x10, 0x05, 0x32, 0xe4, 0x03, 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, 0x52, 0x0a, 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e,
0x74, 0x73, 0x12, 0x21, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f,
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3f, 0x0a, 0x13, 0x53, 0x65, 0x6e,
0x64, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74,
0x12, 0x10, 0x2e, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65,
0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a, 0x17, 0x53, 0x65,
0x6e, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x14, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79,
0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x00, 0x12, 0x3c, 0x0a, 0x10, 0x50, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69,
0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x0e, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69,
0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x16, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69,
0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x00, 0x12, 0x46, 0x0a, 0x0b, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
0x12, 0x19, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63,
0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x57, 0x6f,
0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x47, 0x5a, 0x45, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2d,
0x64, 0x65, 0x76, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2f, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x2f, 0x64, 0x69,
0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63,
0x74, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f,
0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x21, 0x0a, 0x1d, 0x52,
0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x05, 0x32, 0xe4,
0x03, 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, 0x52, 0x0a, 0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f,
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21,
0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b,
0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x0e, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e,
0x74, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3f, 0x0a, 0x13, 0x53, 0x65, 0x6e, 0x64, 0x53, 0x74, 0x65,
0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x10, 0x2e, 0x53,
0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14,
0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a, 0x17, 0x53, 0x65, 0x6e, 0x64, 0x47, 0x72,
0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e,
0x74, 0x12, 0x14, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69,
0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x3c, 0x0a, 0x10, 0x50, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44,
0x61, 0x74, 0x61, 0x12, 0x0e, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44,
0x61, 0x74, 0x61, 0x1a, 0x16, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44,
0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x46, 0x0a,
0x0b, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x19, 0x2e, 0x57,
0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72,
0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x47, 0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2d, 0x64, 0x65, 0x76, 0x2f,
0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,
0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x2f, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74,
0x63, 0x68, 0x65, 0x72, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x73, 0x62, 0x06,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -1686,6 +1698,7 @@ func file_dispatcher_proto_init() {
}
}
}
file_dispatcher_proto_msgTypes[0].OneofWrappers = []interface{}{}
type x struct{}
out := protoimpl.TypeBuilder{
File: protoimpl.DescBuilder{

View File

@@ -165,13 +165,20 @@ func (s *DispatcherImpl) Register(ctx context.Context, request *contracts.Worker
svcs = []string{"default"}
}
// create a worker in the database
worker, err := s.repo.Worker().CreateNewWorker(tenant.ID, &repository.CreateWorkerOpts{
opts := &repository.CreateWorkerOpts{
DispatcherId: s.dispatcherId,
Name: request.WorkerName,
Actions: request.Actions,
Services: svcs,
})
}
if request.MaxRuns != nil {
mr := int(*request.MaxRuns)
opts.MaxRuns = &mr
}
// create a worker in the database
worker, err := s.repo.Worker().CreateNewWorker(tenant.ID, opts)
if err != nil {
s.l.Error().Err(err).Msgf("could not create worker for tenant %s", tenant.ID)

View File

@@ -37,6 +37,7 @@ type GetActionListenerRequest struct {
WorkerName string
Services []string
Actions []string
MaxRuns *int
}
// ActionPayload unmarshals the action payload into the target. It also validates the resulting target.
@@ -168,12 +169,19 @@ func (d *dispatcherClientImpl) newActionListener(ctx context.Context, req *GetAc
return nil, err
}
// register the worker
resp, err := d.client.Register(d.ctx.newContext(ctx), &dispatchercontracts.WorkerRegisterRequest{
registerReq := &dispatchercontracts.WorkerRegisterRequest{
WorkerName: req.WorkerName,
Actions: req.Actions,
Services: req.Services,
})
}
if req.MaxRuns != nil {
mr := int32(*req.MaxRuns)
registerReq.MaxRuns = &mr
}
// register the worker
resp, err := d.client.Register(d.ctx.newContext(ctx), registerReq)
if err != nil {
return nil, fmt.Errorf("could not register the worker: %w", err)

View File

@@ -82,6 +82,8 @@ type Worker struct {
alerter errors.Alerter
middlewares *middlewares
maxRuns *int
}
type WorkerOpt func(*WorkerOpts)
@@ -93,6 +95,7 @@ type WorkerOpts struct {
integrations []integrations.Integration
alerter errors.Alerter
maxRuns *int
}
func defaultWorkerOpts() *WorkerOpts {
@@ -130,6 +133,12 @@ func WithErrorAlerter(alerter errors.Alerter) WorkerOpt {
}
}
func WithMaxRuns(maxRuns int) WorkerOpt {
return func(opts *WorkerOpts) {
opts.maxRuns = &maxRuns
}
}
// NewWorker creates a new worker instance
func NewWorker(fs ...WorkerOpt) (*Worker, error) {
opts := defaultWorkerOpts()
@@ -149,6 +158,7 @@ func NewWorker(fs ...WorkerOpt) (*Worker, error) {
actions: map[string]Action{},
alerter: opts.alerter,
middlewares: mws,
maxRuns: opts.maxRuns,
}
// register all integrations
@@ -268,6 +278,7 @@ func (w *Worker) Start(ctx context.Context) error {
listener, err := w.client.Dispatcher().GetActionListener(ctx, &client.GetActionListenerRequest{
WorkerName: w.name,
Actions: actionNames,
MaxRuns: w.maxRuns,
})
if err != nil {

View File

@@ -1,2 +1,5 @@
-- AlterEnum
ALTER TYPE "ConcurrencyLimitStrategy" ADD VALUE 'GROUP_ROUND_ROBIN';
-- AlterTable
ALTER TABLE "Worker" ADD COLUMN "maxRuns" INTEGER;

View File

@@ -966,6 +966,8 @@ model Worker {
dispatcher Dispatcher @relation(fields: [dispatcherId], references: [id], onDelete: Cascade, onUpdate: Cascade)
dispatcherId String @db.Uuid
maxRuns Int?
services Service[]
// the actions this worker can run

View File

@@ -23,7 +23,7 @@ class ConcurrencyDemoWorkflowRR:
pass
workflow = ConcurrencyDemoWorkflowRR()
worker = hatchet.worker('concurrency-demo-worker-rr', max_threads=1)
worker = hatchet.worker('concurrency-demo-worker-rr', max_runs=1)
worker.register_workflow(workflow)
worker.start()

View File

@@ -26,7 +26,7 @@ class ConcurrencyDemoWorkflow:
print("finished step2")
workflow = ConcurrencyDemoWorkflow()
worker = hatchet.worker('concurrency-demo-worker', max_threads=4)
worker = hatchet.worker('concurrency-demo-worker', max_runs=4)
worker.register_workflow(workflow)
worker.start()

View File

@@ -1,3 +1,4 @@
import time
from hatchet_sdk import Hatchet, Context
from dotenv import load_dotenv
@@ -13,7 +14,7 @@ class MyWorkflow:
@hatchet.step()
def step1(self, context : Context):
overrideValue = context.playground("prompt", "You are an AI assistant...")
time.sleep(5)
print("executed step1", context.workflow_input())
return {
"step1": overrideValue,
@@ -22,6 +23,7 @@ class MyWorkflow:
@hatchet.step()
def step2(self, context : Context):
print("executed step2", context.workflow_input())
time.sleep(5)
return {
"step2": "step2",
}

View File

@@ -40,7 +40,7 @@ class ManualTriggerWorkflow:
workflow = ManualTriggerWorkflow()
worker = hatchet.worker('manual-worker', max_threads=4)
worker = hatchet.worker('manual-worker', max_runs=4)
worker.register_workflow(workflow)
worker.start()

View File

@@ -29,7 +29,7 @@ class MyWorkflow:
print("finished step2")
workflow = MyWorkflow()
worker = hatchet.worker('test-worker', max_threads=4)
worker = hatchet.worker('test-worker', max_runs=4)
worker.register_workflow(workflow)
worker.start()

View File

@@ -21,7 +21,7 @@ class TimeoutWorkflow:
raise e
workflow = TimeoutWorkflow()
worker = hatchet.worker('timeout-worker', max_threads=4)
worker = hatchet.worker('timeout-worker', max_runs=4)
worker.register_workflow(workflow)
worker.start()

View File

@@ -33,10 +33,11 @@ DEFAULT_REGISTER_TIMEOUT = 5
class GetActionListenerRequest:
def __init__(self, worker_name: str, services: List[str], actions: List[str]):
def __init__(self, worker_name: str, services: List[str], actions: List[str], max_runs: int | None = None):
self.worker_name = worker_name
self.services = services
self.actions = actions
self.max_runs = max_runs
class Action:
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):
@@ -196,7 +197,8 @@ class DispatcherClientImpl(DispatcherClient):
response : WorkerRegisterResponse = self.client.Register(WorkerRegisterRequest(
workerName=req.worker_name,
actions=req.actions,
services=req.services
services=req.services,
maxRuns=req.max_runs
), timeout=DEFAULT_REGISTER_TIMEOUT, metadata=get_metadata(self.token))
return ActionListenerImpl(self.client, self.token, response.workerId)

View File

@@ -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\"\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\"9\n SubscribeToWorkflowEventsRequest\x12\x15\n\rworkflowRunId\x18\x01 \x01(\t\"\xe0\x01\n\rWorkflowEvent\x12\x15\n\rworkflowRunId\x18\x01 \x01(\t\x12#\n\x0cresourceType\x18\x02 \x01(\x0e\x32\r.ResourceType\x12%\n\teventType\x18\x03 \x01(\x0e\x32\x12.ResourceEventType\x12\x12\n\nresourceId\x18\x04 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x14\n\x0c\x65ventPayload\x18\x06 \x01(\t\x12\x0e\n\x06hangup\x18\x07 \x01(\x08\"W\n\rOverridesData\x12\x11\n\tstepRunId\x18\x01 \x01(\t\x12\x0c\n\x04path\x18\x02 \x01(\t\x12\r\n\x05value\x18\x03 \x01(\t\x12\x16\n\x0e\x63\x61llerFilename\x18\x04 \x01(\t\"\x17\n\x15OverridesDataResponse*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*e\n\x0cResourceType\x12\x19\n\x15RESOURCE_TYPE_UNKNOWN\x10\x00\x12\x1a\n\x16RESOURCE_TYPE_STEP_RUN\x10\x01\x12\x1e\n\x1aRESOURCE_TYPE_WORKFLOW_RUN\x10\x02*\xde\x01\n\x11ResourceEventType\x12\x1f\n\x1bRESOURCE_EVENT_TYPE_UNKNOWN\x10\x00\x12\x1f\n\x1bRESOURCE_EVENT_TYPE_STARTED\x10\x01\x12!\n\x1dRESOURCE_EVENT_TYPE_COMPLETED\x10\x02\x12\x1e\n\x1aRESOURCE_EVENT_TYPE_FAILED\x10\x03\x12!\n\x1dRESOURCE_EVENT_TYPE_CANCELLED\x10\x04\x12!\n\x1dRESOURCE_EVENT_TYPE_TIMED_OUT\x10\x05\x32\xe4\x03\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\x12R\n\x19SubscribeToWorkflowEvents\x12!.SubscribeToWorkflowEventsRequest\x1a\x0e.WorkflowEvent\"\x00\x30\x01\x12?\n\x13SendStepActionEvent\x12\x10.StepActionEvent\x1a\x14.ActionEventResponse\"\x00\x12G\n\x17SendGroupKeyActionEvent\x12\x14.GroupKeyActionEvent\x1a\x14.ActionEventResponse\"\x00\x12<\n\x10PutOverridesData\x12\x0e.OverridesData\x1a\x16.OverridesDataResponse\"\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\"p\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\x12\x14\n\x07maxRuns\x18\x04 \x01(\x05H\x00\x88\x01\x01\x42\n\n\x08_maxRuns\"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\"9\n SubscribeToWorkflowEventsRequest\x12\x15\n\rworkflowRunId\x18\x01 \x01(\t\"\xe0\x01\n\rWorkflowEvent\x12\x15\n\rworkflowRunId\x18\x01 \x01(\t\x12#\n\x0cresourceType\x18\x02 \x01(\x0e\x32\r.ResourceType\x12%\n\teventType\x18\x03 \x01(\x0e\x32\x12.ResourceEventType\x12\x12\n\nresourceId\x18\x04 \x01(\t\x12\x32\n\x0e\x65ventTimestamp\x18\x05 \x01(\x0b\x32\x1a.google.protobuf.Timestamp\x12\x14\n\x0c\x65ventPayload\x18\x06 \x01(\t\x12\x0e\n\x06hangup\x18\x07 \x01(\x08\"W\n\rOverridesData\x12\x11\n\tstepRunId\x18\x01 \x01(\t\x12\x0c\n\x04path\x18\x02 \x01(\t\x12\r\n\x05value\x18\x03 \x01(\t\x12\x16\n\x0e\x63\x61llerFilename\x18\x04 \x01(\t\"\x17\n\x15OverridesDataResponse*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*e\n\x0cResourceType\x12\x19\n\x15RESOURCE_TYPE_UNKNOWN\x10\x00\x12\x1a\n\x16RESOURCE_TYPE_STEP_RUN\x10\x01\x12\x1e\n\x1aRESOURCE_TYPE_WORKFLOW_RUN\x10\x02*\xde\x01\n\x11ResourceEventType\x12\x1f\n\x1bRESOURCE_EVENT_TYPE_UNKNOWN\x10\x00\x12\x1f\n\x1bRESOURCE_EVENT_TYPE_STARTED\x10\x01\x12!\n\x1dRESOURCE_EVENT_TYPE_COMPLETED\x10\x02\x12\x1e\n\x1aRESOURCE_EVENT_TYPE_FAILED\x10\x03\x12!\n\x1dRESOURCE_EVENT_TYPE_CANCELLED\x10\x04\x12!\n\x1dRESOURCE_EVENT_TYPE_TIMED_OUT\x10\x05\x32\xe4\x03\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\x12R\n\x19SubscribeToWorkflowEvents\x12!.SubscribeToWorkflowEventsRequest\x1a\x0e.WorkflowEvent\"\x00\x30\x01\x12?\n\x13SendStepActionEvent\x12\x10.StepActionEvent\x1a\x14.ActionEventResponse\"\x00\x12G\n\x17SendGroupKeyActionEvent\x12\x14.GroupKeyActionEvent\x1a\x14.ActionEventResponse\"\x00\x12<\n\x10PutOverridesData\x12\x0e.OverridesData\x1a\x16.OverridesDataResponse\"\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,42 +23,42 @@ _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=1538
_globals['_ACTIONTYPE']._serialized_end=1616
_globals['_GROUPKEYACTIONEVENTTYPE']._serialized_start=1619
_globals['_GROUPKEYACTIONEVENTTYPE']._serialized_end=1781
_globals['_STEPACTIONEVENTTYPE']._serialized_start=1784
_globals['_STEPACTIONEVENTTYPE']._serialized_end=1922
_globals['_RESOURCETYPE']._serialized_start=1924
_globals['_RESOURCETYPE']._serialized_end=2025
_globals['_RESOURCEEVENTTYPE']._serialized_start=2028
_globals['_RESOURCEEVENTTYPE']._serialized_end=2250
_globals['_ACTIONTYPE']._serialized_start=1572
_globals['_ACTIONTYPE']._serialized_end=1650
_globals['_GROUPKEYACTIONEVENTTYPE']._serialized_start=1653
_globals['_GROUPKEYACTIONEVENTTYPE']._serialized_end=1815
_globals['_STEPACTIONEVENTTYPE']._serialized_start=1818
_globals['_STEPACTIONEVENTTYPE']._serialized_end=1956
_globals['_RESOURCETYPE']._serialized_start=1958
_globals['_RESOURCETYPE']._serialized_end=2059
_globals['_RESOURCEEVENTTYPE']._serialized_start=2062
_globals['_RESOURCEEVENTTYPE']._serialized_end=2284
_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=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['_SUBSCRIBETOWORKFLOWEVENTSREQUEST']._serialized_start=1138
_globals['_SUBSCRIBETOWORKFLOWEVENTSREQUEST']._serialized_end=1195
_globals['_WORKFLOWEVENT']._serialized_start=1198
_globals['_WORKFLOWEVENT']._serialized_end=1422
_globals['_OVERRIDESDATA']._serialized_start=1424
_globals['_OVERRIDESDATA']._serialized_end=1511
_globals['_OVERRIDESDATARESPONSE']._serialized_start=1513
_globals['_OVERRIDESDATARESPONSE']._serialized_end=1536
_globals['_DISPATCHER']._serialized_start=2253
_globals['_DISPATCHER']._serialized_end=2737
_globals['_WORKERREGISTERREQUEST']._serialized_end=165
_globals['_WORKERREGISTERRESPONSE']._serialized_start=167
_globals['_WORKERREGISTERRESPONSE']._serialized_end=247
_globals['_ASSIGNEDACTION']._serialized_start=250
_globals['_ASSIGNEDACTION']._serialized_end=492
_globals['_WORKERLISTENREQUEST']._serialized_start=494
_globals['_WORKERLISTENREQUEST']._serialized_end=533
_globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_start=535
_globals['_WORKERUNSUBSCRIBEREQUEST']._serialized_end=579
_globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_start=581
_globals['_WORKERUNSUBSCRIBERESPONSE']._serialized_end=644
_globals['_GROUPKEYACTIONEVENT']._serialized_start=647
_globals['_GROUPKEYACTIONEVENT']._serialized_end=872
_globals['_STEPACTIONEVENT']._serialized_start=875
_globals['_STEPACTIONEVENT']._serialized_end=1111
_globals['_ACTIONEVENTRESPONSE']._serialized_start=1113
_globals['_ACTIONEVENTRESPONSE']._serialized_end=1170
_globals['_SUBSCRIBETOWORKFLOWEVENTSREQUEST']._serialized_start=1172
_globals['_SUBSCRIBETOWORKFLOWEVENTSREQUEST']._serialized_end=1229
_globals['_WORKFLOWEVENT']._serialized_start=1232
_globals['_WORKFLOWEVENT']._serialized_end=1456
_globals['_OVERRIDESDATA']._serialized_start=1458
_globals['_OVERRIDESDATA']._serialized_end=1545
_globals['_OVERRIDESDATARESPONSE']._serialized_start=1547
_globals['_OVERRIDESDATARESPONSE']._serialized_end=1570
_globals['_DISPATCHER']._serialized_start=2287
_globals['_DISPATCHER']._serialized_end=2771
# @@protoc_insertion_point(module_scope)

View File

@@ -63,14 +63,16 @@ RESOURCE_EVENT_TYPE_CANCELLED: ResourceEventType
RESOURCE_EVENT_TYPE_TIMED_OUT: ResourceEventType
class WorkerRegisterRequest(_message.Message):
__slots__ = ("workerName", "actions", "services")
__slots__ = ("workerName", "actions", "services", "maxRuns")
WORKERNAME_FIELD_NUMBER: _ClassVar[int]
ACTIONS_FIELD_NUMBER: _ClassVar[int]
SERVICES_FIELD_NUMBER: _ClassVar[int]
MAXRUNS_FIELD_NUMBER: _ClassVar[int]
workerName: str
actions: _containers.RepeatedScalarFieldContainer[str]
services: _containers.RepeatedScalarFieldContainer[str]
def __init__(self, workerName: _Optional[str] = ..., actions: _Optional[_Iterable[str]] = ..., services: _Optional[_Iterable[str]] = ...) -> None: ...
maxRuns: int
def __init__(self, workerName: _Optional[str] = ..., actions: _Optional[_Iterable[str]] = ..., services: _Optional[_Iterable[str]] = ..., maxRuns: _Optional[int] = ...) -> None: ...
class WorkerRegisterResponse(_message.Message):
__slots__ = ("tenantId", "workerId", "workerName")

View File

@@ -48,5 +48,5 @@ class Hatchet:
return inner
def worker(self, name: str, max_threads: int = 200):
return Worker(name, max_threads)
def worker(self, name: str, max_runs: int | None = None):
return Worker(name=name, max_runs=max_runs)

View File

@@ -19,10 +19,11 @@ from .logger import logger
class Worker:
def __init__(self, name: str, max_threads: int = 200, debug=False, handle_kill=True):
def __init__(self, name: str, max_runs: int | None = None, debug=False, handle_kill=True):
self.name = name
self.threads: Dict[str, Thread] = {} # Store step run ids and threads
self.thread_pool = ThreadPoolExecutor(max_workers=max_threads)
self.max_runs = max_runs
self.thread_pool = ThreadPoolExecutor(max_workers=max_runs)
self.futures: Dict[str, Future] = {} # Store step run ids and futures
self.contexts: Dict[str, Context] = {} # Store step run ids and contexts
self.action_registry : dict[str, Callable[..., Any]] = {}
@@ -138,7 +139,6 @@ class Worker:
if not errored:
# Create an action event
try:
print(action)
event = self.get_group_key_action_finished_event(action, output)
except Exception as e:
logger.error(f"Could not get action finished event: {e}")
@@ -335,6 +335,7 @@ class Worker:
worker_name=self.name,
services=["default"],
actions=self.action_registry.keys(),
max_runs=self.max_runs,
))
generator = self.listener.actions()

View File

@@ -1,6 +1,6 @@
[tool.poetry]
name = "hatchet-sdk"
version = "0.11.0"
version = "0.12.0"
description = ""
authors = ["Alexander Belanger <alexander@hatchet.run>"]
readme = "README.md"

View File

@@ -28,8 +28,9 @@ const workflow: Workflow = {
{
name: 'dag-step2',
parents: ['dag-step1'],
run: (ctx) => {
run: async (ctx) => {
console.log('executed step2!');
await sleep(5000);
return { step2: 'step2' };
},
},
@@ -53,7 +54,7 @@ const workflow: Workflow = {
};
async function main() {
const worker = await hatchet.worker('example-worker');
const worker = await hatchet.worker('example-worker', 1);
await worker.registerWorkflow(workflow);
worker.start();
}

View File

@@ -14,6 +14,7 @@ interface GetActionListenerOptions {
workerName: string;
services: string[];
actions: string[];
maxRuns?: number;
}
export class DispatcherClient {

View File

@@ -132,16 +132,18 @@ export class HatchetClient {
return new HatchetClient(config, options, axiosConfig);
}
// @deprecated
async run(workflow: string | Workflow): Promise<Worker> {
const worker = await this.worker(workflow);
worker.start();
return worker;
}
async worker(workflow: string | Workflow): Promise<Worker> {
async worker(workflow: string | Workflow, maxRuns?: number): Promise<Worker> {
const name = typeof workflow === 'string' ? workflow : workflow.id;
const worker = new Worker(this, {
name,
maxRuns,
});
if (typeof workflow !== 'string') {

View File

@@ -32,13 +32,18 @@ export class Worker {
listener: ActionListener | undefined;
futures: Record<Action['stepRunId'], HatchetPromise<any>> = {};
contexts: Record<Action['stepRunId'], Context<any, any>> = {};
maxRuns?: number;
logger: Logger;
constructor(client: HatchetClient, options: { name: string; handleKill?: boolean }) {
constructor(
client: HatchetClient,
options: { name: string; handleKill?: boolean; maxRuns?: number }
) {
this.client = client;
this.name = options.name;
this.action_registry = {};
this.maxRuns = options.maxRuns;
process.on('SIGTERM', () => this.exitGracefully());
process.on('SIGINT', () => this.exitGracefully());
@@ -338,6 +343,7 @@ export class Worker {
workerName: this.name,
services: ['default'],
actions: Object.keys(this.action_registry),
maxRuns: this.maxRuns,
});
const generator = this.listener.actions();

View File

@@ -237,6 +237,8 @@ export interface WorkerRegisterRequest {
actions: string[];
/** (optional) the services for this worker */
services: string[];
/** (optional) the max number of runs this worker can handle */
maxRuns?: number | undefined;
}
export interface WorkerRegisterResponse {
@@ -374,7 +376,7 @@ export interface OverridesDataResponse {
}
function createBaseWorkerRegisterRequest(): WorkerRegisterRequest {
return { workerName: "", actions: [], services: [] };
return { workerName: "", actions: [], services: [], maxRuns: undefined };
}
export const WorkerRegisterRequest = {
@@ -388,6 +390,9 @@ export const WorkerRegisterRequest = {
for (const v of message.services) {
writer.uint32(26).string(v!);
}
if (message.maxRuns !== undefined) {
writer.uint32(32).int32(message.maxRuns);
}
return writer;
},
@@ -419,6 +424,13 @@ export const WorkerRegisterRequest = {
message.services.push(reader.string());
continue;
case 4:
if (tag !== 32) {
break;
}
message.maxRuns = reader.int32();
continue;
}
if ((tag & 7) === 4 || tag === 0) {
break;
@@ -433,6 +445,7 @@ export const WorkerRegisterRequest = {
workerName: isSet(object.workerName) ? globalThis.String(object.workerName) : "",
actions: globalThis.Array.isArray(object?.actions) ? object.actions.map((e: any) => globalThis.String(e)) : [],
services: globalThis.Array.isArray(object?.services) ? object.services.map((e: any) => globalThis.String(e)) : [],
maxRuns: isSet(object.maxRuns) ? globalThis.Number(object.maxRuns) : undefined,
};
},
@@ -447,6 +460,9 @@ export const WorkerRegisterRequest = {
if (message.services?.length) {
obj.services = message.services;
}
if (message.maxRuns !== undefined) {
obj.maxRuns = Math.round(message.maxRuns);
}
return obj;
},
@@ -458,6 +474,7 @@ export const WorkerRegisterRequest = {
message.workerName = object.workerName ?? "";
message.actions = object.actions?.map((e) => e) || [];
message.services = object.services?.map((e) => e) || [];
message.maxRuns = object.maxRuns ?? undefined;
return message;
},
};