mirror of
https://github.com/hatchet-dev/hatchet.git
synced 2026-01-06 00:40:10 -06:00
feat: multi-workflow runs listener on a single endpoint
* new api-contract for workflow run events * feat: initial implementation for new subscribe listener * fix: sync issues and send workflow runs immediately * refactor: add context to all engine db queries, fix deadlocking query * fix: use new ctx for deleting dispatcher and ticker * add cancellation reasons * fix: docs linting --------- Co-authored-by: gabriel ruttner <gabriel.ruttner@gmail.com>
This commit is contained in:
@@ -18,6 +18,8 @@ service Dispatcher {
|
||||
|
||||
rpc SubscribeToWorkflowEvents(SubscribeToWorkflowEventsRequest) returns (stream WorkflowEvent) {}
|
||||
|
||||
rpc SubscribeToWorkflowRuns(stream SubscribeToWorkflowRunsRequest) returns (stream WorkflowRunEvent) {}
|
||||
|
||||
rpc SendStepActionEvent(StepActionEvent) returns (ActionEventResponse) {}
|
||||
|
||||
rpc SendGroupKeyActionEvent(GroupKeyActionEvent) returns (ActionEventResponse) {}
|
||||
@@ -190,6 +192,11 @@ message SubscribeToWorkflowEventsRequest {
|
||||
string workflowRunId = 1;
|
||||
}
|
||||
|
||||
message SubscribeToWorkflowRunsRequest {
|
||||
// the id of the workflow run
|
||||
string workflowRunId = 1;
|
||||
}
|
||||
|
||||
enum ResourceType {
|
||||
RESOURCE_TYPE_UNKNOWN = 0;
|
||||
RESOURCE_TYPE_STEP_RUN = 1;
|
||||
@@ -232,6 +239,33 @@ message WorkflowEvent {
|
||||
optional int32 retryCount = 9;
|
||||
}
|
||||
|
||||
enum WorkflowRunEventType {
|
||||
WORKFLOW_RUN_EVENT_TYPE_FINISHED = 0;
|
||||
}
|
||||
|
||||
message WorkflowRunEvent {
|
||||
// the id of the workflow run
|
||||
string workflowRunId = 1;
|
||||
|
||||
WorkflowRunEventType eventType = 2;
|
||||
|
||||
google.protobuf.Timestamp eventTimestamp = 3;
|
||||
|
||||
repeated StepRunResult results = 4;
|
||||
}
|
||||
|
||||
message StepRunResult {
|
||||
string stepRunId = 1;
|
||||
|
||||
string stepReadableId = 2;
|
||||
|
||||
string jobRunId = 3;
|
||||
|
||||
optional string error = 4;
|
||||
|
||||
optional string output = 5;
|
||||
}
|
||||
|
||||
message OverridesData {
|
||||
// the step run id
|
||||
string stepRunId = 1;
|
||||
|
||||
@@ -177,7 +177,7 @@ func (a *AuthN) handleBearerAuth(c echo.Context) error {
|
||||
}
|
||||
|
||||
// Validate the token.
|
||||
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(token)
|
||||
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(c.Request().Context(), token)
|
||||
|
||||
if err != nil {
|
||||
a.l.Debug().Err(err).Msg("error validating tenant token")
|
||||
|
||||
@@ -17,7 +17,7 @@ func (a *APITokenService) ApiTokenCreate(ctx echo.Context, request gen.ApiTokenC
|
||||
return gen.ApiTokenCreate400JSONResponse(*apiErrors), nil
|
||||
}
|
||||
|
||||
token, err := a.config.Auth.JWTManager.GenerateTenantToken(tenant.ID, request.Body.Name)
|
||||
token, err := a.config.Auth.JWTManager.GenerateTenantToken(ctx.Request().Context(), tenant.ID, request.Body.Name)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
@@ -19,7 +19,7 @@ func (t *EventService) EventUpdateReplay(ctx echo.Context, request gen.EventUpda
|
||||
eventIds[i] = request.Body.EventIds[i].String()
|
||||
}
|
||||
|
||||
events, err := t.config.EngineRepository.Event().ListEventsByIds(tenant.ID, eventIds)
|
||||
events, err := t.config.EngineRepository.Event().ListEventsByIds(ctx.Request().Context(), tenant.ID, eventIds)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
@@ -27,7 +27,7 @@ func (t *StepRunService) StepRunUpdateCancel(ctx echo.Context, request gen.StepR
|
||||
), nil
|
||||
}
|
||||
|
||||
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(tenant.ID, stepRun.ID)
|
||||
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(ctx.Request().Context(), tenant.ID, stepRun.ID)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not get step run for engine: %w", err)
|
||||
|
||||
@@ -70,7 +70,7 @@ func (t *StepRunService) StepRunUpdateRerun(ctx echo.Context, request gen.StepRu
|
||||
return nil, err
|
||||
}
|
||||
|
||||
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(tenant.ID, stepRun.ID)
|
||||
engineStepRun, err := t.config.EngineRepository.StepRun().GetStepRunForEngine(ctx.Request().Context(), tenant.ID, stepRun.ID)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not get step run for engine: %w", err)
|
||||
|
||||
@@ -36,7 +36,7 @@ func (t *WorkflowService) WorkflowRunCreate(ctx echo.Context, request gen.Workfl
|
||||
workflowVersionId = versions[0].ID
|
||||
}
|
||||
|
||||
workflowVersion, err := t.config.EngineRepository.Workflow().GetWorkflowVersionById(tenant.ID, workflowVersionId)
|
||||
workflowVersion, err := t.config.EngineRepository.Workflow().GetWorkflowVersionById(ctx.Request().Context(), tenant.ID, workflowVersionId)
|
||||
|
||||
if err != nil {
|
||||
if errors.Is(err, db.ErrNotFound) {
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package cli
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"log"
|
||||
@@ -123,14 +124,14 @@ func runSeed(cf *loader.ConfigLoader) error {
|
||||
}
|
||||
|
||||
func seedDev(repo repository.EngineRepository, tenantId string) error {
|
||||
_, err := repo.Workflow().GetWorkflowByName(tenantId, "test-workflow")
|
||||
_, err := repo.Workflow().GetWorkflowByName(context.Background(), tenantId, "test-workflow")
|
||||
|
||||
if err != nil {
|
||||
if !errors.Is(err, pgx.ErrNoRows) {
|
||||
return err
|
||||
}
|
||||
|
||||
wf, err := repo.Workflow().CreateNewWorkflow(tenantId, &repository.CreateWorkflowVersionOpts{
|
||||
wf, err := repo.Workflow().CreateNewWorkflow(context.Background(), tenantId, &repository.CreateWorkflowVersionOpts{
|
||||
Name: "test-workflow",
|
||||
Description: repository.StringPtr("This is a test workflow."),
|
||||
Version: repository.StringPtr("v0.1.0"),
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package cli
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
@@ -68,7 +69,7 @@ func runCreateAPIToken() error {
|
||||
|
||||
defer serverConf.Disconnect() // nolint:errcheck
|
||||
|
||||
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(tokenTenantId, tokenName)
|
||||
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(context.Background(), tokenTenantId, tokenName)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
@@ -58,7 +58,7 @@ func run(ch <-chan interface{}, events chan<- string) error {
|
||||
interruptCtx, cancel := cmdutils.InterruptContextFromChan(ch)
|
||||
defer cancel()
|
||||
|
||||
err = c.Subscribe().On(interruptCtx, workflowRunId, func(event client.RunEvent) error {
|
||||
err = c.Subscribe().On(interruptCtx, workflowRunId, func(event client.WorkflowEvent) error {
|
||||
fmt.Println(event.EventPayload)
|
||||
|
||||
return nil
|
||||
|
||||
@@ -2,6 +2,7 @@ package main
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/joho/godotenv"
|
||||
@@ -95,7 +96,8 @@ func run(events chan<- string) (func() error, error) {
|
||||
|
||||
eg.SetLimit(NUM_CHILDREN)
|
||||
|
||||
childOutputs := make([]int, NUM_CHILDREN)
|
||||
childOutputs := make([]int, 0)
|
||||
childOutputsMu := sync.Mutex{}
|
||||
|
||||
for i, childWorkflow := range childWorkflows {
|
||||
eg.Go(func(i int, childWorkflow *worker.ChildWorkflow) func() error {
|
||||
@@ -114,7 +116,9 @@ func run(events chan<- string) (func() error, error) {
|
||||
return err
|
||||
}
|
||||
|
||||
childOutputs[i] = childOutput.Index
|
||||
childOutputsMu.Lock()
|
||||
childOutputs = append(childOutputs, childOutput.Index)
|
||||
childOutputsMu.Unlock()
|
||||
|
||||
events <- fmt.Sprintf("child-%d-completed", childOutput.Index)
|
||||
|
||||
@@ -124,10 +128,38 @@ func run(events chan<- string) (func() error, error) {
|
||||
}(i, childWorkflow))
|
||||
}
|
||||
|
||||
err = eg.Wait()
|
||||
finishedCh := make(chan struct{})
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
go func() {
|
||||
defer close(finishedCh)
|
||||
err = eg.Wait()
|
||||
}()
|
||||
|
||||
timer := time.NewTimer(60 * time.Second)
|
||||
|
||||
select {
|
||||
case <-finishedCh:
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case <-timer.C:
|
||||
incomplete := make([]int, 0)
|
||||
// print non-complete children
|
||||
for i := range childWorkflows {
|
||||
completed := false
|
||||
for _, childOutput := range childOutputs {
|
||||
if childOutput == i {
|
||||
completed = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if !completed {
|
||||
incomplete = append(incomplete, i)
|
||||
}
|
||||
}
|
||||
|
||||
return nil, fmt.Errorf("timed out waiting for the following child workflows to complete: %v", incomplete)
|
||||
}
|
||||
|
||||
sum := 0
|
||||
@@ -140,7 +172,7 @@ func run(events chan<- string) (func() error, error) {
|
||||
ChildSum: sum,
|
||||
}, nil
|
||||
},
|
||||
),
|
||||
).SetTimeout("10m"),
|
||||
},
|
||||
},
|
||||
)
|
||||
|
||||
@@ -283,7 +283,7 @@ Which gives us much more promising output:
|
||||
This works — and you can modify this logic to be more distributed by maintaining a lease when a worker starts for a set amount of time — as long as the polling interval is below the query duration time (or more specifically, `pollingTime / numWorkers` is below the query duration time). But what happens when our queue starts to fill up? Let's add 10,000 enqueued tasks and run an `EXPLAIN ANALYZE` for this query to take a look at performance:
|
||||
|
||||
```sql
|
||||
QUERY PLAN
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Update on tasks (cost=259.44..514.23 rows=1 width=78) (actual time=132.717..154.337 rows=100 loops=1)
|
||||
-> Hash Join (cost=259.44..514.23 rows=1 width=78) (actual time=125.423..141.271 rows=100 loops=1)
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package token
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
@@ -12,8 +13,8 @@ import (
|
||||
)
|
||||
|
||||
type JWTManager interface {
|
||||
GenerateTenantToken(tenantId, name string) (string, error)
|
||||
ValidateTenantToken(token string) (string, error)
|
||||
GenerateTenantToken(ctx context.Context, tenantId, name string) (string, error)
|
||||
ValidateTenantToken(ctx context.Context, token string) (string, error)
|
||||
}
|
||||
|
||||
type TokenOpts struct {
|
||||
@@ -45,7 +46,7 @@ func NewJWTManager(encryptionSvc encryption.EncryptionService, tokenRepo reposit
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (j *jwtManagerImpl) GenerateTenantToken(tenantId, name string) (string, error) {
|
||||
func (j *jwtManagerImpl) GenerateTenantToken(ctx context.Context, tenantId, name string) (string, error) {
|
||||
// Retrieve the JWT Signer primitive from privateKeysetHandle.
|
||||
signer, err := jwt.NewSigner(j.encryption.GetPrivateJWTHandle())
|
||||
|
||||
@@ -68,7 +69,7 @@ func (j *jwtManagerImpl) GenerateTenantToken(tenantId, name string) (string, err
|
||||
}
|
||||
|
||||
// write the token to the database
|
||||
_, err = j.tokenRepo.CreateAPIToken(&repository.CreateAPITokenOpts{
|
||||
_, err = j.tokenRepo.CreateAPIToken(ctx, &repository.CreateAPITokenOpts{
|
||||
ID: tokenId,
|
||||
ExpiresAt: expiresAt,
|
||||
TenantId: &tenantId,
|
||||
@@ -82,7 +83,7 @@ func (j *jwtManagerImpl) GenerateTenantToken(tenantId, name string) (string, err
|
||||
return token, nil
|
||||
}
|
||||
|
||||
func (j *jwtManagerImpl) ValidateTenantToken(token string) (tenantId string, err error) {
|
||||
func (j *jwtManagerImpl) ValidateTenantToken(ctx context.Context, token string) (tenantId string, err error) {
|
||||
// Verify the signed token.
|
||||
audience := j.opts.Audience
|
||||
|
||||
@@ -140,7 +141,7 @@ func (j *jwtManagerImpl) ValidateTenantToken(token string) (tenantId string, err
|
||||
}
|
||||
|
||||
// read the token from the database
|
||||
dbToken, err := j.tokenRepo.GetAPITokenById(tokenId)
|
||||
dbToken, err := j.tokenRepo.GetAPITokenById(ctx, tokenId)
|
||||
|
||||
if err != nil {
|
||||
return "", fmt.Errorf("failed to read token from database: %v", err)
|
||||
|
||||
@@ -3,6 +3,7 @@
|
||||
package token_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"testing"
|
||||
@@ -40,14 +41,14 @@ func TestCreateTenantToken(t *testing.T) { // make sure no cache is used for tes
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
|
||||
token, err := jwtManager.GenerateTenantToken(tenantId, "test token")
|
||||
token, err := jwtManager.GenerateTenantToken(context.Background(), tenantId, "test token")
|
||||
|
||||
if err != nil {
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
|
||||
// validate the token
|
||||
newTenantId, err := jwtManager.ValidateTenantToken(token)
|
||||
newTenantId, err := jwtManager.ValidateTenantToken(context.Background(), token)
|
||||
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, tenantId, newTenantId)
|
||||
@@ -81,14 +82,14 @@ func TestRevokeTenantToken(t *testing.T) {
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
|
||||
token, err := jwtManager.GenerateTenantToken(tenantId, "test token")
|
||||
token, err := jwtManager.GenerateTenantToken(context.Background(), tenantId, "test token")
|
||||
|
||||
if err != nil {
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
|
||||
// validate the token
|
||||
_, err = jwtManager.ValidateTenantToken(token)
|
||||
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
|
||||
|
||||
assert.NoError(t, err)
|
||||
|
||||
@@ -107,7 +108,7 @@ func TestRevokeTenantToken(t *testing.T) {
|
||||
}
|
||||
|
||||
// validate the token again
|
||||
_, err = jwtManager.ValidateTenantToken(token)
|
||||
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
|
||||
|
||||
// error as the token was revoked
|
||||
assert.Error(t, err)
|
||||
@@ -141,14 +142,14 @@ func TestRevokeTenantTokenCache(t *testing.T) {
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
|
||||
token, err := jwtManager.GenerateTenantToken(tenantId, "test token")
|
||||
token, err := jwtManager.GenerateTenantToken(context.Background(), tenantId, "test token")
|
||||
|
||||
if err != nil {
|
||||
t.Fatal(err.Error())
|
||||
}
|
||||
|
||||
// validate the token
|
||||
_, err = jwtManager.ValidateTenantToken(token)
|
||||
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
|
||||
|
||||
assert.NoError(t, err)
|
||||
|
||||
@@ -167,7 +168,7 @@ func TestRevokeTenantTokenCache(t *testing.T) {
|
||||
}
|
||||
|
||||
// validate the token again
|
||||
_, err = jwtManager.ValidateTenantToken(token)
|
||||
_, err = jwtManager.ValidateTenantToken(context.Background(), token)
|
||||
|
||||
// no error as it is cached
|
||||
assert.NoError(t, err)
|
||||
|
||||
@@ -336,7 +336,7 @@ func GetServerConfigFromConfigfile(dc *database.Config, cf *server.ServerConfigF
|
||||
}
|
||||
|
||||
// generate a token for the internal client
|
||||
token, err := auth.JWTManager.GenerateTenantToken(internalTenant.ID, fmt.Sprintf("internal-%s", tokenSuffix))
|
||||
token, err := auth.JWTManager.GenerateTenantToken(context.Background(), internalTenant.ID, fmt.Sprintf("internal-%s", tokenSuffix))
|
||||
|
||||
if err != nil {
|
||||
return nil, nil, fmt.Errorf("could not generate internal token: %w", err)
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
|
||||
@@ -28,6 +29,6 @@ type APITokenRepository interface {
|
||||
}
|
||||
|
||||
type EngineTokenRepository interface {
|
||||
CreateAPIToken(opts *CreateAPITokenOpts) (*dbsqlc.APIToken, error)
|
||||
GetAPITokenById(id string) (*dbsqlc.APIToken, error)
|
||||
CreateAPIToken(ctx context.Context, opts *CreateAPITokenOpts) (*dbsqlc.APIToken, error)
|
||||
GetAPITokenById(ctx context.Context, id string) (*dbsqlc.APIToken, error)
|
||||
}
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
@@ -16,12 +17,12 @@ type UpdateDispatcherOpts struct {
|
||||
|
||||
type DispatcherEngineRepository interface {
|
||||
// CreateNewDispatcher creates a new dispatcher for a given tenant.
|
||||
CreateNewDispatcher(opts *CreateDispatcherOpts) (*dbsqlc.Dispatcher, error)
|
||||
CreateNewDispatcher(ctx context.Context, opts *CreateDispatcherOpts) (*dbsqlc.Dispatcher, error)
|
||||
|
||||
// UpdateDispatcher updates a dispatcher for a given tenant.
|
||||
UpdateDispatcher(dispatcherId string, opts *UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error)
|
||||
UpdateDispatcher(ctx context.Context, dispatcherId string, opts *UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error)
|
||||
|
||||
Delete(dispatcherId string) error
|
||||
Delete(ctx context.Context, dispatcherId string) error
|
||||
|
||||
UpdateStaleDispatchers(onStale func(dispatcherId string, getValidDispatcherId func() string) error) error
|
||||
UpdateStaleDispatchers(ctx context.Context, onStale func(dispatcherId string, getValidDispatcherId func() string) error) error
|
||||
}
|
||||
|
||||
@@ -74,7 +74,7 @@ type EventEngineRepository interface {
|
||||
CreateEvent(ctx context.Context, opts *CreateEventOpts) (*dbsqlc.Event, error)
|
||||
|
||||
// GetEventForEngine returns an event for the engine by id.
|
||||
GetEventForEngine(tenantId, id string) (*dbsqlc.Event, error)
|
||||
GetEventForEngine(ctx context.Context, tenantId, id string) (*dbsqlc.Event, error)
|
||||
|
||||
ListEventsByIds(tenantId string, ids []string) ([]*dbsqlc.Event, error)
|
||||
ListEventsByIds(ctx context.Context, tenantId string, ids []string) ([]*dbsqlc.Event, error)
|
||||
}
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
|
||||
@@ -35,14 +36,14 @@ type UpdateGetGroupKeyRunOpts struct {
|
||||
|
||||
type GetGroupKeyRunEngineRepository interface {
|
||||
// ListStepRunsToRequeue returns a list of step runs which are in a requeueable state.
|
||||
ListGetGroupKeyRunsToRequeue(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
|
||||
ListGetGroupKeyRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
|
||||
|
||||
ListGetGroupKeyRunsToReassign(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
|
||||
ListGetGroupKeyRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error)
|
||||
|
||||
AssignGetGroupKeyRunToWorker(tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error)
|
||||
AssignGetGroupKeyRunToTicker(tenantId, getGroupKeyRunId string) (tickerId string, err error)
|
||||
AssignGetGroupKeyRunToWorker(ctx context.Context, tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error)
|
||||
AssignGetGroupKeyRunToTicker(ctx context.Context, tenantId, getGroupKeyRunId string) (tickerId string, err error)
|
||||
|
||||
UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId string, opts *UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
|
||||
UpdateGetGroupKeyRun(ctx context.Context, tenantId, getGroupKeyRunId string, opts *UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
|
||||
|
||||
GetGroupKeyRunForEngine(tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
|
||||
GetGroupKeyRunForEngine(ctx context.Context, tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error)
|
||||
}
|
||||
|
||||
@@ -1,6 +1,8 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/jackc/pgx/v5/pgtype"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
|
||||
@@ -32,7 +34,7 @@ type JobRunAPIRepository interface {
|
||||
type JobRunEngineRepository interface {
|
||||
// SetJobRunStatusRunning resets the status of a job run to a RUNNING status. This is useful if a step
|
||||
// run is being manually replayed, but shouldn't be used by most callers.
|
||||
SetJobRunStatusRunning(tenantId, jobRunId string) error
|
||||
SetJobRunStatusRunning(ctx context.Context, tenantId, jobRunId string) error
|
||||
|
||||
ListJobRunsForWorkflowRun(tenantId, workflowRunId string) ([]pgtype.UUID, error)
|
||||
ListJobRunsForWorkflowRun(ctx context.Context, tenantId, workflowRunId string) ([]pgtype.UUID, error)
|
||||
}
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
@@ -58,5 +59,5 @@ type LogsAPIRepository interface {
|
||||
|
||||
type LogsEngineRepository interface {
|
||||
// PutLog creates a new log line.
|
||||
PutLog(tenantId string, opts *CreateLogLineOpts) (*dbsqlc.LogLine, error)
|
||||
PutLog(ctx context.Context, tenantId string, opts *CreateLogLineOpts) (*dbsqlc.LogLine, error)
|
||||
}
|
||||
|
||||
@@ -100,7 +100,7 @@ func NewEngineTokenRepository(pool *pgxpool.Pool, v validator.Validator, l *zero
|
||||
}
|
||||
}
|
||||
|
||||
func (a *engineTokenRepository) CreateAPIToken(opts *repository.CreateAPITokenOpts) (*dbsqlc.APIToken, error) {
|
||||
func (a *engineTokenRepository) CreateAPIToken(ctx context.Context, opts *repository.CreateAPITokenOpts) (*dbsqlc.APIToken, error) {
|
||||
if err := a.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -118,11 +118,11 @@ func (a *engineTokenRepository) CreateAPIToken(opts *repository.CreateAPITokenOp
|
||||
createParams.Name = sqlchelpers.TextFromStr(*opts.Name)
|
||||
}
|
||||
|
||||
return a.queries.CreateAPIToken(context.Background(), a.pool, createParams)
|
||||
return a.queries.CreateAPIToken(ctx, a.pool, createParams)
|
||||
}
|
||||
|
||||
func (a *engineTokenRepository) GetAPITokenById(id string) (*dbsqlc.APIToken, error) {
|
||||
func (a *engineTokenRepository) GetAPITokenById(ctx context.Context, id string) (*dbsqlc.APIToken, error) {
|
||||
return cache.MakeCacheable[dbsqlc.APIToken](a.cache, id, func() (*dbsqlc.APIToken, error) {
|
||||
return a.queries.GetAPITokenById(context.Background(), a.pool, sqlchelpers.UUIDFromStr(id))
|
||||
return a.queries.GetAPITokenById(ctx, a.pool, sqlchelpers.UUIDFromStr(id))
|
||||
})
|
||||
}
|
||||
|
||||
@@ -96,14 +96,17 @@ FROM
|
||||
JOIN
|
||||
"JobRun" ON "StepRun"."jobRunId" = "JobRun"."id"
|
||||
WHERE
|
||||
"StepRun"."tenantId" = @tenantId::uuid
|
||||
(
|
||||
sqlc.narg('tenantId')::uuid IS NULL OR
|
||||
"StepRun"."tenantId" = sqlc.narg('tenantId')::uuid
|
||||
)
|
||||
AND (
|
||||
sqlc.narg('status')::"StepRunStatus" IS NULL OR
|
||||
"StepRun"."status" = sqlc.narg('status')::"StepRunStatus"
|
||||
)
|
||||
AND (
|
||||
sqlc.narg('workflowRunId')::uuid IS NULL OR
|
||||
"JobRun"."workflowRunId" = sqlc.narg('workflowRunId')::uuid
|
||||
sqlc.narg('workflowRunIds')::uuid[] IS NULL OR
|
||||
"JobRun"."workflowRunId" = ANY(sqlc.narg('workflowRunIds')::uuid[])
|
||||
)
|
||||
AND (
|
||||
sqlc.narg('jobRunId')::uuid IS NULL OR
|
||||
|
||||
@@ -432,14 +432,17 @@ FROM
|
||||
JOIN
|
||||
"JobRun" ON "StepRun"."jobRunId" = "JobRun"."id"
|
||||
WHERE
|
||||
"StepRun"."tenantId" = $1::uuid
|
||||
(
|
||||
$1::uuid IS NULL OR
|
||||
"StepRun"."tenantId" = $1::uuid
|
||||
)
|
||||
AND (
|
||||
$2::"StepRunStatus" IS NULL OR
|
||||
"StepRun"."status" = $2::"StepRunStatus"
|
||||
)
|
||||
AND (
|
||||
$3::uuid IS NULL OR
|
||||
"JobRun"."workflowRunId" = $3::uuid
|
||||
$3::uuid[] IS NULL OR
|
||||
"JobRun"."workflowRunId" = ANY($3::uuid[])
|
||||
)
|
||||
AND (
|
||||
$4::uuid IS NULL OR
|
||||
@@ -452,18 +455,18 @@ WHERE
|
||||
`
|
||||
|
||||
type ListStepRunsParams struct {
|
||||
Tenantid pgtype.UUID `json:"tenantid"`
|
||||
Status NullStepRunStatus `json:"status"`
|
||||
WorkflowRunId pgtype.UUID `json:"workflowRunId"`
|
||||
JobRunId pgtype.UUID `json:"jobRunId"`
|
||||
TickerId pgtype.UUID `json:"tickerId"`
|
||||
TenantId pgtype.UUID `json:"tenantId"`
|
||||
Status NullStepRunStatus `json:"status"`
|
||||
WorkflowRunIds []pgtype.UUID `json:"workflowRunIds"`
|
||||
JobRunId pgtype.UUID `json:"jobRunId"`
|
||||
TickerId pgtype.UUID `json:"tickerId"`
|
||||
}
|
||||
|
||||
func (q *Queries) ListStepRuns(ctx context.Context, db DBTX, arg ListStepRunsParams) ([]pgtype.UUID, error) {
|
||||
rows, err := db.Query(ctx, listStepRuns,
|
||||
arg.Tenantid,
|
||||
arg.TenantId,
|
||||
arg.Status,
|
||||
arg.WorkflowRunId,
|
||||
arg.WorkflowRunIds,
|
||||
arg.JobRunId,
|
||||
arg.TickerId,
|
||||
)
|
||||
|
||||
@@ -21,6 +21,10 @@ WHERE
|
||||
sqlc.narg('workflowId')::uuid IS NULL OR
|
||||
workflow."id" = sqlc.narg('workflowId')::uuid
|
||||
) AND
|
||||
(
|
||||
sqlc.narg('ids')::uuid[] IS NULL OR
|
||||
runs."id" = ANY(sqlc.narg('ids')::uuid[])
|
||||
) AND
|
||||
(
|
||||
sqlc.narg('parentId')::uuid IS NULL OR
|
||||
runs."parentId" = sqlc.narg('parentId')::uuid
|
||||
@@ -70,6 +74,10 @@ WHERE
|
||||
sqlc.narg('workflowId')::uuid IS NULL OR
|
||||
workflow."id" = sqlc.narg('workflowId')::uuid
|
||||
) AND
|
||||
(
|
||||
sqlc.narg('ids')::uuid[] IS NULL OR
|
||||
runs."id" = ANY(sqlc.narg('ids')::uuid[])
|
||||
) AND
|
||||
(
|
||||
sqlc.narg('parentId')::uuid IS NULL OR
|
||||
runs."parentId" = sqlc.narg('parentId')::uuid
|
||||
|
||||
@@ -35,36 +35,41 @@ WHERE
|
||||
workflow."id" = $3::uuid
|
||||
) AND
|
||||
(
|
||||
$4::uuid IS NULL OR
|
||||
runs."parentId" = $4::uuid
|
||||
$4::uuid[] IS NULL OR
|
||||
runs."id" = ANY($4::uuid[])
|
||||
) AND
|
||||
(
|
||||
$5::uuid IS NULL OR
|
||||
runs."parentStepRunId" = $5::uuid
|
||||
runs."parentId" = $5::uuid
|
||||
) AND
|
||||
(
|
||||
$6::uuid IS NULL OR
|
||||
events."id" = $6::uuid
|
||||
runs."parentStepRunId" = $6::uuid
|
||||
) AND
|
||||
(
|
||||
$7::text IS NULL OR
|
||||
runs."concurrencyGroupId" = $7::text
|
||||
$7::uuid IS NULL OR
|
||||
events."id" = $7::uuid
|
||||
) AND
|
||||
(
|
||||
$8::text[] IS NULL OR
|
||||
"status" = ANY(cast($8::text[] as "WorkflowRunStatus"[]))
|
||||
$8::text IS NULL OR
|
||||
runs."concurrencyGroupId" = $8::text
|
||||
) AND
|
||||
(
|
||||
$9::text[] IS NULL OR
|
||||
"status" = ANY(cast($9::text[] as "WorkflowRunStatus"[]))
|
||||
)
|
||||
`
|
||||
|
||||
type CountWorkflowRunsParams struct {
|
||||
TenantId pgtype.UUID `json:"tenantId"`
|
||||
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
|
||||
WorkflowId pgtype.UUID `json:"workflowId"`
|
||||
ParentId pgtype.UUID `json:"parentId"`
|
||||
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
|
||||
EventId pgtype.UUID `json:"eventId"`
|
||||
GroupKey pgtype.Text `json:"groupKey"`
|
||||
Statuses []string `json:"statuses"`
|
||||
TenantId pgtype.UUID `json:"tenantId"`
|
||||
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
|
||||
WorkflowId pgtype.UUID `json:"workflowId"`
|
||||
Ids []pgtype.UUID `json:"ids"`
|
||||
ParentId pgtype.UUID `json:"parentId"`
|
||||
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
|
||||
EventId pgtype.UUID `json:"eventId"`
|
||||
GroupKey pgtype.Text `json:"groupKey"`
|
||||
Statuses []string `json:"statuses"`
|
||||
}
|
||||
|
||||
func (q *Queries) CountWorkflowRuns(ctx context.Context, db DBTX, arg CountWorkflowRunsParams) (int64, error) {
|
||||
@@ -72,6 +77,7 @@ func (q *Queries) CountWorkflowRuns(ctx context.Context, db DBTX, arg CountWorkf
|
||||
arg.TenantId,
|
||||
arg.WorkflowVersionId,
|
||||
arg.WorkflowId,
|
||||
arg.Ids,
|
||||
arg.ParentId,
|
||||
arg.ParentStepRunId,
|
||||
arg.EventId,
|
||||
@@ -709,46 +715,51 @@ WHERE
|
||||
workflow."id" = $3::uuid
|
||||
) AND
|
||||
(
|
||||
$4::uuid IS NULL OR
|
||||
runs."parentId" = $4::uuid
|
||||
$4::uuid[] IS NULL OR
|
||||
runs."id" = ANY($4::uuid[])
|
||||
) AND
|
||||
(
|
||||
$5::uuid IS NULL OR
|
||||
runs."parentStepRunId" = $5::uuid
|
||||
runs."parentId" = $5::uuid
|
||||
) AND
|
||||
(
|
||||
$6::uuid IS NULL OR
|
||||
events."id" = $6::uuid
|
||||
runs."parentStepRunId" = $6::uuid
|
||||
) AND
|
||||
(
|
||||
$7::text IS NULL OR
|
||||
runs."concurrencyGroupId" = $7::text
|
||||
$7::uuid IS NULL OR
|
||||
events."id" = $7::uuid
|
||||
) AND
|
||||
(
|
||||
$8::text[] IS NULL OR
|
||||
"status" = ANY(cast($8::text[] as "WorkflowRunStatus"[]))
|
||||
$8::text IS NULL OR
|
||||
runs."concurrencyGroupId" = $8::text
|
||||
) AND
|
||||
(
|
||||
$9::text[] IS NULL OR
|
||||
"status" = ANY(cast($9::text[] as "WorkflowRunStatus"[]))
|
||||
)
|
||||
ORDER BY
|
||||
case when $9 = 'createdAt ASC' THEN runs."createdAt" END ASC ,
|
||||
case when $9 = 'createdAt DESC' then runs."createdAt" END DESC
|
||||
case when $10 = 'createdAt ASC' THEN runs."createdAt" END ASC ,
|
||||
case when $10 = 'createdAt DESC' then runs."createdAt" END DESC
|
||||
OFFSET
|
||||
COALESCE($10, 0)
|
||||
COALESCE($11, 0)
|
||||
LIMIT
|
||||
COALESCE($11, 50)
|
||||
COALESCE($12, 50)
|
||||
`
|
||||
|
||||
type ListWorkflowRunsParams struct {
|
||||
TenantId pgtype.UUID `json:"tenantId"`
|
||||
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
|
||||
WorkflowId pgtype.UUID `json:"workflowId"`
|
||||
ParentId pgtype.UUID `json:"parentId"`
|
||||
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
|
||||
EventId pgtype.UUID `json:"eventId"`
|
||||
GroupKey pgtype.Text `json:"groupKey"`
|
||||
Statuses []string `json:"statuses"`
|
||||
Orderby interface{} `json:"orderby"`
|
||||
Offset interface{} `json:"offset"`
|
||||
Limit interface{} `json:"limit"`
|
||||
TenantId pgtype.UUID `json:"tenantId"`
|
||||
WorkflowVersionId pgtype.UUID `json:"workflowVersionId"`
|
||||
WorkflowId pgtype.UUID `json:"workflowId"`
|
||||
Ids []pgtype.UUID `json:"ids"`
|
||||
ParentId pgtype.UUID `json:"parentId"`
|
||||
ParentStepRunId pgtype.UUID `json:"parentStepRunId"`
|
||||
EventId pgtype.UUID `json:"eventId"`
|
||||
GroupKey pgtype.Text `json:"groupKey"`
|
||||
Statuses []string `json:"statuses"`
|
||||
Orderby interface{} `json:"orderby"`
|
||||
Offset interface{} `json:"offset"`
|
||||
Limit interface{} `json:"limit"`
|
||||
}
|
||||
|
||||
type ListWorkflowRunsRow struct {
|
||||
@@ -767,6 +778,7 @@ func (q *Queries) ListWorkflowRuns(ctx context.Context, db DBTX, arg ListWorkflo
|
||||
arg.TenantId,
|
||||
arg.WorkflowVersionId,
|
||||
arg.WorkflowId,
|
||||
arg.Ids,
|
||||
arg.ParentId,
|
||||
arg.ParentStepRunId,
|
||||
arg.EventId,
|
||||
|
||||
@@ -31,33 +31,33 @@ func NewDispatcherRepository(pool *pgxpool.Pool, v validator.Validator, l *zerol
|
||||
}
|
||||
}
|
||||
|
||||
func (d *dispatcherRepository) CreateNewDispatcher(opts *repository.CreateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
|
||||
func (d *dispatcherRepository) CreateNewDispatcher(ctx context.Context, opts *repository.CreateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
|
||||
if err := d.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return d.queries.CreateDispatcher(context.Background(), d.pool, sqlchelpers.UUIDFromStr(opts.ID))
|
||||
return d.queries.CreateDispatcher(ctx, d.pool, sqlchelpers.UUIDFromStr(opts.ID))
|
||||
}
|
||||
|
||||
func (d *dispatcherRepository) UpdateDispatcher(dispatcherId string, opts *repository.UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
|
||||
func (d *dispatcherRepository) UpdateDispatcher(ctx context.Context, dispatcherId string, opts *repository.UpdateDispatcherOpts) (*dbsqlc.Dispatcher, error) {
|
||||
if err := d.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return d.queries.UpdateDispatcher(context.Background(), d.pool, dbsqlc.UpdateDispatcherParams{
|
||||
return d.queries.UpdateDispatcher(ctx, d.pool, dbsqlc.UpdateDispatcherParams{
|
||||
ID: sqlchelpers.UUIDFromStr(dispatcherId),
|
||||
LastHeartbeatAt: sqlchelpers.TimestampFromTime(opts.LastHeartbeatAt.UTC()),
|
||||
})
|
||||
}
|
||||
|
||||
func (d *dispatcherRepository) Delete(dispatcherId string) error {
|
||||
_, err := d.queries.DeleteDispatcher(context.Background(), d.pool, sqlchelpers.UUIDFromStr(dispatcherId))
|
||||
func (d *dispatcherRepository) Delete(ctx context.Context, dispatcherId string) error {
|
||||
_, err := d.queries.DeleteDispatcher(ctx, d.pool, sqlchelpers.UUIDFromStr(dispatcherId))
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func (d *dispatcherRepository) UpdateStaleDispatchers(onStale func(dispatcherId string, getValidDispatcherId func() string) error) error {
|
||||
tx, err := d.pool.Begin(context.Background())
|
||||
func (d *dispatcherRepository) UpdateStaleDispatchers(ctx context.Context, onStale func(dispatcherId string, getValidDispatcherId func() string) error) error {
|
||||
tx, err := d.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
@@ -206,8 +206,8 @@ func NewEventEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zero
|
||||
}
|
||||
}
|
||||
|
||||
func (r *eventEngineRepository) GetEventForEngine(tenantId, id string) (*dbsqlc.Event, error) {
|
||||
return r.queries.GetEventForEngine(context.Background(), r.pool, sqlchelpers.UUIDFromStr(id))
|
||||
func (r *eventEngineRepository) GetEventForEngine(ctx context.Context, tenantId, id string) (*dbsqlc.Event, error) {
|
||||
return r.queries.GetEventForEngine(ctx, r.pool, sqlchelpers.UUIDFromStr(id))
|
||||
}
|
||||
|
||||
func (r *eventEngineRepository) CreateEvent(ctx context.Context, opts *repository.CreateEventOpts) (*dbsqlc.Event, error) {
|
||||
@@ -242,7 +242,7 @@ func (r *eventEngineRepository) CreateEvent(ctx context.Context, opts *repositor
|
||||
return e, nil
|
||||
}
|
||||
|
||||
func (r *eventEngineRepository) ListEventsByIds(tenantId string, ids []string) ([]*dbsqlc.Event, error) {
|
||||
func (r *eventEngineRepository) ListEventsByIds(ctx context.Context, tenantId string, ids []string) ([]*dbsqlc.Event, error) {
|
||||
pgIds := make([]pgtype.UUID, len(ids))
|
||||
|
||||
for i, id := range ids {
|
||||
@@ -253,7 +253,7 @@ func (r *eventEngineRepository) ListEventsByIds(tenantId string, ids []string) (
|
||||
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
return r.queries.ListEventsByIDs(context.Background(), r.pool, dbsqlc.ListEventsByIDsParams{
|
||||
return r.queries.ListEventsByIDs(ctx, r.pool, dbsqlc.ListEventsByIDsParams{
|
||||
Tenantid: pgTenantId,
|
||||
Ids: pgIds,
|
||||
})
|
||||
|
||||
@@ -34,20 +34,20 @@ func NewGetGroupKeyRunRepository(pool *pgxpool.Pool, v validator.Validator, l *z
|
||||
}
|
||||
}
|
||||
|
||||
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToRequeue(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
|
||||
return s.queries.ListGetGroupKeyRunsToRequeue(context.Background(), s.pool, sqlchelpers.UUIDFromStr(tenantId))
|
||||
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
|
||||
return s.queries.ListGetGroupKeyRunsToRequeue(ctx, s.pool, sqlchelpers.UUIDFromStr(tenantId))
|
||||
}
|
||||
|
||||
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToReassign(tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
|
||||
return s.queries.ListGetGroupKeyRunsToReassign(context.Background(), s.pool, sqlchelpers.UUIDFromStr(tenantId))
|
||||
func (s *getGroupKeyRunRepository) ListGetGroupKeyRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetGroupKeyRun, error) {
|
||||
return s.queries.ListGetGroupKeyRunsToReassign(ctx, s.pool, sqlchelpers.UUIDFromStr(tenantId))
|
||||
}
|
||||
|
||||
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToWorker(tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error) {
|
||||
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToWorker(ctx context.Context, tenantId, getGroupKeyRunId string) (workerId string, dispatcherId string, err error) {
|
||||
// var assigned
|
||||
var assigned *dbsqlc.AssignGetGroupKeyRunToWorkerRow
|
||||
|
||||
err = retrier(s.l, func() (err error) {
|
||||
assigned, err = s.queries.AssignGetGroupKeyRunToWorker(context.Background(), s.pool, dbsqlc.AssignGetGroupKeyRunToWorkerParams{
|
||||
assigned, err = s.queries.AssignGetGroupKeyRunToWorker(ctx, s.pool, dbsqlc.AssignGetGroupKeyRunToWorkerParams{
|
||||
Getgroupkeyrunid: sqlchelpers.UUIDFromStr(getGroupKeyRunId),
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
@@ -70,12 +70,12 @@ func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToWorker(tenantId, getGro
|
||||
return sqlchelpers.UUIDToStr(assigned.WorkerId), sqlchelpers.UUIDToStr(assigned.DispatcherId), nil
|
||||
}
|
||||
|
||||
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToTicker(tenantId, getGroupKeyRunId string) (tickerId string, err error) {
|
||||
func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToTicker(ctx context.Context, tenantId, getGroupKeyRunId string) (tickerId string, err error) {
|
||||
// var assigned
|
||||
var assigned *dbsqlc.AssignGetGroupKeyRunToTickerRow
|
||||
|
||||
err = retrier(s.l, func() (err error) {
|
||||
assigned, err = s.queries.AssignGetGroupKeyRunToTicker(context.Background(), s.pool, dbsqlc.AssignGetGroupKeyRunToTickerParams{
|
||||
assigned, err = s.queries.AssignGetGroupKeyRunToTicker(ctx, s.pool, dbsqlc.AssignGetGroupKeyRunToTickerParams{
|
||||
Getgroupkeyrunid: sqlchelpers.UUIDFromStr(getGroupKeyRunId),
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
@@ -98,7 +98,7 @@ func (s *getGroupKeyRunRepository) AssignGetGroupKeyRunToTicker(tenantId, getGro
|
||||
return sqlchelpers.UUIDToStr(assigned.TickerId), nil
|
||||
}
|
||||
|
||||
func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId string, opts *repository.UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
|
||||
func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(ctx context.Context, tenantId, getGroupKeyRunId string, opts *repository.UpdateGetGroupKeyRunOpts) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
|
||||
if err := s.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -158,15 +158,15 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
|
||||
updateParams.ScheduleTimeoutAt = sqlchelpers.TimestampFromTime(*opts.ScheduleTimeoutAt)
|
||||
}
|
||||
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
res1, err := s.queries.UpdateGetGroupKeyRun(context.Background(), tx, updateParams)
|
||||
res1, err := s.queries.UpdateGetGroupKeyRun(ctx, tx, updateParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not update get group key run: %w", err)
|
||||
@@ -174,14 +174,14 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
|
||||
|
||||
// only update workflow run if status or output has changed
|
||||
if opts.Status != nil || opts.Output != nil {
|
||||
_, err = s.queries.UpdateWorkflowRunGroupKey(context.Background(), tx, updateWorkflowRunParams)
|
||||
_, err = s.queries.UpdateWorkflowRunGroupKey(ctx, tx, updateWorkflowRunParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not resolve workflow run status from get group key run: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
getGroupKeyRuns, err := s.queries.GetGroupKeyRunForEngine(context.Background(), tx, dbsqlc.GetGroupKeyRunForEngineParams{
|
||||
getGroupKeyRuns, err := s.queries.GetGroupKeyRunForEngine(ctx, tx, dbsqlc.GetGroupKeyRunForEngineParams{
|
||||
Ids: []pgtype.UUID{res1.ID},
|
||||
Tenantid: pgTenantId,
|
||||
})
|
||||
@@ -194,7 +194,7 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
|
||||
return nil, fmt.Errorf("could not find get group key run for engine")
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -203,8 +203,8 @@ func (s *getGroupKeyRunRepository) UpdateGetGroupKeyRun(tenantId, getGroupKeyRun
|
||||
return getGroupKeyRuns[0], nil
|
||||
}
|
||||
|
||||
func (s *getGroupKeyRunRepository) GetGroupKeyRunForEngine(tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
|
||||
res, err := s.queries.GetGroupKeyRunForEngine(context.Background(), s.pool, dbsqlc.GetGroupKeyRunForEngineParams{
|
||||
func (s *getGroupKeyRunRepository) GetGroupKeyRunForEngine(ctx context.Context, tenantId, getGroupKeyRunId string) (*dbsqlc.GetGroupKeyRunForEngineRow, error) {
|
||||
res, err := s.queries.GetGroupKeyRunForEngine(ctx, s.pool, dbsqlc.GetGroupKeyRunForEngineParams{
|
||||
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(getGroupKeyRunId)},
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
|
||||
@@ -56,16 +56,16 @@ func NewJobRunEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zer
|
||||
}
|
||||
}
|
||||
|
||||
func (j *jobRunEngineRepository) SetJobRunStatusRunning(tenantId, jobRunId string) error {
|
||||
return setJobRunStatusRunning(context.Background(), j.pool, j.queries, j.l, tenantId, jobRunId)
|
||||
func (j *jobRunEngineRepository) SetJobRunStatusRunning(ctx context.Context, tenantId, jobRunId string) error {
|
||||
return setJobRunStatusRunning(ctx, j.pool, j.queries, j.l, tenantId, jobRunId)
|
||||
}
|
||||
|
||||
func (j *jobRunEngineRepository) ListJobRunsForWorkflowRun(tenantId, workflowRunId string) ([]pgtype.UUID, error) {
|
||||
return j.queries.ListJobRunsForWorkflowRun(context.Background(), j.pool, sqlchelpers.UUIDFromStr(workflowRunId))
|
||||
func (j *jobRunEngineRepository) ListJobRunsForWorkflowRun(ctx context.Context, tenantId, workflowRunId string) ([]pgtype.UUID, error) {
|
||||
return j.queries.ListJobRunsForWorkflowRun(ctx, j.pool, sqlchelpers.UUIDFromStr(workflowRunId))
|
||||
}
|
||||
|
||||
func setJobRunStatusRunning(ctx context.Context, pool *pgxpool.Pool, queries *dbsqlc.Queries, l *zerolog.Logger, tenantId, jobRunId string) error {
|
||||
tx, err := pool.Begin(context.Background())
|
||||
tx, err := pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
|
||||
@@ -150,7 +150,7 @@ func NewLogEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zerolo
|
||||
}
|
||||
}
|
||||
|
||||
func (r *logEngineRepository) PutLog(tenantId string, opts *repository.CreateLogLineOpts) (*dbsqlc.LogLine, error) {
|
||||
func (r *logEngineRepository) PutLog(ctx context.Context, tenantId string, opts *repository.CreateLogLineOpts) (*dbsqlc.LogLine, error) {
|
||||
if err := r.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -177,16 +177,16 @@ func (r *logEngineRepository) PutLog(tenantId string, opts *repository.CreateLog
|
||||
createParams.Metadata = opts.Metadata
|
||||
}
|
||||
|
||||
tx, err := r.pool.Begin(context.Background())
|
||||
tx, err := r.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), r.l, tx.Rollback)
|
||||
defer deferRollback(ctx, r.l, tx.Rollback)
|
||||
|
||||
logLine, err := r.queries.CreateLogLine(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
createParams,
|
||||
)
|
||||
@@ -195,7 +195,7 @@ func (r *logEngineRepository) PutLog(tenantId string, opts *repository.CreateLog
|
||||
return nil, fmt.Errorf("could not create log line: %w", err)
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not commit transaction: %w", err)
|
||||
|
||||
@@ -31,7 +31,7 @@ func NewRateLimitEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *
|
||||
}
|
||||
}
|
||||
|
||||
func (r *rateLimitEngineRepository) UpsertRateLimit(tenantId string, key string, opts *repository.UpsertRateLimitOpts) (*dbsqlc.RateLimit, error) {
|
||||
func (r *rateLimitEngineRepository) UpsertRateLimit(ctx context.Context, tenantId string, key string, opts *repository.UpsertRateLimitOpts) (*dbsqlc.RateLimit, error) {
|
||||
if err := r.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -46,7 +46,7 @@ func (r *rateLimitEngineRepository) UpsertRateLimit(tenantId string, key string,
|
||||
upsertParams.Window = sqlchelpers.TextFromStr(fmt.Sprintf("1 %s", *opts.Duration))
|
||||
}
|
||||
|
||||
rateLimit, err := r.queries.UpsertRateLimit(context.Background(), r.pool, upsertParams)
|
||||
rateLimit, err := r.queries.UpsertRateLimit(ctx, r.pool, upsertParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not upsert rate limit: %w", err)
|
||||
|
||||
@@ -91,18 +91,19 @@ func NewStepRunEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *ze
|
||||
}
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(tickerId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(ctx context.Context, tickerId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
srs, err := s.queries.ListStepRuns(context.Background(), s.pool, dbsqlc.ListStepRunsParams{
|
||||
srs, err := s.queries.ListStepRuns(ctx, tx, dbsqlc.ListStepRunsParams{
|
||||
Status: dbsqlc.NullStepRunStatus{
|
||||
StepRunStatus: dbsqlc.StepRunStatusRUNNING,
|
||||
Valid: true,
|
||||
},
|
||||
TickerId: sqlchelpers.UUIDFromStr(tickerId),
|
||||
})
|
||||
@@ -111,7 +112,7 @@ func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(tickerId string)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
res, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
|
||||
res, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
|
||||
Ids: srs,
|
||||
})
|
||||
|
||||
@@ -119,32 +120,50 @@ func (s *stepRunEngineRepository) ListRunningStepRunsForTicker(tickerId string)
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
return res, err
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) ListRunningStepRunsForWorkflowRun(tenantId, workflowRunId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
func (s *stepRunEngineRepository) ListStepRuns(ctx context.Context, tenantId string, opts *repository.ListStepRunsOpts) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
if err := s.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
srs, err := s.queries.ListStepRuns(context.Background(), s.pool, dbsqlc.ListStepRunsParams{
|
||||
Status: dbsqlc.NullStepRunStatus{
|
||||
StepRunStatus: dbsqlc.StepRunStatusRUNNING,
|
||||
},
|
||||
WorkflowRunId: sqlchelpers.UUIDFromStr(workflowRunId),
|
||||
})
|
||||
listOpts := dbsqlc.ListStepRunsParams{
|
||||
TenantId: sqlchelpers.UUIDFromStr(tenantId),
|
||||
}
|
||||
|
||||
if opts.Status != nil {
|
||||
listOpts.Status = dbsqlc.NullStepRunStatus{
|
||||
StepRunStatus: *opts.Status,
|
||||
Valid: true,
|
||||
}
|
||||
}
|
||||
|
||||
if opts.WorkflowRunIds != nil {
|
||||
listOpts.WorkflowRunIds = make([]pgtype.UUID, len(opts.WorkflowRunIds))
|
||||
|
||||
for i, id := range opts.WorkflowRunIds {
|
||||
listOpts.WorkflowRunIds[i] = sqlchelpers.UUIDFromStr(id)
|
||||
}
|
||||
}
|
||||
|
||||
srs, err := s.queries.ListStepRuns(ctx, tx, listOpts)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
res, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
|
||||
res, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
|
||||
Ids: srs,
|
||||
})
|
||||
|
||||
@@ -152,30 +171,30 @@ func (s *stepRunEngineRepository) ListRunningStepRunsForWorkflowRun(tenantId, wo
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
return res, err
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) ListStepRunsToRequeue(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
func (s *stepRunEngineRepository) ListStepRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
// get the step run and make sure it's still in pending
|
||||
stepRunIds, err := s.queries.ListStepRunsToRequeue(context.Background(), tx, pgTenantId)
|
||||
stepRunIds, err := s.queries.ListStepRunsToRequeue(ctx, tx, pgTenantId)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
stepRuns, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
|
||||
stepRuns, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
|
||||
Ids: stepRunIds,
|
||||
TenantId: pgTenantId,
|
||||
})
|
||||
@@ -184,7 +203,7 @@ func (s *stepRunEngineRepository) ListStepRunsToRequeue(tenantId string) ([]*dbs
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -193,25 +212,25 @@ func (s *stepRunEngineRepository) ListStepRunsToRequeue(tenantId string) ([]*dbs
|
||||
return stepRuns, nil
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) ListStepRunsToReassign(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
func (s *stepRunEngineRepository) ListStepRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
// get the step run and make sure it's still in pending
|
||||
stepRunIds, err := s.queries.ListStepRunsToReassign(context.Background(), tx, pgTenantId)
|
||||
stepRunIds, err := s.queries.ListStepRunsToReassign(ctx, tx, pgTenantId)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
stepRuns, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
|
||||
stepRuns, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
|
||||
Ids: stepRunIds,
|
||||
TenantId: pgTenantId,
|
||||
})
|
||||
@@ -220,7 +239,7 @@ func (s *stepRunEngineRepository) ListStepRunsToReassign(tenantId string) ([]*db
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -266,13 +285,13 @@ var retrier = func(l *zerolog.Logger, f func() error) error {
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) AssignStepRunToWorker(ctx context.Context, stepRun *dbsqlc.GetStepRunForEngineRow) (string, string, error) {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
// Update the old worker semaphore. This will only increment if the step run was already assigned to a worker,
|
||||
// which means the step run is being retried or rerun.
|
||||
@@ -336,7 +355,7 @@ func (s *stepRunEngineRepository) AssignStepRunToWorker(ctx context.Context, ste
|
||||
}
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return "", "", err
|
||||
@@ -364,13 +383,13 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
|
||||
var stepRun *dbsqlc.GetStepRunForEngineRow
|
||||
|
||||
err = retrier(s.l, func() error {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
stepRun, err = s.updateStepRunCore(ctx, tx, tenantId, updateParams, updateJobRunLookupDataParams)
|
||||
|
||||
@@ -378,7 +397,7 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
|
||||
return err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
return err
|
||||
})
|
||||
@@ -388,13 +407,13 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
|
||||
}
|
||||
|
||||
err = retrier(s.l, func() error {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
updateInfo, err = s.updateStepRunExtra(ctx, tx, tenantId, resolveJobRunParams, resolveLaterStepRunsParams)
|
||||
|
||||
@@ -402,7 +421,7 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
|
||||
return err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
return err
|
||||
})
|
||||
@@ -414,18 +433,18 @@ func (s *stepRunEngineRepository) UpdateStepRun(ctx context.Context, tenantId, s
|
||||
return stepRun, updateInfo, nil
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId string, opts *repository.UpdateStepRunOverridesDataOpts) ([]byte, error) {
|
||||
func (s *stepRunEngineRepository) UpdateStepRunOverridesData(ctx context.Context, tenantId, stepRunId string, opts *repository.UpdateStepRunOverridesDataOpts) ([]byte, error) {
|
||||
if err := s.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -441,7 +460,7 @@ func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId
|
||||
}
|
||||
|
||||
input, err := s.queries.UpdateStepRunOverridesData(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.UpdateStepRunOverridesDataParams{
|
||||
Steprunid: pgStepRunId,
|
||||
@@ -462,7 +481,7 @@ func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId
|
||||
return nil, fmt.Errorf("could not update step run overrides data: %w", err)
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -471,20 +490,20 @@ func (s *stepRunEngineRepository) UpdateStepRunOverridesData(tenantId, stepRunId
|
||||
return input, nil
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) UpdateStepRunInputSchema(tenantId, stepRunId string, schema []byte) ([]byte, error) {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
func (s *stepRunEngineRepository) UpdateStepRunInputSchema(ctx context.Context, tenantId, stepRunId string, schema []byte) ([]byte, error) {
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
pgStepRunId := sqlchelpers.UUIDFromStr(stepRunId)
|
||||
|
||||
inputSchema, err := s.queries.UpdateStepRunInputSchema(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.UpdateStepRunInputSchemaParams{
|
||||
Steprunid: pgStepRunId,
|
||||
@@ -497,7 +516,7 @@ func (s *stepRunEngineRepository) UpdateStepRunInputSchema(tenantId, stepRunId s
|
||||
return nil, fmt.Errorf("could not update step run input schema: %w", err)
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -524,16 +543,16 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
|
||||
var isNotPending bool
|
||||
|
||||
retrierErr := retrier(s.l, func() error {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
// get the step run and make sure it's still in pending
|
||||
innerStepRun, err := s.queries.GetStepRun(context.Background(), tx, dbsqlc.GetStepRunParams{
|
||||
innerStepRun, err := s.queries.GetStepRun(ctx, tx, dbsqlc.GetStepRunParams{
|
||||
ID: sqlchelpers.UUIDFromStr(stepRunId),
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
@@ -560,7 +579,7 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
|
||||
return err
|
||||
}
|
||||
|
||||
if err := tx.Commit(context.Background()); err != nil {
|
||||
if err := tx.Commit(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
@@ -576,13 +595,13 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
|
||||
}
|
||||
|
||||
retrierExtraErr := retrier(s.l, func() error {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
_, err = s.updateStepRunExtra(ctx, tx, tenantId, resolveJobRunParams, resolveLaterStepRunsParams)
|
||||
|
||||
@@ -590,7 +609,7 @@ func (s *stepRunEngineRepository) QueueStepRun(ctx context.Context, tenantId, st
|
||||
return err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
return err
|
||||
})
|
||||
@@ -763,13 +782,13 @@ func (s *stepRunEngineRepository) updateStepRunExtra(
|
||||
ctx, span := telemetry.NewSpan(ctx, "update-step-run-extra") // nolint:ineffassign
|
||||
defer span.End()
|
||||
|
||||
_, err := s.queries.ResolveLaterStepRuns(context.Background(), tx, resolveLaterStepRunsParams)
|
||||
_, err := s.queries.ResolveLaterStepRuns(ctx, tx, resolveLaterStepRunsParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not resolve later step runs: %w", err)
|
||||
}
|
||||
|
||||
jobRun, err := s.queries.ResolveJobRunStatus(context.Background(), tx, resolveJobRunParams)
|
||||
jobRun, err := s.queries.ResolveJobRunStatus(ctx, tx, resolveJobRunParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not resolve job run status: %w", err)
|
||||
@@ -780,7 +799,7 @@ func (s *stepRunEngineRepository) updateStepRunExtra(
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
}
|
||||
|
||||
workflowRun, err := s.queries.ResolveWorkflowRunStatus(context.Background(), tx, resolveWorkflowRunParams)
|
||||
workflowRun, err := s.queries.ResolveWorkflowRunStatus(ctx, tx, resolveWorkflowRunParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not resolve workflow run status: %w", err)
|
||||
@@ -810,8 +829,8 @@ func isFinalWorkflowRunStatus(status dbsqlc.WorkflowRunStatus) bool {
|
||||
}
|
||||
|
||||
// performant query for step run id, only returns what the engine needs
|
||||
func (s *stepRunEngineRepository) GetStepRunForEngine(tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
res, err := s.queries.GetStepRunForEngine(context.Background(), s.pool, dbsqlc.GetStepRunForEngineParams{
|
||||
func (s *stepRunEngineRepository) GetStepRunForEngine(ctx context.Context, tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
res, err := s.queries.GetStepRunForEngine(ctx, s.pool, dbsqlc.GetStepRunForEngineParams{
|
||||
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(stepRunId)},
|
||||
TenantId: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
@@ -827,14 +846,14 @@ func (s *stepRunEngineRepository) GetStepRunForEngine(tenantId, stepRunId string
|
||||
return res[0], nil
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) ListStartableStepRuns(tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
tx, err := s.pool.Begin(context.Background())
|
||||
func (s *stepRunEngineRepository) ListStartableStepRuns(ctx context.Context, tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error) {
|
||||
tx, err := s.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), s.l, tx.Rollback)
|
||||
defer deferRollback(ctx, s.l, tx.Rollback)
|
||||
|
||||
params := dbsqlc.ListStartableStepRunsParams{
|
||||
Jobrunid: sqlchelpers.UUIDFromStr(jobRunId),
|
||||
@@ -844,13 +863,13 @@ func (s *stepRunEngineRepository) ListStartableStepRuns(tenantId, jobRunId strin
|
||||
params.ParentStepRunId = sqlchelpers.UUIDFromStr(*parentStepRunId)
|
||||
}
|
||||
|
||||
srs, err := s.queries.ListStartableStepRuns(context.Background(), tx, params)
|
||||
srs, err := s.queries.ListStartableStepRuns(ctx, tx, params)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
res, err := s.queries.GetStepRunForEngine(context.Background(), tx, dbsqlc.GetStepRunForEngineParams{
|
||||
res, err := s.queries.GetStepRunForEngine(ctx, tx, dbsqlc.GetStepRunForEngineParams{
|
||||
Ids: srs,
|
||||
TenantId: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
@@ -859,13 +878,13 @@ func (s *stepRunEngineRepository) ListStartableStepRuns(tenantId, jobRunId strin
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
return res, err
|
||||
}
|
||||
|
||||
func (s *stepRunEngineRepository) ArchiveStepRunResult(tenantId, stepRunId string) error {
|
||||
_, err := s.queries.ArchiveStepRunResultFromStepRun(context.Background(), s.pool, dbsqlc.ArchiveStepRunResultFromStepRunParams{
|
||||
func (s *stepRunEngineRepository) ArchiveStepRunResult(ctx context.Context, tenantId, stepRunId string) error {
|
||||
_, err := s.queries.ArchiveStepRunResultFromStepRun(ctx, s.pool, dbsqlc.ArchiveStepRunResultFromStepRunParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Steprunid: sqlchelpers.UUIDFromStr(stepRunId),
|
||||
})
|
||||
|
||||
@@ -33,7 +33,7 @@ func NewStreamEventsEngineRepository(pool *pgxpool.Pool, v validator.Validator,
|
||||
}
|
||||
}
|
||||
|
||||
func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repository.CreateStreamEventOpts) (*dbsqlc.StreamEvent, error) {
|
||||
func (r *streamEventEngineRepository) PutStreamEvent(ctx context.Context, tenantId string, opts *repository.CreateStreamEventOpts) (*dbsqlc.StreamEvent, error) {
|
||||
if err := r.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -53,16 +53,16 @@ func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repo
|
||||
createParams.Metadata = opts.Metadata
|
||||
}
|
||||
|
||||
tx, err := r.pool.Begin(context.Background())
|
||||
tx, err := r.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), r.l, tx.Rollback)
|
||||
defer deferRollback(ctx, r.l, tx.Rollback)
|
||||
|
||||
streamEvent, err := r.queries.CreateStreamEvent(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
createParams,
|
||||
)
|
||||
@@ -71,7 +71,7 @@ func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repo
|
||||
return nil, fmt.Errorf("could not create stream event: %w", err)
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not commit transaction: %w", err)
|
||||
@@ -80,16 +80,16 @@ func (r *streamEventEngineRepository) PutStreamEvent(tenantId string, opts *repo
|
||||
return streamEvent, nil
|
||||
}
|
||||
|
||||
func (r *streamEventEngineRepository) GetStreamEvent(tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error) {
|
||||
func (r *streamEventEngineRepository) GetStreamEvent(ctx context.Context, tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error) {
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
tx, err := r.pool.Begin(context.Background())
|
||||
tx, err := r.pool.Begin(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer deferRollback(context.Background(), r.l, tx.Rollback)
|
||||
defer deferRollback(ctx, r.l, tx.Rollback)
|
||||
|
||||
streamEvent, err := r.queries.GetStreamEvent(context.Background(), tx, dbsqlc.GetStreamEventParams{
|
||||
streamEvent, err := r.queries.GetStreamEvent(ctx, tx, dbsqlc.GetStreamEventParams{
|
||||
ID: streamEventId,
|
||||
Tenantid: pgTenantId,
|
||||
})
|
||||
@@ -101,7 +101,7 @@ func (r *streamEventEngineRepository) GetStreamEvent(tenantId string, streamEven
|
||||
return nil, fmt.Errorf("could not get stream event: %w", err)
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not commit transaction: %w", err)
|
||||
@@ -110,15 +110,15 @@ func (r *streamEventEngineRepository) GetStreamEvent(tenantId string, streamEven
|
||||
return streamEvent, nil
|
||||
}
|
||||
|
||||
func (r *streamEventEngineRepository) CleanupStreamEvents() error {
|
||||
tx, err := r.pool.Begin(context.Background())
|
||||
func (r *streamEventEngineRepository) CleanupStreamEvents(ctx context.Context) error {
|
||||
tx, err := r.pool.Begin(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), r.l, tx.Rollback)
|
||||
defer deferRollback(ctx, r.l, tx.Rollback)
|
||||
|
||||
err = r.queries.CleanupStreamEvents(context.Background(), r.pool)
|
||||
err = r.queries.CleanupStreamEvents(ctx, r.pool)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not cleanup stream events: %w", err)
|
||||
|
||||
@@ -151,12 +151,12 @@ func NewTenantEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zer
|
||||
}
|
||||
}
|
||||
|
||||
func (r *tenantEngineRepository) ListTenants() ([]*dbsqlc.Tenant, error) {
|
||||
return r.queries.ListTenants(context.Background(), r.pool)
|
||||
func (r *tenantEngineRepository) ListTenants(ctx context.Context) ([]*dbsqlc.Tenant, error) {
|
||||
return r.queries.ListTenants(ctx, r.pool)
|
||||
}
|
||||
|
||||
func (r *tenantEngineRepository) GetTenantByID(tenantId string) (*dbsqlc.Tenant, error) {
|
||||
func (r *tenantEngineRepository) GetTenantByID(ctx context.Context, tenantId string) (*dbsqlc.Tenant, error) {
|
||||
return cache.MakeCacheable[dbsqlc.Tenant](r.cache, tenantId, func() (*dbsqlc.Tenant, error) {
|
||||
return r.queries.GetTenantByID(context.Background(), r.pool, sqlchelpers.UUIDFromStr(tenantId))
|
||||
return r.queries.GetTenantByID(ctx, r.pool, sqlchelpers.UUIDFromStr(tenantId))
|
||||
})
|
||||
}
|
||||
|
||||
@@ -30,21 +30,21 @@ func NewTickerRepository(pool *pgxpool.Pool, v validator.Validator, l *zerolog.L
|
||||
}
|
||||
}
|
||||
|
||||
func (t *tickerRepository) CreateNewTicker(opts *repository.CreateTickerOpts) (*dbsqlc.Ticker, error) {
|
||||
func (t *tickerRepository) CreateNewTicker(ctx context.Context, opts *repository.CreateTickerOpts) (*dbsqlc.Ticker, error) {
|
||||
if err := t.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return t.queries.CreateTicker(context.Background(), t.pool, sqlchelpers.UUIDFromStr(opts.ID))
|
||||
return t.queries.CreateTicker(ctx, t.pool, sqlchelpers.UUIDFromStr(opts.ID))
|
||||
}
|
||||
|
||||
func (t *tickerRepository) UpdateTicker(tickerId string, opts *repository.UpdateTickerOpts) (*dbsqlc.Ticker, error) {
|
||||
func (t *tickerRepository) UpdateTicker(ctx context.Context, tickerId string, opts *repository.UpdateTickerOpts) (*dbsqlc.Ticker, error) {
|
||||
if err := t.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return t.queries.UpdateTicker(
|
||||
context.Background(),
|
||||
ctx,
|
||||
t.pool,
|
||||
dbsqlc.UpdateTickerParams{
|
||||
ID: sqlchelpers.UUIDFromStr(tickerId),
|
||||
@@ -53,7 +53,7 @@ func (t *tickerRepository) UpdateTicker(tickerId string, opts *repository.Update
|
||||
)
|
||||
}
|
||||
|
||||
func (t *tickerRepository) ListTickers(opts *repository.ListTickerOpts) ([]*dbsqlc.Ticker, error) {
|
||||
func (t *tickerRepository) ListTickers(ctx context.Context, opts *repository.ListTickerOpts) ([]*dbsqlc.Ticker, error) {
|
||||
if err := t.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -69,15 +69,15 @@ func (t *tickerRepository) ListTickers(opts *repository.ListTickerOpts) ([]*dbsq
|
||||
}
|
||||
|
||||
return t.queries.ListTickers(
|
||||
context.Background(),
|
||||
ctx,
|
||||
t.pool,
|
||||
params,
|
||||
)
|
||||
}
|
||||
|
||||
func (t *tickerRepository) Delete(tickerId string) error {
|
||||
func (t *tickerRepository) Delete(ctx context.Context, tickerId string) error {
|
||||
_, err := t.queries.DeleteTicker(
|
||||
context.Background(),
|
||||
ctx,
|
||||
t.pool,
|
||||
sqlchelpers.UUIDFromStr(tickerId),
|
||||
)
|
||||
@@ -85,163 +85,18 @@ func (t *tickerRepository) Delete(tickerId string) error {
|
||||
return err
|
||||
}
|
||||
|
||||
func (t *tickerRepository) PollStepRuns(tickerId string) ([]*dbsqlc.StepRun, error) {
|
||||
return t.queries.PollStepRuns(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
func (t *tickerRepository) PollStepRuns(ctx context.Context, tickerId string) ([]*dbsqlc.StepRun, error) {
|
||||
return t.queries.PollStepRuns(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
}
|
||||
|
||||
func (t *tickerRepository) PollGetGroupKeyRuns(tickerId string) ([]*dbsqlc.GetGroupKeyRun, error) {
|
||||
return t.queries.PollGetGroupKeyRuns(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
func (t *tickerRepository) PollGetGroupKeyRuns(ctx context.Context, tickerId string) ([]*dbsqlc.GetGroupKeyRun, error) {
|
||||
return t.queries.PollGetGroupKeyRuns(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
}
|
||||
|
||||
func (t *tickerRepository) PollCronSchedules(tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error) {
|
||||
return t.queries.PollCronSchedules(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
func (t *tickerRepository) PollCronSchedules(ctx context.Context, tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error) {
|
||||
return t.queries.PollCronSchedules(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
}
|
||||
|
||||
func (t *tickerRepository) PollScheduledWorkflows(tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error) {
|
||||
return t.queries.PollScheduledWorkflows(context.Background(), t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
func (t *tickerRepository) PollScheduledWorkflows(ctx context.Context, tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error) {
|
||||
return t.queries.PollScheduledWorkflows(ctx, t.pool, sqlchelpers.UUIDFromStr(tickerId))
|
||||
}
|
||||
|
||||
// func (t *tickerRepository) AddJobRun(tickerId string, jobRun *db.JobRunModel) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.JobRuns.Link(
|
||||
// db.JobRun.ID.Equals(jobRun.ID),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) AddStepRun(tickerId, stepRunId string) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.StepRuns.Link(
|
||||
// db.StepRun.ID.Equals(stepRunId),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) AddGetGroupKeyRun(tickerId, getGroupKeyRunId string) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.GroupKeyRuns.Link(
|
||||
// db.GetGroupKeyRun.ID.Equals(getGroupKeyRunId),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) AddCron(tickerId string, cron *db.WorkflowTriggerCronRefModel) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.Crons.Link(
|
||||
// db.WorkflowTriggerCronRef.ParentIDCron(
|
||||
// db.WorkflowTriggerCronRef.ParentID.Equals(cron.ParentID),
|
||||
// db.WorkflowTriggerCronRef.Cron.Equals(cron.Cron),
|
||||
// ),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) RemoveCron(tickerId string, cron *db.WorkflowTriggerCronRefModel) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.Crons.Unlink(
|
||||
// db.WorkflowTriggerCronRef.ParentIDCron(
|
||||
// db.WorkflowTriggerCronRef.ParentID.Equals(cron.ParentID),
|
||||
// db.WorkflowTriggerCronRef.Cron.Equals(cron.Cron),
|
||||
// ),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) AddScheduledWorkflow(tickerId string, schedule *db.WorkflowTriggerScheduledRefModel) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.Scheduled.Link(
|
||||
// db.WorkflowTriggerScheduledRef.ID.Equals(schedule.ID),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) RemoveScheduledWorkflow(tickerId string, schedule *db.WorkflowTriggerScheduledRefModel) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).Update(
|
||||
// db.Ticker.Scheduled.Unlink(
|
||||
// db.WorkflowTriggerScheduledRef.ID.Equals(schedule.ID),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) GetTickerById(tickerId string) (*db.TickerModel, error) {
|
||||
// return t.client.Ticker.FindUnique(
|
||||
// db.Ticker.ID.Equals(tickerId),
|
||||
// ).With(
|
||||
// db.Ticker.Crons.Fetch().With(
|
||||
// db.WorkflowTriggerCronRef.Parent.Fetch().With(
|
||||
// db.WorkflowTriggers.Workflow.Fetch().With(
|
||||
// db.WorkflowVersion.Workflow.Fetch(),
|
||||
// ),
|
||||
// ),
|
||||
// ),
|
||||
// db.Ticker.Scheduled.Fetch().With(
|
||||
// db.WorkflowTriggerScheduledRef.Parent.Fetch().With(
|
||||
// db.WorkflowVersion.Workflow.Fetch(),
|
||||
// ),
|
||||
// ),
|
||||
// ).Exec(context.Background())
|
||||
// }
|
||||
|
||||
// func (t *tickerRepository) UpdateStaleTickers(onStale func(tickerId string, getValidTickerId func() string) error) error {
|
||||
// tx, err := t.pool.Begin(context.Background())
|
||||
|
||||
// if err != nil {
|
||||
// return err
|
||||
// }
|
||||
|
||||
// defer deferRollback(context.Background(), t.l, tx.Rollback)
|
||||
|
||||
// staleTickers, err := t.queries.ListNewlyStaleTickers(context.Background(), tx)
|
||||
|
||||
// if err != nil {
|
||||
// return err
|
||||
// }
|
||||
|
||||
// activeTickers, err := t.queries.ListActiveTickers(context.Background(), tx)
|
||||
|
||||
// if err != nil {
|
||||
// return err
|
||||
// }
|
||||
|
||||
// // if there are no active tickers, we can't reassign the stale tickers
|
||||
// if len(activeTickers) == 0 {
|
||||
// return nil
|
||||
// }
|
||||
|
||||
// tickersToDelete := make([]pgtype.UUID, 0)
|
||||
|
||||
// for i, ticker := range staleTickers {
|
||||
// err := onStale(sqlchelpers.UUIDToStr(ticker.Ticker.ID), func() string {
|
||||
// // assign tickers in round-robin fashion
|
||||
// return sqlchelpers.UUIDToStr(activeTickers[i%len(activeTickers)].Ticker.ID)
|
||||
// })
|
||||
|
||||
// if err != nil {
|
||||
// return err
|
||||
// }
|
||||
|
||||
// tickersToDelete = append(tickersToDelete, ticker.Ticker.ID)
|
||||
// }
|
||||
|
||||
// _, err = t.queries.SetTickersInactive(context.Background(), tx, tickersToDelete)
|
||||
|
||||
// if err != nil {
|
||||
// return err
|
||||
// }
|
||||
|
||||
// return tx.Commit(context.Background())
|
||||
// }
|
||||
|
||||
@@ -138,25 +138,25 @@ func NewWorkerEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *zer
|
||||
}
|
||||
}
|
||||
|
||||
func (w *workerEngineRepository) GetWorkerForEngine(tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error) {
|
||||
return w.queries.GetWorkerForEngine(context.Background(), w.pool, dbsqlc.GetWorkerForEngineParams{
|
||||
func (w *workerEngineRepository) GetWorkerForEngine(ctx context.Context, tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error) {
|
||||
return w.queries.GetWorkerForEngine(ctx, w.pool, dbsqlc.GetWorkerForEngineParams{
|
||||
ID: sqlchelpers.UUIDFromStr(workerId),
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
})
|
||||
}
|
||||
|
||||
func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *repository.CreateWorkerOpts) (*dbsqlc.Worker, error) {
|
||||
func (w *workerEngineRepository) CreateNewWorker(ctx context.Context, tenantId string, opts *repository.CreateWorkerOpts) (*dbsqlc.Worker, error) {
|
||||
if err := w.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tx, err := w.pool.Begin(context.Background())
|
||||
tx, err := w.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), w.l, tx.Rollback)
|
||||
defer deferRollback(ctx, w.l, tx.Rollback)
|
||||
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
@@ -180,7 +180,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
}
|
||||
}
|
||||
|
||||
worker, err := w.queries.CreateWorker(context.Background(), tx, createParams)
|
||||
worker, err := w.queries.CreateWorker(ctx, tx, createParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not create worker: %w", err)
|
||||
@@ -188,7 +188,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
|
||||
createSemParams.Workerid = worker.ID
|
||||
|
||||
_, err = w.queries.CreateWorkerSemaphore(context.Background(), tx, createSemParams)
|
||||
_, err = w.queries.CreateWorkerSemaphore(ctx, tx, createSemParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not create worker semaphore: %w", err)
|
||||
@@ -197,7 +197,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
svcUUIDs := make([]pgtype.UUID, len(opts.Services))
|
||||
|
||||
for i, svc := range opts.Services {
|
||||
dbSvc, err := w.queries.UpsertService(context.Background(), tx, dbsqlc.UpsertServiceParams{
|
||||
dbSvc, err := w.queries.UpsertService(ctx, tx, dbsqlc.UpsertServiceParams{
|
||||
Name: svc,
|
||||
Tenantid: pgTenantId,
|
||||
})
|
||||
@@ -209,7 +209,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
svcUUIDs[i] = dbSvc.ID
|
||||
}
|
||||
|
||||
err = w.queries.LinkServicesToWorker(context.Background(), tx, dbsqlc.LinkServicesToWorkerParams{
|
||||
err = w.queries.LinkServicesToWorker(ctx, tx, dbsqlc.LinkServicesToWorkerParams{
|
||||
Services: svcUUIDs,
|
||||
Workerid: worker.ID,
|
||||
})
|
||||
@@ -221,7 +221,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
actionUUIDs := make([]pgtype.UUID, len(opts.Actions))
|
||||
|
||||
for i, action := range opts.Actions {
|
||||
dbAction, err := w.queries.UpsertAction(context.Background(), tx, dbsqlc.UpsertActionParams{
|
||||
dbAction, err := w.queries.UpsertAction(ctx, tx, dbsqlc.UpsertActionParams{
|
||||
Action: action,
|
||||
Tenantid: pgTenantId,
|
||||
})
|
||||
@@ -233,7 +233,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
actionUUIDs[i] = dbAction.ID
|
||||
}
|
||||
|
||||
err = w.queries.LinkActionsToWorker(context.Background(), tx, dbsqlc.LinkActionsToWorkerParams{
|
||||
err = w.queries.LinkActionsToWorker(ctx, tx, dbsqlc.LinkActionsToWorkerParams{
|
||||
Actionids: actionUUIDs,
|
||||
Workerid: worker.ID,
|
||||
})
|
||||
@@ -242,7 +242,7 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
return nil, fmt.Errorf("could not link actions to worker: %w", err)
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not commit transaction: %w", err)
|
||||
@@ -251,18 +251,18 @@ func (w *workerEngineRepository) CreateNewWorker(tenantId string, opts *reposito
|
||||
return worker, nil
|
||||
}
|
||||
|
||||
func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *repository.UpdateWorkerOpts) (*dbsqlc.Worker, error) {
|
||||
func (w *workerEngineRepository) UpdateWorker(ctx context.Context, tenantId, workerId string, opts *repository.UpdateWorkerOpts) (*dbsqlc.Worker, error) {
|
||||
if err := w.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tx, err := w.pool.Begin(context.Background())
|
||||
tx, err := w.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), w.l, tx.Rollback)
|
||||
defer deferRollback(ctx, w.l, tx.Rollback)
|
||||
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
@@ -285,7 +285,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
|
||||
updateParams.DispatcherId = sqlchelpers.UUIDFromStr(*opts.DispatcherId)
|
||||
}
|
||||
|
||||
worker, err := w.queries.UpdateWorker(context.Background(), tx, updateParams)
|
||||
worker, err := w.queries.UpdateWorker(ctx, tx, updateParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not update worker: %w", err)
|
||||
@@ -295,7 +295,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
|
||||
actionUUIDs := make([]pgtype.UUID, len(opts.Actions))
|
||||
|
||||
for i, action := range opts.Actions {
|
||||
dbAction, err := w.queries.UpsertAction(context.Background(), tx, dbsqlc.UpsertActionParams{
|
||||
dbAction, err := w.queries.UpsertAction(ctx, tx, dbsqlc.UpsertActionParams{
|
||||
Action: action,
|
||||
Tenantid: pgTenantId,
|
||||
})
|
||||
@@ -307,7 +307,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
|
||||
actionUUIDs[i] = dbAction.ID
|
||||
}
|
||||
|
||||
err = w.queries.LinkActionsToWorker(context.Background(), tx, dbsqlc.LinkActionsToWorkerParams{
|
||||
err = w.queries.LinkActionsToWorker(ctx, tx, dbsqlc.LinkActionsToWorkerParams{
|
||||
Actionids: actionUUIDs,
|
||||
Workerid: sqlchelpers.UUIDFromStr(workerId),
|
||||
})
|
||||
@@ -317,7 +317,7 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
|
||||
}
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not commit transaction: %w", err)
|
||||
@@ -326,8 +326,8 @@ func (w *workerEngineRepository) UpdateWorker(tenantId, workerId string, opts *r
|
||||
return worker, nil
|
||||
}
|
||||
|
||||
func (w *workerEngineRepository) DeleteWorker(tenantId, workerId string) error {
|
||||
_, err := w.queries.DeleteWorker(context.Background(), w.pool, sqlchelpers.UUIDFromStr(workerId))
|
||||
func (w *workerEngineRepository) DeleteWorker(ctx context.Context, tenantId, workerId string) error {
|
||||
_, err := w.queries.DeleteWorker(ctx, w.pool, sqlchelpers.UUIDFromStr(workerId))
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
@@ -298,7 +298,7 @@ func NewWorkflowEngineRepository(pool *pgxpool.Pool, v validator.Validator, l *z
|
||||
}
|
||||
}
|
||||
|
||||
func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
func (r *workflowEngineRepository) CreateNewWorkflow(ctx context.Context, tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
if err := r.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -313,7 +313,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
|
||||
}
|
||||
|
||||
// preflight check to ensure the workflow doesn't already exist
|
||||
workflow, err := r.queries.GetWorkflowByName(context.Background(), r.pool, dbsqlc.GetWorkflowByNameParams{
|
||||
workflow, err := r.queries.GetWorkflowByName(ctx, r.pool, dbsqlc.GetWorkflowByNameParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Name: opts.Name,
|
||||
})
|
||||
@@ -329,20 +329,20 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
|
||||
)
|
||||
}
|
||||
|
||||
tx, err := r.pool.Begin(context.Background())
|
||||
tx, err := r.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), r.l, tx.Rollback)
|
||||
defer deferRollback(ctx, r.l, tx.Rollback)
|
||||
|
||||
workflowId := sqlchelpers.UUIDFromStr(uuid.New().String())
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
// create a workflow
|
||||
_, err = r.queries.CreateWorkflow(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateWorkflowParams{
|
||||
ID: workflowId,
|
||||
@@ -366,7 +366,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
|
||||
}
|
||||
|
||||
err = r.queries.UpsertWorkflowTag(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.UpsertWorkflowTagParams{
|
||||
Tenantid: pgTenantId,
|
||||
@@ -381,13 +381,13 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
|
||||
}
|
||||
}
|
||||
|
||||
workflowVersionId, err := r.createWorkflowVersionTxs(context.Background(), tx, pgTenantId, workflowId, opts)
|
||||
workflowVersionId, err := r.createWorkflowVersionTxs(ctx, tx, pgTenantId, workflowId, opts)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(context.Background(), tx, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(ctx, tx, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
Tenantid: pgTenantId,
|
||||
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(workflowVersionId)},
|
||||
})
|
||||
@@ -400,7 +400,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
|
||||
return nil, fmt.Errorf("expected 1 workflow version when creating new, got %d", len(workflowVersion))
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -409,7 +409,7 @@ func (r *workflowEngineRepository) CreateNewWorkflow(tenantId string, opts *repo
|
||||
return workflowVersion[0], nil
|
||||
}
|
||||
|
||||
func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
func (r *workflowEngineRepository) CreateWorkflowVersion(ctx context.Context, tenantId string, opts *repository.CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
if err := r.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@@ -424,7 +424,7 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
|
||||
}
|
||||
|
||||
// preflight check to ensure the workflow already exists
|
||||
workflow, err := r.queries.GetWorkflowByName(context.Background(), r.pool, dbsqlc.GetWorkflowByNameParams{
|
||||
workflow, err := r.queries.GetWorkflowByName(ctx, r.pool, dbsqlc.GetWorkflowByNameParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Name: opts.Name,
|
||||
})
|
||||
@@ -440,23 +440,23 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
|
||||
)
|
||||
}
|
||||
|
||||
tx, err := r.pool.Begin(context.Background())
|
||||
tx, err := r.pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), r.l, tx.Rollback)
|
||||
defer deferRollback(ctx, r.l, tx.Rollback)
|
||||
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
workflowVersionId, err := r.createWorkflowVersionTxs(context.Background(), tx, pgTenantId, workflow.ID, opts)
|
||||
workflowVersionId, err := r.createWorkflowVersionTxs(ctx, tx, pgTenantId, workflow.ID, opts)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(context.Background(), tx, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
workflowVersion, err := r.queries.GetWorkflowVersionForEngine(ctx, tx, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
Tenantid: pgTenantId,
|
||||
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(workflowVersionId)},
|
||||
})
|
||||
@@ -469,7 +469,7 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
|
||||
return nil, fmt.Errorf("expected 1 workflow version when creating version, got %d", len(workflowVersion))
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -479,6 +479,7 @@ func (r *workflowEngineRepository) CreateWorkflowVersion(tenantId string, opts *
|
||||
}
|
||||
|
||||
func (r *workflowEngineRepository) CreateSchedules(
|
||||
ctx context.Context,
|
||||
tenantId, workflowVersionId string,
|
||||
opts *repository.CreateWorkflowSchedulesOpts,
|
||||
) ([]*dbsqlc.WorkflowTriggerScheduledRef, error) {
|
||||
@@ -496,17 +497,17 @@ func (r *workflowEngineRepository) CreateSchedules(
|
||||
createParams.Triggertimes[i] = sqlchelpers.TimestampFromTime(scheduledTrigger)
|
||||
}
|
||||
|
||||
return r.queries.CreateSchedules(context.Background(), r.pool, createParams)
|
||||
return r.queries.CreateSchedules(ctx, r.pool, createParams)
|
||||
}
|
||||
|
||||
func (r *workflowEngineRepository) GetLatestWorkflowVersion(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
versionId, err := r.queries.GetWorkflowLatestVersion(context.Background(), r.pool, sqlchelpers.UUIDFromStr(workflowId))
|
||||
func (r *workflowEngineRepository) GetLatestWorkflowVersion(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
versionId, err := r.queries.GetWorkflowLatestVersion(ctx, r.pool, sqlchelpers.UUIDFromStr(workflowId))
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to fetch latest version: %w", err)
|
||||
}
|
||||
|
||||
versions, err := r.queries.GetWorkflowVersionForEngine(context.Background(), r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
versions, err := r.queries.GetWorkflowVersionForEngine(ctx, r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Ids: []pgtype.UUID{versionId},
|
||||
})
|
||||
@@ -522,15 +523,15 @@ func (r *workflowEngineRepository) GetLatestWorkflowVersion(tenantId, workflowId
|
||||
return versions[0], nil
|
||||
}
|
||||
|
||||
func (r *workflowEngineRepository) GetWorkflowByName(tenantId, workflowName string) (*dbsqlc.Workflow, error) {
|
||||
return r.queries.GetWorkflowByName(context.Background(), r.pool, dbsqlc.GetWorkflowByNameParams{
|
||||
func (r *workflowEngineRepository) GetWorkflowByName(ctx context.Context, tenantId, workflowName string) (*dbsqlc.Workflow, error) {
|
||||
return r.queries.GetWorkflowByName(ctx, r.pool, dbsqlc.GetWorkflowByNameParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Name: workflowName,
|
||||
})
|
||||
}
|
||||
|
||||
func (r *workflowEngineRepository) GetWorkflowVersionById(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
versions, err := r.queries.GetWorkflowVersionForEngine(context.Background(), r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
func (r *workflowEngineRepository) GetWorkflowVersionById(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error) {
|
||||
versions, err := r.queries.GetWorkflowVersionForEngine(ctx, r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Ids: []pgtype.UUID{sqlchelpers.UUIDFromStr(workflowId)},
|
||||
})
|
||||
@@ -571,7 +572,7 @@ func (r *workflowEngineRepository) ListWorkflowsForEvent(ctx context.Context, te
|
||||
ctx, span3 := telemetry.NewSpan(ctx, "db-get-workflow-versions-for-engine") // nolint: ineffassign
|
||||
defer span3.End()
|
||||
|
||||
workflows, err := r.queries.GetWorkflowVersionForEngine(context.Background(), r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
workflows, err := r.queries.GetWorkflowVersionForEngine(ctx, r.pool, dbsqlc.GetWorkflowVersionForEngineParams{
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Ids: workflowVersionIds,
|
||||
})
|
||||
@@ -610,7 +611,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
}
|
||||
|
||||
sqlcWorkflowVersion, err := r.queries.CreateWorkflowVersion(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
createParams,
|
||||
)
|
||||
@@ -623,7 +624,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
if opts.Concurrency != nil {
|
||||
// upsert the action
|
||||
action, err := r.queries.UpsertAction(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.UpsertActionParams{
|
||||
Action: opts.Concurrency.Action,
|
||||
@@ -659,7 +660,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
}
|
||||
|
||||
_, err = r.queries.CreateWorkflowConcurrency(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
params,
|
||||
)
|
||||
@@ -682,7 +683,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
}
|
||||
|
||||
sqlcJob, err := r.queries.CreateJob(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateJobParams{
|
||||
ID: sqlchelpers.UUIDFromStr(jobId),
|
||||
@@ -724,7 +725,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
|
||||
// upsert the action
|
||||
_, err := r.queries.UpsertAction(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.UpsertActionParams{
|
||||
Action: stepOpts.Action,
|
||||
@@ -752,7 +753,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
}
|
||||
|
||||
_, err = r.queries.CreateStep(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
createStepParams,
|
||||
)
|
||||
@@ -763,7 +764,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
|
||||
if len(stepOpts.Parents) > 0 {
|
||||
err := r.queries.AddStepParents(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.AddStepParentsParams{
|
||||
ID: sqlchelpers.UUIDFromStr(stepId),
|
||||
@@ -780,7 +781,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
if len(stepOpts.RateLimits) > 0 {
|
||||
for _, rateLimit := range stepOpts.RateLimits {
|
||||
_, err := r.queries.CreateStepRateLimit(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateStepRateLimitParams{
|
||||
Stepid: sqlchelpers.UUIDFromStr(stepId),
|
||||
@@ -802,7 +803,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
workflowTriggersId := uuid.New().String()
|
||||
|
||||
sqlcWorkflowTriggers, err := r.queries.CreateWorkflowTriggers(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateWorkflowTriggersParams{
|
||||
ID: sqlchelpers.UUIDFromStr(workflowTriggersId),
|
||||
@@ -817,7 +818,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
|
||||
for _, eventTrigger := range opts.EventTriggers {
|
||||
_, err := r.queries.CreateWorkflowTriggerEventRef(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateWorkflowTriggerEventRefParams{
|
||||
Workflowtriggersid: sqlcWorkflowTriggers.ID,
|
||||
@@ -832,7 +833,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
|
||||
for _, cronTrigger := range opts.CronTriggers {
|
||||
_, err := r.queries.CreateWorkflowTriggerCronRef(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateWorkflowTriggerCronRefParams{
|
||||
Workflowtriggersid: sqlcWorkflowTriggers.ID,
|
||||
@@ -848,7 +849,7 @@ func (r *workflowEngineRepository) createWorkflowVersionTxs(ctx context.Context,
|
||||
|
||||
for _, scheduledTrigger := range opts.ScheduledTriggers {
|
||||
_, err := r.queries.CreateWorkflowTriggerScheduledRef(
|
||||
context.Background(),
|
||||
ctx,
|
||||
tx,
|
||||
dbsqlc.CreateWorkflowTriggerScheduledRefParams{
|
||||
Workflowversionid: sqlcWorkflowVersion.ID,
|
||||
|
||||
@@ -138,8 +138,8 @@ func NewWorkflowRunEngineRepository(pool *pgxpool.Pool, v validator.Validator, l
|
||||
}
|
||||
}
|
||||
|
||||
func (w *workflowRunEngineRepository) GetWorkflowRunById(tenantId, id string) (*dbsqlc.GetWorkflowRunRow, error) {
|
||||
runs, err := w.queries.GetWorkflowRun(context.Background(), w.pool, dbsqlc.GetWorkflowRunParams{
|
||||
func (w *workflowRunEngineRepository) GetWorkflowRunById(ctx context.Context, tenantId, id string) (*dbsqlc.GetWorkflowRunRow, error) {
|
||||
runs, err := w.queries.GetWorkflowRun(ctx, w.pool, dbsqlc.GetWorkflowRunParams{
|
||||
Ids: []pgtype.UUID{
|
||||
sqlchelpers.UUIDFromStr(id),
|
||||
},
|
||||
@@ -157,15 +157,15 @@ func (w *workflowRunEngineRepository) GetWorkflowRunById(tenantId, id string) (*
|
||||
return runs[0], nil
|
||||
}
|
||||
|
||||
func (w *workflowRunEngineRepository) ListWorkflowRuns(tenantId string, opts *repository.ListWorkflowRunsOpts) (*repository.ListWorkflowRunsResult, error) {
|
||||
func (w *workflowRunEngineRepository) ListWorkflowRuns(ctx context.Context, tenantId string, opts *repository.ListWorkflowRunsOpts) (*repository.ListWorkflowRunsResult, error) {
|
||||
if err := w.v.Validate(opts); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return listWorkflowRuns(context.Background(), w.pool, w.queries, w.l, tenantId, opts)
|
||||
return listWorkflowRuns(ctx, w.pool, w.queries, w.l, tenantId, opts)
|
||||
}
|
||||
|
||||
func (w *workflowRunEngineRepository) GetChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error) {
|
||||
func (w *workflowRunEngineRepository) GetChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error) {
|
||||
params := dbsqlc.GetChildWorkflowRunParams{
|
||||
Parentid: sqlchelpers.UUIDFromStr(parentId),
|
||||
Parentsteprunid: sqlchelpers.UUIDFromStr(parentStepRunId),
|
||||
@@ -179,10 +179,10 @@ func (w *workflowRunEngineRepository) GetChildWorkflowRun(parentId, parentStepRu
|
||||
params.ChildKey = sqlchelpers.TextFromStr(*childkey)
|
||||
}
|
||||
|
||||
return w.queries.GetChildWorkflowRun(context.Background(), w.pool, params)
|
||||
return w.queries.GetChildWorkflowRun(ctx, w.pool, params)
|
||||
}
|
||||
|
||||
func (w *workflowRunEngineRepository) GetScheduledChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error) {
|
||||
func (w *workflowRunEngineRepository) GetScheduledChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error) {
|
||||
params := dbsqlc.GetScheduledChildWorkflowRunParams{
|
||||
Parentid: sqlchelpers.UUIDFromStr(parentId),
|
||||
Parentsteprunid: sqlchelpers.UUIDFromStr(parentStepRunId),
|
||||
@@ -196,11 +196,11 @@ func (w *workflowRunEngineRepository) GetScheduledChildWorkflowRun(parentId, par
|
||||
params.ChildKey = sqlchelpers.TextFromStr(*childkey)
|
||||
}
|
||||
|
||||
return w.queries.GetScheduledChildWorkflowRun(context.Background(), w.pool, params)
|
||||
return w.queries.GetScheduledChildWorkflowRun(ctx, w.pool, params)
|
||||
}
|
||||
|
||||
func (w *workflowRunEngineRepository) PopWorkflowRunsRoundRobin(tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error) {
|
||||
return w.queries.PopWorkflowRunsRoundRobin(context.Background(), w.pool, dbsqlc.PopWorkflowRunsRoundRobinParams{
|
||||
func (w *workflowRunEngineRepository) PopWorkflowRunsRoundRobin(ctx context.Context, tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error) {
|
||||
return w.queries.PopWorkflowRunsRoundRobin(ctx, w.pool, dbsqlc.PopWorkflowRunsRoundRobinParams{
|
||||
Maxruns: int32(maxRuns),
|
||||
Tenantid: sqlchelpers.UUIDFromStr(tenantId),
|
||||
Workflowid: sqlchelpers.UUIDFromStr(workflowId),
|
||||
@@ -254,6 +254,17 @@ func listWorkflowRuns(ctx context.Context, pool *pgxpool.Pool, queries *dbsqlc.Q
|
||||
countParams.WorkflowVersionId = pgWorkflowVersionId
|
||||
}
|
||||
|
||||
if opts.Ids != nil && len(opts.Ids) > 0 {
|
||||
pgIds := make([]pgtype.UUID, len(opts.Ids))
|
||||
|
||||
for i, id := range opts.Ids {
|
||||
pgIds[i] = sqlchelpers.UUIDFromStr(id)
|
||||
}
|
||||
|
||||
queryParams.Ids = pgIds
|
||||
countParams.Ids = pgIds
|
||||
}
|
||||
|
||||
if opts.ParentId != nil {
|
||||
pgParentId := sqlchelpers.UUIDFromStr(*opts.ParentId)
|
||||
|
||||
@@ -305,27 +316,27 @@ func listWorkflowRuns(ctx context.Context, pool *pgxpool.Pool, queries *dbsqlc.Q
|
||||
|
||||
queryParams.Orderby = orderByField + " " + orderByDirection
|
||||
|
||||
tx, err := pool.Begin(context.Background())
|
||||
tx, err := pool.Begin(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), l, tx.Rollback)
|
||||
defer deferRollback(ctx, l, tx.Rollback)
|
||||
|
||||
workflowRuns, err := queries.ListWorkflowRuns(context.Background(), tx, queryParams)
|
||||
workflowRuns, err := queries.ListWorkflowRuns(ctx, tx, queryParams)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
count, err := queries.CountWorkflowRuns(context.Background(), tx, countParams)
|
||||
count, err := queries.CountWorkflowRuns(ctx, tx, countParams)
|
||||
|
||||
if err != nil && !errors.Is(err, pgx.ErrNoRows) {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = tx.Commit(context.Background())
|
||||
err = tx.Commit(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -354,7 +365,7 @@ func createNewWorkflowRun(ctx context.Context, pool *pgxpool.Pool, queries *dbsq
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer deferRollback(context.Background(), l, tx.Rollback)
|
||||
defer deferRollback(ctx, l, tx.Rollback)
|
||||
|
||||
pgTenantId := sqlchelpers.UUIDFromStr(tenantId)
|
||||
|
||||
|
||||
@@ -1,6 +1,10 @@
|
||||
package repository
|
||||
|
||||
import "github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
)
|
||||
|
||||
type UpsertRateLimitOpts struct {
|
||||
// The rate limit max value
|
||||
@@ -12,5 +16,5 @@ type UpsertRateLimitOpts struct {
|
||||
|
||||
type RateLimitEngineRepository interface {
|
||||
// CreateRateLimit creates a new rate limit record
|
||||
UpsertRateLimit(tenantId string, key string, opts *UpsertRateLimitOpts) (*dbsqlc.RateLimit, error)
|
||||
UpsertRateLimit(ctx context.Context, tenantId string, key string, opts *UpsertRateLimitOpts) (*dbsqlc.RateLimit, error)
|
||||
}
|
||||
|
||||
@@ -9,20 +9,10 @@ import (
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
)
|
||||
|
||||
type ListAllStepRunsOpts struct {
|
||||
TickerId *string
|
||||
|
||||
NoTickerId *bool
|
||||
|
||||
Status *db.StepRunStatus
|
||||
}
|
||||
|
||||
type ListStepRunsOpts struct {
|
||||
JobRunId *string
|
||||
WorkflowRunIds []string `validate:"dive,uuid"`
|
||||
|
||||
WorkflowRunId *string
|
||||
|
||||
Status *db.StepRunStatus
|
||||
Status *dbsqlc.StepRunStatus
|
||||
}
|
||||
|
||||
type UpdateStepRunOpts struct {
|
||||
@@ -81,35 +71,32 @@ type StepRunAPIRepository interface {
|
||||
}
|
||||
|
||||
type StepRunEngineRepository interface {
|
||||
// ListRunningStepRunsForTicker returns a list of step runs which are currently running for a ticker.
|
||||
ListRunningStepRunsForTicker(tickerId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
// ListRunningStepRunsForWorkflowRun returns a list of step runs which are currently running for a workflow run.
|
||||
ListRunningStepRunsForWorkflowRun(tenantId, workflowRunId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
// ListStepRunsForWorkflowRun returns a list of step runs for a workflow run.
|
||||
ListStepRuns(ctx context.Context, tenantId string, opts *ListStepRunsOpts) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
// ListStepRunsToRequeue returns a list of step runs which are in a requeueable state.
|
||||
ListStepRunsToRequeue(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
ListStepRunsToRequeue(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
// ListStepRunsToReassign returns a list of step runs which are in a reassignable state.
|
||||
ListStepRunsToReassign(tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
ListStepRunsToReassign(ctx context.Context, tenantId string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
UpdateStepRun(ctx context.Context, tenantId, stepRunId string, opts *UpdateStepRunOpts) (*dbsqlc.GetStepRunForEngineRow, *StepRunUpdateInfo, error)
|
||||
|
||||
// UpdateStepRunOverridesData updates the overrides data field in the input for a step run. This returns the input
|
||||
// bytes.
|
||||
UpdateStepRunOverridesData(tenantId, stepRunId string, opts *UpdateStepRunOverridesDataOpts) ([]byte, error)
|
||||
UpdateStepRunOverridesData(ctx context.Context, tenantId, stepRunId string, opts *UpdateStepRunOverridesDataOpts) ([]byte, error)
|
||||
|
||||
UpdateStepRunInputSchema(tenantId, stepRunId string, schema []byte) ([]byte, error)
|
||||
UpdateStepRunInputSchema(ctx context.Context, tenantId, stepRunId string, schema []byte) ([]byte, error)
|
||||
|
||||
AssignStepRunToWorker(ctx context.Context, stepRun *dbsqlc.GetStepRunForEngineRow) (workerId string, dispatcherId string, err error)
|
||||
|
||||
GetStepRunForEngine(tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error)
|
||||
GetStepRunForEngine(ctx context.Context, tenantId, stepRunId string) (*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
// QueueStepRun is like UpdateStepRun, except that it will only update the step run if it is in
|
||||
// a pending state.
|
||||
QueueStepRun(ctx context.Context, tenantId, stepRunId string, opts *UpdateStepRunOpts) (*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
ListStartableStepRuns(tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
ListStartableStepRuns(ctx context.Context, tenantId, jobRunId string, parentStepRunId *string) ([]*dbsqlc.GetStepRunForEngineRow, error)
|
||||
|
||||
ArchiveStepRunResult(tenantId, stepRunId string) error
|
||||
ArchiveStepRunResult(ctx context.Context, tenantId, stepRunId string) error
|
||||
}
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
@@ -22,11 +23,11 @@ type CreateStreamEventOpts struct {
|
||||
|
||||
type StreamEventsEngineRepository interface {
|
||||
// PutStreamEvent creates a new StreamEvent line.
|
||||
PutStreamEvent(tenantId string, opts *CreateStreamEventOpts) (*dbsqlc.StreamEvent, error)
|
||||
PutStreamEvent(ctx context.Context, tenantId string, opts *CreateStreamEventOpts) (*dbsqlc.StreamEvent, error)
|
||||
|
||||
// GetStreamEvent returns a StreamEvent line by id.
|
||||
GetStreamEvent(tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error)
|
||||
GetStreamEvent(ctx context.Context, tenantId string, streamEventId int64) (*dbsqlc.StreamEvent, error)
|
||||
|
||||
// CleanupStreamEvents deletes all stale StreamEvents.
|
||||
CleanupStreamEvents() error
|
||||
CleanupStreamEvents(ctx context.Context) error
|
||||
}
|
||||
|
||||
@@ -1,6 +1,8 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
)
|
||||
@@ -59,8 +61,8 @@ type TenantAPIRepository interface {
|
||||
|
||||
type TenantEngineRepository interface {
|
||||
// ListTenants lists all tenants in the instance
|
||||
ListTenants() ([]*dbsqlc.Tenant, error)
|
||||
ListTenants(ctx context.Context) ([]*dbsqlc.Tenant, error)
|
||||
|
||||
// GetTenantByID returns the tenant with the given id
|
||||
GetTenantByID(tenantId string) (*dbsqlc.Tenant, error)
|
||||
GetTenantByID(ctx context.Context, tenantId string) (*dbsqlc.Tenant, error)
|
||||
}
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/dbsqlc"
|
||||
@@ -23,27 +24,27 @@ type ListTickerOpts struct {
|
||||
|
||||
type TickerEngineRepository interface {
|
||||
// CreateNewTicker creates a new ticker.
|
||||
CreateNewTicker(opts *CreateTickerOpts) (*dbsqlc.Ticker, error)
|
||||
CreateNewTicker(ctx context.Context, opts *CreateTickerOpts) (*dbsqlc.Ticker, error)
|
||||
|
||||
// UpdateTicker updates a ticker.
|
||||
UpdateTicker(tickerId string, opts *UpdateTickerOpts) (*dbsqlc.Ticker, error)
|
||||
UpdateTicker(ctx context.Context, tickerId string, opts *UpdateTickerOpts) (*dbsqlc.Ticker, error)
|
||||
|
||||
// ListTickers lists tickers.
|
||||
ListTickers(opts *ListTickerOpts) ([]*dbsqlc.Ticker, error)
|
||||
ListTickers(ctx context.Context, opts *ListTickerOpts) ([]*dbsqlc.Ticker, error)
|
||||
|
||||
// Delete deletes a ticker.
|
||||
Delete(tickerId string) error
|
||||
Delete(ctx context.Context, tickerId string) error
|
||||
|
||||
// PollStepRuns looks for step runs who are close to past their timeoutAt value and are in a running state
|
||||
PollStepRuns(tickerId string) ([]*dbsqlc.StepRun, error)
|
||||
PollStepRuns(ctx context.Context, tickerId string) ([]*dbsqlc.StepRun, error)
|
||||
|
||||
// PollJobRuns looks for get group key runs who are close to past their timeoutAt value and are in a running state
|
||||
PollGetGroupKeyRuns(tickerId string) ([]*dbsqlc.GetGroupKeyRun, error)
|
||||
PollGetGroupKeyRuns(ctx context.Context, tickerId string) ([]*dbsqlc.GetGroupKeyRun, error)
|
||||
|
||||
// PollCronSchedules returns all cron schedules which should be managed by the ticker
|
||||
PollCronSchedules(tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error)
|
||||
PollCronSchedules(ctx context.Context, tickerId string) ([]*dbsqlc.PollCronSchedulesRow, error)
|
||||
|
||||
PollScheduledWorkflows(tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error)
|
||||
PollScheduledWorkflows(ctx context.Context, tickerId string) ([]*dbsqlc.PollScheduledWorkflowsRow, error)
|
||||
|
||||
// // AddJobRun assigns a job run to a ticker.
|
||||
// AddJobRun(tickerId string, jobRun *db.JobRunModel) (*db.TickerModel, error)
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package repository
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/repository/prisma/db"
|
||||
@@ -64,13 +65,13 @@ type WorkerAPIRepository interface {
|
||||
|
||||
type WorkerEngineRepository interface {
|
||||
// CreateNewWorker creates a new worker for a given tenant.
|
||||
CreateNewWorker(tenantId string, opts *CreateWorkerOpts) (*dbsqlc.Worker, error)
|
||||
CreateNewWorker(ctx context.Context, tenantId string, opts *CreateWorkerOpts) (*dbsqlc.Worker, error)
|
||||
|
||||
// UpdateWorker updates a worker for a given tenant.
|
||||
UpdateWorker(tenantId, workerId string, opts *UpdateWorkerOpts) (*dbsqlc.Worker, error)
|
||||
UpdateWorker(ctx context.Context, tenantId, workerId string, opts *UpdateWorkerOpts) (*dbsqlc.Worker, error)
|
||||
|
||||
// DeleteWorker removes the worker from the database
|
||||
DeleteWorker(tenantId, workerId string) error
|
||||
DeleteWorker(ctx context.Context, tenantId, workerId string) error
|
||||
|
||||
GetWorkerForEngine(tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error)
|
||||
GetWorkerForEngine(ctx context.Context, tenantId, workerId string) (*dbsqlc.GetWorkerForEngineRow, error)
|
||||
}
|
||||
|
||||
@@ -215,22 +215,22 @@ type WorkflowAPIRepository interface {
|
||||
type WorkflowEngineRepository interface {
|
||||
// CreateNewWorkflow creates a new workflow for a given tenant. It will create the parent
|
||||
// workflow based on the version's name.
|
||||
CreateNewWorkflow(tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
CreateNewWorkflow(ctx context.Context, tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
|
||||
// CreateWorkflowVersion creates a new workflow version for a given tenant. This will fail if there is
|
||||
// not a parent workflow with the same name already in the database.
|
||||
CreateWorkflowVersion(tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
CreateWorkflowVersion(ctx context.Context, tenantId string, opts *CreateWorkflowVersionOpts) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
|
||||
// CreateSchedules creates schedules for a given workflow version.
|
||||
CreateSchedules(tenantId, workflowVersionId string, opts *CreateWorkflowSchedulesOpts) ([]*dbsqlc.WorkflowTriggerScheduledRef, error)
|
||||
CreateSchedules(ctx context.Context, tenantId, workflowVersionId string, opts *CreateWorkflowSchedulesOpts) ([]*dbsqlc.WorkflowTriggerScheduledRef, error)
|
||||
|
||||
// GetScheduledById returns a scheduled workflow by its id.
|
||||
// GetScheduledById(tenantId, scheduleTriggerId string) (*db.WorkflowTriggerScheduledRefModel, error)
|
||||
|
||||
GetLatestWorkflowVersion(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
GetLatestWorkflowVersion(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
|
||||
// GetWorkflowByName returns a workflow by its name. It will return db.ErrNotFound if the workflow does not exist.
|
||||
GetWorkflowByName(tenantId, workflowName string) (*dbsqlc.Workflow, error)
|
||||
GetWorkflowByName(ctx context.Context, tenantId, workflowName string) (*dbsqlc.Workflow, error)
|
||||
|
||||
// ListWorkflowsForEvent returns the latest workflow versions for a given tenant that are triggered by the
|
||||
// given event.
|
||||
@@ -238,5 +238,5 @@ type WorkflowEngineRepository interface {
|
||||
|
||||
// GetWorkflowVersionById returns a workflow version by its id. It will return db.ErrNotFound if the workflow
|
||||
// version does not exist.
|
||||
GetWorkflowVersionById(tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
GetWorkflowVersionById(ctx context.Context, tenantId, workflowId string) (*dbsqlc.GetWorkflowVersionForEngineRow, error)
|
||||
}
|
||||
|
||||
@@ -202,6 +202,9 @@ type ListWorkflowRunsOpts struct {
|
||||
// (optional) the workflow version id
|
||||
WorkflowVersionId *string `validate:"omitempty,uuid"`
|
||||
|
||||
// (optional) a list of workflow run ids to filter by
|
||||
Ids []string `validate:"omitempty,dive,uuid"`
|
||||
|
||||
// (optional) the parent workflow run id
|
||||
ParentId *string `validate:"omitempty,uuid"`
|
||||
|
||||
@@ -284,17 +287,17 @@ type WorkflowRunAPIRepository interface {
|
||||
|
||||
type WorkflowRunEngineRepository interface {
|
||||
// ListWorkflowRuns returns workflow runs for a given workflow version id.
|
||||
ListWorkflowRuns(tenantId string, opts *ListWorkflowRunsOpts) (*ListWorkflowRunsResult, error)
|
||||
ListWorkflowRuns(ctx context.Context, tenantId string, opts *ListWorkflowRunsOpts) (*ListWorkflowRunsResult, error)
|
||||
|
||||
GetChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error)
|
||||
GetChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowRun, error)
|
||||
|
||||
GetScheduledChildWorkflowRun(parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error)
|
||||
GetScheduledChildWorkflowRun(ctx context.Context, parentId, parentStepRunId string, childIndex int, childkey *string) (*dbsqlc.WorkflowTriggerScheduledRef, error)
|
||||
|
||||
PopWorkflowRunsRoundRobin(tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error)
|
||||
PopWorkflowRunsRoundRobin(ctx context.Context, tenantId, workflowId string, maxRuns int) ([]*dbsqlc.WorkflowRun, error)
|
||||
|
||||
// CreateNewWorkflowRun creates a new workflow run for a workflow version.
|
||||
CreateNewWorkflowRun(ctx context.Context, tenantId string, opts *CreateWorkflowRunOpts) (string, error)
|
||||
|
||||
// GetWorkflowRunById returns a workflow run by id.
|
||||
GetWorkflowRunById(tenantId, runId string) (*dbsqlc.GetWorkflowRunRow, error)
|
||||
GetWorkflowRunById(ctx context.Context, tenantId, runId string) (*dbsqlc.GetWorkflowRunRow, error)
|
||||
}
|
||||
|
||||
@@ -43,6 +43,7 @@ func (a *AdminServiceImpl) TriggerWorkflow(ctx context.Context, req *contracts.T
|
||||
}
|
||||
|
||||
workflowRun, err := a.repo.WorkflowRun().GetChildWorkflowRun(
|
||||
ctx,
|
||||
*req.ParentId,
|
||||
*req.ParentStepRunId,
|
||||
int(*req.ChildIndex),
|
||||
@@ -63,6 +64,7 @@ func (a *AdminServiceImpl) TriggerWorkflow(ctx context.Context, req *contracts.T
|
||||
}
|
||||
|
||||
workflow, err := a.repo.Workflow().GetWorkflowByName(
|
||||
ctx,
|
||||
tenantId,
|
||||
req.Name,
|
||||
)
|
||||
@@ -79,6 +81,7 @@ func (a *AdminServiceImpl) TriggerWorkflow(ctx context.Context, req *contracts.T
|
||||
}
|
||||
|
||||
workflowVersion, err := a.repo.Workflow().GetLatestWorkflowVersion(
|
||||
ctx,
|
||||
tenantId,
|
||||
sqlchelpers.UUIDToStr(workflow.ID),
|
||||
)
|
||||
@@ -144,6 +147,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
|
||||
var oldWorkflowVersion *dbsqlc.GetWorkflowVersionForEngineRow
|
||||
|
||||
currWorkflow, err := a.repo.Workflow().GetWorkflowByName(
|
||||
ctx,
|
||||
tenantId,
|
||||
req.Opts.Name,
|
||||
)
|
||||
@@ -155,6 +159,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
|
||||
|
||||
// workflow does not exist, create it
|
||||
workflowVersion, err = a.repo.Workflow().CreateNewWorkflow(
|
||||
ctx,
|
||||
tenantId,
|
||||
createOpts,
|
||||
)
|
||||
@@ -164,6 +169,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
|
||||
}
|
||||
} else {
|
||||
oldWorkflowVersion, err = a.repo.Workflow().GetLatestWorkflowVersion(
|
||||
ctx,
|
||||
tenantId,
|
||||
sqlchelpers.UUIDToStr(currWorkflow.ID),
|
||||
)
|
||||
@@ -181,6 +187,7 @@ func (a *AdminServiceImpl) PutWorkflow(ctx context.Context, req *contracts.PutWo
|
||||
|
||||
if oldWorkflowVersion.WorkflowVersion.Checksum != newCS {
|
||||
workflowVersion, err = a.repo.Workflow().CreateWorkflowVersion(
|
||||
ctx,
|
||||
tenantId,
|
||||
createOpts,
|
||||
)
|
||||
@@ -203,6 +210,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
|
||||
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
|
||||
|
||||
workflow, err := a.repo.Workflow().GetWorkflowByName(
|
||||
ctx,
|
||||
tenantId,
|
||||
req.Name,
|
||||
)
|
||||
@@ -221,6 +229,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
|
||||
workflowId := sqlchelpers.UUIDToStr(workflow.ID)
|
||||
|
||||
currWorkflow, err := a.repo.Workflow().GetLatestWorkflowVersion(
|
||||
ctx,
|
||||
tenantId,
|
||||
workflowId,
|
||||
)
|
||||
@@ -251,6 +260,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
|
||||
}
|
||||
|
||||
existing, err := a.repo.WorkflowRun().GetScheduledChildWorkflowRun(
|
||||
ctx,
|
||||
*req.ParentId,
|
||||
*req.ParentStepRunId,
|
||||
int(*req.ChildIndex),
|
||||
@@ -277,6 +287,7 @@ func (a *AdminServiceImpl) ScheduleWorkflow(ctx context.Context, req *contracts.
|
||||
workflowVersionId := sqlchelpers.UUIDToStr(currWorkflow.WorkflowVersion.ID)
|
||||
|
||||
_, err = a.repo.Workflow().CreateSchedules(
|
||||
ctx,
|
||||
tenantId,
|
||||
workflowVersionId,
|
||||
&repository.CreateWorkflowSchedulesOpts{
|
||||
@@ -313,7 +324,7 @@ func (a *AdminServiceImpl) PutRateLimit(ctx context.Context, req *contracts.PutR
|
||||
Duration: &duration,
|
||||
}
|
||||
|
||||
_, err := a.repo.RateLimit().UpsertRateLimit(tenantId, req.Key, createOpts)
|
||||
_, err := a.repo.RateLimit().UpsertRateLimit(ctx, tenantId, req.Key, createOpts)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
||||
@@ -239,14 +239,14 @@ func (ec *JobsControllerImpl) handleJobRunQueued(ctx context.Context, task *msgq
|
||||
return fmt.Errorf("could not decode job task metadata: %w", err)
|
||||
}
|
||||
|
||||
err = ec.repo.JobRun().SetJobRunStatusRunning(metadata.TenantId, payload.JobRunId)
|
||||
err = ec.repo.JobRun().SetJobRunStatusRunning(ctx, metadata.TenantId, payload.JobRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not set job run status to running: %w", err)
|
||||
}
|
||||
|
||||
// list the step runs which are startable
|
||||
startableStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(metadata.TenantId, payload.JobRunId, nil)
|
||||
startableStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(ctx, metadata.TenantId, payload.JobRunId, nil)
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list startable step runs: %w", err)
|
||||
}
|
||||
@@ -294,7 +294,7 @@ func (ec *JobsControllerImpl) handleStepRunRetry(ctx context.Context, task *msgq
|
||||
return fmt.Errorf("could not decode job task metadata: %w", err)
|
||||
}
|
||||
|
||||
err = ec.repo.StepRun().ArchiveStepRunResult(metadata.TenantId, payload.StepRunId)
|
||||
err = ec.repo.StepRun().ArchiveStepRunResult(ctx, metadata.TenantId, payload.StepRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not archive step run result: %w", err)
|
||||
@@ -302,7 +302,7 @@ func (ec *JobsControllerImpl) handleStepRunRetry(ctx context.Context, task *msgq
|
||||
|
||||
ec.l.Error().Err(fmt.Errorf("starting step run retry"))
|
||||
|
||||
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
|
||||
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get step run: %w", err)
|
||||
@@ -405,10 +405,13 @@ func (ec *JobsControllerImpl) handleStepRunQueued(ctx context.Context, task *msg
|
||||
|
||||
func (jc *JobsControllerImpl) runStepRunRequeue(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
jc.l.Debug().Msgf("jobs controller: checking step run requeue")
|
||||
|
||||
// list all tenants
|
||||
tenants, err := jc.repo.Tenant().ListTenants()
|
||||
tenants, err := jc.repo.Tenant().ListTenants(ctx)
|
||||
|
||||
if err != nil {
|
||||
jc.l.Err(err).Msg("could not list tenants")
|
||||
@@ -438,7 +441,7 @@ func (ec *JobsControllerImpl) runStepRunRequeueTenant(ctx context.Context, tenan
|
||||
ctx, span := telemetry.NewSpan(ctx, "handle-step-run-requeue")
|
||||
defer span.End()
|
||||
|
||||
stepRuns, err := ec.repo.StepRun().ListStepRunsToRequeue(tenantId)
|
||||
stepRuns, err := ec.repo.StepRun().ListStepRunsToRequeue(ctx, tenantId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list step runs to requeue: %w", err)
|
||||
@@ -495,10 +498,13 @@ func (ec *JobsControllerImpl) runStepRunRequeueTenant(ctx context.Context, tenan
|
||||
|
||||
func (jc *JobsControllerImpl) runStepRunReassign(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
jc.l.Debug().Msgf("jobs controller: checking step run reassignment")
|
||||
|
||||
// list all tenants
|
||||
tenants, err := jc.repo.Tenant().ListTenants()
|
||||
tenants, err := jc.repo.Tenant().ListTenants(ctx)
|
||||
|
||||
if err != nil {
|
||||
jc.l.Err(err).Msg("could not list tenants")
|
||||
@@ -529,7 +535,7 @@ func (ec *JobsControllerImpl) runStepRunReassignTenant(ctx context.Context, tena
|
||||
ctx, span := telemetry.NewSpan(ctx, "handle-step-run-reassign")
|
||||
defer span.End()
|
||||
|
||||
stepRuns, err := ec.repo.StepRun().ListStepRunsToReassign(tenantId)
|
||||
stepRuns, err := ec.repo.StepRun().ListStepRunsToReassign(ctx, tenantId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list step runs to reassign: %w", err)
|
||||
@@ -589,7 +595,7 @@ func (ec *JobsControllerImpl) queueStepRun(ctx context.Context, tenantId, stepId
|
||||
defer span.End()
|
||||
|
||||
// add the rendered data to the step run
|
||||
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(tenantId, stepRunId)
|
||||
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(ctx, tenantId, stepRunId)
|
||||
|
||||
errData := map[string]interface{}{
|
||||
"tenant_id": tenantId,
|
||||
@@ -798,7 +804,7 @@ func (ec *JobsControllerImpl) handleStepRunFinished(ctx context.Context, task *m
|
||||
jobRunId := sqlchelpers.UUIDToStr(stepRun.JobRunId)
|
||||
stepRunId := sqlchelpers.UUIDToStr(stepRun.StepRun.ID)
|
||||
|
||||
nextStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(metadata.TenantId, jobRunId, &stepRunId)
|
||||
nextStepRuns, err := ec.repo.StepRun().ListStartableStepRuns(ctx, metadata.TenantId, jobRunId, &stepRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list startable step runs: %w", err)
|
||||
@@ -843,7 +849,7 @@ func (ec *JobsControllerImpl) handleStepRunFailed(ctx context.Context, task *msg
|
||||
return fmt.Errorf("could not parse failed at: %w", err)
|
||||
}
|
||||
|
||||
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
|
||||
stepRun, err := ec.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get step run: %w", err)
|
||||
@@ -957,7 +963,7 @@ func (ec *JobsControllerImpl) cancelStepRun(ctx context.Context, tenantId, stepR
|
||||
|
||||
workerId := sqlchelpers.UUIDToStr(stepRun.StepRun.WorkerId)
|
||||
|
||||
worker, err := ec.repo.Worker().GetWorkerForEngine(tenantId, workerId)
|
||||
worker, err := ec.repo.Worker().GetWorkerForEngine(ctx, tenantId, workerId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get worker: %w", err)
|
||||
|
||||
@@ -224,7 +224,7 @@ func (ec *WorkflowsControllerImpl) handleGroupKeyRunStarted(ctx context.Context,
|
||||
return fmt.Errorf("could not parse started at: %w", err)
|
||||
}
|
||||
|
||||
_, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
_, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
StartedAt: &startedAt,
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusRunning),
|
||||
})
|
||||
@@ -258,7 +258,7 @@ func (wc *WorkflowsControllerImpl) handleGroupKeyRunFinished(ctx context.Context
|
||||
return fmt.Errorf("could not parse started at: %w", err)
|
||||
}
|
||||
|
||||
groupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
groupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
FinishedAt: &finishedAt,
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusSucceeded),
|
||||
Output: &payload.GroupKey,
|
||||
@@ -272,7 +272,7 @@ func (wc *WorkflowsControllerImpl) handleGroupKeyRunFinished(ctx context.Context
|
||||
|
||||
errGroup.Go(func() error {
|
||||
workflowVersionId := sqlchelpers.UUIDToStr(groupKeyRun.WorkflowVersionId)
|
||||
workflowVersion, err := wc.repo.Workflow().GetWorkflowVersionById(metadata.TenantId, workflowVersionId)
|
||||
workflowVersion, err := wc.repo.Workflow().GetWorkflowVersionById(ctx, metadata.TenantId, workflowVersionId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get workflow version: %w", err)
|
||||
@@ -320,7 +320,7 @@ func (wc *WorkflowsControllerImpl) handleGroupKeyRunFailed(ctx context.Context,
|
||||
return fmt.Errorf("could not parse started at: %w", err)
|
||||
}
|
||||
|
||||
_, err = wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
_, err = wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, metadata.TenantId, payload.GetGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
FinishedAt: &failedAt,
|
||||
Error: &payload.Error,
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusFailed),
|
||||
@@ -362,7 +362,7 @@ func (wc *WorkflowsControllerImpl) cancelGetGroupKeyRun(ctx context.Context, ten
|
||||
// cancel current step run
|
||||
now := time.Now().UTC()
|
||||
|
||||
_, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
_, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
CancelledAt: &now,
|
||||
CancelledReason: repository.StringPtr(reason),
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusCancelled),
|
||||
|
||||
@@ -40,7 +40,7 @@ func (wc *WorkflowsControllerImpl) handleWorkflowRunQueued(ctx context.Context,
|
||||
}
|
||||
|
||||
// get the workflow run in the database
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId)
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, metadata.TenantId, payload.WorkflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get job run: %w", err)
|
||||
@@ -63,7 +63,7 @@ func (wc *WorkflowsControllerImpl) handleWorkflowRunQueued(ctx context.Context,
|
||||
return fmt.Errorf("could not get group key run")
|
||||
}
|
||||
|
||||
sqlcGroupKeyRun, err := wc.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(metadata.TenantId, groupKeyRunId)
|
||||
sqlcGroupKeyRun, err := wc.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(ctx, metadata.TenantId, groupKeyRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get group key run for engine: %w", err)
|
||||
@@ -107,7 +107,7 @@ func (wc *WorkflowsControllerImpl) handleWorkflowRunFinished(ctx context.Context
|
||||
}
|
||||
|
||||
// get the workflow run in the database
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId)
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, metadata.TenantId, payload.WorkflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get job run: %w", err)
|
||||
@@ -159,7 +159,7 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
|
||||
getGroupKeyRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.GetGroupKeyRun.ID)
|
||||
workflowRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.WorkflowRunId)
|
||||
|
||||
getGroupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
getGroupKeyRun, err := wc.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusPendingAssignment),
|
||||
})
|
||||
|
||||
@@ -168,6 +168,7 @@ func (wc *WorkflowsControllerImpl) scheduleGetGroupAction(
|
||||
}
|
||||
|
||||
selectedWorkerId, dispatcherId, err := wc.repo.GetGroupKeyRun().AssignGetGroupKeyRunToWorker(
|
||||
ctx,
|
||||
tenantId,
|
||||
getGroupKeyRunId,
|
||||
)
|
||||
@@ -207,7 +208,7 @@ func (wc *WorkflowsControllerImpl) queueWorkflowRunJobs(ctx context.Context, wor
|
||||
tenantId := sqlchelpers.UUIDToStr(workflowRun.WorkflowRun.TenantId)
|
||||
workflowRunId := sqlchelpers.UUIDToStr(workflowRun.WorkflowRun.ID)
|
||||
|
||||
jobRuns, err := wc.repo.JobRun().ListJobRunsForWorkflowRun(tenantId, workflowRunId)
|
||||
jobRuns, err := wc.repo.JobRun().ListJobRunsForWorkflowRun(ctx, tenantId, workflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list job runs: %w", err)
|
||||
@@ -237,7 +238,7 @@ func (wc *WorkflowsControllerImpl) runGetGroupKeyRunRequeue(ctx context.Context)
|
||||
wc.l.Debug().Msgf("workflows controller: checking get group key run requeue")
|
||||
|
||||
// list all tenants
|
||||
tenants, err := wc.repo.Tenant().ListTenants()
|
||||
tenants, err := wc.repo.Tenant().ListTenants(ctx)
|
||||
|
||||
if err != nil {
|
||||
wc.l.Err(err).Msg("could not list tenants")
|
||||
@@ -268,7 +269,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunRequeueTenant(ctx context.Co
|
||||
ctx, span := telemetry.NewSpan(ctx, "handle-get-group-key-run-requeue")
|
||||
defer span.End()
|
||||
|
||||
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToRequeue(tenantId)
|
||||
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToRequeue(ctx, tenantId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list group key runs: %w", err)
|
||||
@@ -300,7 +301,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunRequeueTenant(ctx context.Co
|
||||
isTimedOut := !scheduleTimeoutAt.IsZero() && scheduleTimeoutAt.Before(now)
|
||||
|
||||
if isTimedOut {
|
||||
_, err := ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
_, err := ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
CancelledAt: &now,
|
||||
CancelledReason: repository.StringPtr("SCHEDULING_TIMED_OUT"),
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusCancelled),
|
||||
@@ -315,7 +316,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunRequeueTenant(ctx context.Co
|
||||
|
||||
requeueAfter := time.Now().UTC().Add(time.Second * 4)
|
||||
|
||||
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
RequeueAfter: &requeueAfter,
|
||||
})
|
||||
|
||||
@@ -335,7 +336,7 @@ func (wc *WorkflowsControllerImpl) runGetGroupKeyRunReassign(ctx context.Context
|
||||
wc.l.Debug().Msgf("workflows controller: checking get group key run reassign")
|
||||
|
||||
// list all tenants
|
||||
tenants, err := wc.repo.Tenant().ListTenants()
|
||||
tenants, err := wc.repo.Tenant().ListTenants(ctx)
|
||||
|
||||
if err != nil {
|
||||
wc.l.Err(err).Msg("could not list tenants")
|
||||
@@ -365,7 +366,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunReassignTenant(ctx context.C
|
||||
ctx, span := telemetry.NewSpan(ctx, "handle-get-group-key-run-reassign")
|
||||
defer span.End()
|
||||
|
||||
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToReassign(tenantId)
|
||||
getGroupKeyRuns, err := ec.repo.GetGroupKeyRun().ListGetGroupKeyRunsToReassign(ctx, tenantId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list get group key runs: %w", err)
|
||||
@@ -389,7 +390,7 @@ func (ec *WorkflowsControllerImpl) runGetGroupKeyRunReassignTenant(ctx context.C
|
||||
|
||||
requeueAfter := time.Now().UTC().Add(time.Second * 4)
|
||||
|
||||
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
innerGetGroupKeyRun, err = ec.repo.GetGroupKeyRun().UpdateGetGroupKeyRun(ctx, tenantId, getGroupKeyRunId, &repository.UpdateGetGroupKeyRunOpts{
|
||||
RequeueAfter: &requeueAfter,
|
||||
Status: repository.StepRunStatusPtr(db.StepRunStatusPendingAssignment),
|
||||
})
|
||||
@@ -416,7 +417,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
|
||||
|
||||
workflowVersionId := sqlchelpers.UUIDToStr(workflowVersion.WorkflowVersion.ID)
|
||||
|
||||
runningWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(tenantId, &repository.ListWorkflowRunsOpts{
|
||||
runningWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(ctx, tenantId, &repository.ListWorkflowRunsOpts{
|
||||
WorkflowVersionId: &workflowVersionId,
|
||||
GroupKey: &groupKey,
|
||||
Statuses: &[]db.WorkflowRunStatus{running},
|
||||
@@ -434,7 +435,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
|
||||
|
||||
maxRuns := int(workflowVersion.ConcurrencyMaxRuns.Int32)
|
||||
|
||||
queuedWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(tenantId, &repository.ListWorkflowRunsOpts{
|
||||
queuedWorkflowRuns, err := wc.repo.WorkflowRun().ListWorkflowRuns(ctx, tenantId, &repository.ListWorkflowRunsOpts{
|
||||
WorkflowVersionId: &workflowVersionId,
|
||||
GroupKey: &groupKey,
|
||||
Statuses: &[]db.WorkflowRunStatus{queued},
|
||||
@@ -462,7 +463,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
|
||||
|
||||
errGroup.Go(func() error {
|
||||
workflowRunId := sqlchelpers.UUIDToStr(row.WorkflowRun.ID)
|
||||
return wc.cancelWorkflowRun(tenantId, workflowRunId)
|
||||
return wc.cancelWorkflowRun(ctx, tenantId, workflowRunId)
|
||||
})
|
||||
}
|
||||
|
||||
@@ -481,7 +482,7 @@ func (wc *WorkflowsControllerImpl) queueByCancelInProgress(ctx context.Context,
|
||||
|
||||
errGroup.Go(func() error {
|
||||
workflowRunId := sqlchelpers.UUIDToStr(row.WorkflowRun.ID)
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(tenantId, workflowRunId)
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, tenantId, workflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get workflow run: %w", err)
|
||||
@@ -509,7 +510,7 @@ func (wc *WorkflowsControllerImpl) queueByGroupRoundRobin(ctx context.Context, t
|
||||
wc.l.Info().Msgf("handling queue with strategy GROUP_ROUND_ROBIN for workflow version %s", workflowVersionId)
|
||||
|
||||
// get workflow runs which are queued for this group key
|
||||
poppedWorkflowRuns, err := wc.repo.WorkflowRun().PopWorkflowRunsRoundRobin(tenantId, workflowId, maxRuns)
|
||||
poppedWorkflowRuns, err := wc.repo.WorkflowRun().PopWorkflowRunsRoundRobin(ctx, tenantId, workflowId, maxRuns)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list queued workflow runs: %w", err)
|
||||
@@ -524,7 +525,7 @@ func (wc *WorkflowsControllerImpl) queueByGroupRoundRobin(ctx context.Context, t
|
||||
workflowRunId := sqlchelpers.UUIDToStr(row.ID)
|
||||
|
||||
wc.l.Info().Msgf("popped workflow run %s", workflowRunId)
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(tenantId, workflowRunId)
|
||||
workflowRun, err := wc.repo.WorkflowRun().GetWorkflowRunById(ctx, tenantId, workflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get workflow run: %w", err)
|
||||
@@ -541,9 +542,15 @@ func (wc *WorkflowsControllerImpl) queueByGroupRoundRobin(ctx context.Context, t
|
||||
return nil
|
||||
}
|
||||
|
||||
func (wc *WorkflowsControllerImpl) cancelWorkflowRun(tenantId, workflowRunId string) error {
|
||||
func (wc *WorkflowsControllerImpl) cancelWorkflowRun(ctx context.Context, tenantId, workflowRunId string) error {
|
||||
// cancel all running step runs
|
||||
stepRuns, err := wc.repo.StepRun().ListRunningStepRunsForWorkflowRun(tenantId, workflowRunId)
|
||||
runningStatus := dbsqlc.StepRunStatusRUNNING
|
||||
stepRuns, err := wc.repo.StepRun().ListStepRuns(ctx, tenantId, &repository.ListStepRunsOpts{
|
||||
Status: &runningStatus,
|
||||
WorkflowRunIds: []string{
|
||||
workflowRunId,
|
||||
},
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not list step runs: %w", err)
|
||||
|
||||
@@ -284,6 +284,49 @@ func (ResourceEventType) EnumDescriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{4}
|
||||
}
|
||||
|
||||
type WorkflowRunEventType int32
|
||||
|
||||
const (
|
||||
WorkflowRunEventType_WORKFLOW_RUN_EVENT_TYPE_FINISHED WorkflowRunEventType = 0
|
||||
)
|
||||
|
||||
// Enum value maps for WorkflowRunEventType.
|
||||
var (
|
||||
WorkflowRunEventType_name = map[int32]string{
|
||||
0: "WORKFLOW_RUN_EVENT_TYPE_FINISHED",
|
||||
}
|
||||
WorkflowRunEventType_value = map[string]int32{
|
||||
"WORKFLOW_RUN_EVENT_TYPE_FINISHED": 0,
|
||||
}
|
||||
)
|
||||
|
||||
func (x WorkflowRunEventType) Enum() *WorkflowRunEventType {
|
||||
p := new(WorkflowRunEventType)
|
||||
*p = x
|
||||
return p
|
||||
}
|
||||
|
||||
func (x WorkflowRunEventType) String() string {
|
||||
return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
|
||||
}
|
||||
|
||||
func (WorkflowRunEventType) Descriptor() protoreflect.EnumDescriptor {
|
||||
return file_dispatcher_proto_enumTypes[5].Descriptor()
|
||||
}
|
||||
|
||||
func (WorkflowRunEventType) Type() protoreflect.EnumType {
|
||||
return &file_dispatcher_proto_enumTypes[5]
|
||||
}
|
||||
|
||||
func (x WorkflowRunEventType) Number() protoreflect.EnumNumber {
|
||||
return protoreflect.EnumNumber(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use WorkflowRunEventType.Descriptor instead.
|
||||
func (WorkflowRunEventType) EnumDescriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{5}
|
||||
}
|
||||
|
||||
type WorkerRegisterRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@@ -1049,6 +1092,54 @@ func (x *SubscribeToWorkflowEventsRequest) GetWorkflowRunId() string {
|
||||
return ""
|
||||
}
|
||||
|
||||
type SubscribeToWorkflowRunsRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
// the id of the workflow run
|
||||
WorkflowRunId string `protobuf:"bytes,1,opt,name=workflowRunId,proto3" json:"workflowRunId,omitempty"`
|
||||
}
|
||||
|
||||
func (x *SubscribeToWorkflowRunsRequest) Reset() {
|
||||
*x = SubscribeToWorkflowRunsRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[10]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *SubscribeToWorkflowRunsRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*SubscribeToWorkflowRunsRequest) ProtoMessage() {}
|
||||
|
||||
func (x *SubscribeToWorkflowRunsRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[10]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use SubscribeToWorkflowRunsRequest.ProtoReflect.Descriptor instead.
|
||||
func (*SubscribeToWorkflowRunsRequest) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{10}
|
||||
}
|
||||
|
||||
func (x *SubscribeToWorkflowRunsRequest) GetWorkflowRunId() string {
|
||||
if x != nil {
|
||||
return x.WorkflowRunId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type WorkflowEvent struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@@ -1074,7 +1165,7 @@ type WorkflowEvent struct {
|
||||
func (x *WorkflowEvent) Reset() {
|
||||
*x = WorkflowEvent{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[10]
|
||||
mi := &file_dispatcher_proto_msgTypes[11]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1087,7 +1178,7 @@ func (x *WorkflowEvent) String() string {
|
||||
func (*WorkflowEvent) ProtoMessage() {}
|
||||
|
||||
func (x *WorkflowEvent) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[10]
|
||||
mi := &file_dispatcher_proto_msgTypes[11]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1100,7 +1191,7 @@ func (x *WorkflowEvent) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use WorkflowEvent.ProtoReflect.Descriptor instead.
|
||||
func (*WorkflowEvent) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{10}
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{11}
|
||||
}
|
||||
|
||||
func (x *WorkflowEvent) GetWorkflowRunId() string {
|
||||
@@ -1166,6 +1257,157 @@ func (x *WorkflowEvent) GetRetryCount() int32 {
|
||||
return 0
|
||||
}
|
||||
|
||||
type WorkflowRunEvent struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
// the id of the workflow run
|
||||
WorkflowRunId string `protobuf:"bytes,1,opt,name=workflowRunId,proto3" json:"workflowRunId,omitempty"`
|
||||
EventType WorkflowRunEventType `protobuf:"varint,2,opt,name=eventType,proto3,enum=WorkflowRunEventType" json:"eventType,omitempty"`
|
||||
EventTimestamp *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=eventTimestamp,proto3" json:"eventTimestamp,omitempty"`
|
||||
Results []*StepRunResult `protobuf:"bytes,4,rep,name=results,proto3" json:"results,omitempty"`
|
||||
}
|
||||
|
||||
func (x *WorkflowRunEvent) Reset() {
|
||||
*x = WorkflowRunEvent{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[12]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *WorkflowRunEvent) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*WorkflowRunEvent) ProtoMessage() {}
|
||||
|
||||
func (x *WorkflowRunEvent) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[12]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use WorkflowRunEvent.ProtoReflect.Descriptor instead.
|
||||
func (*WorkflowRunEvent) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{12}
|
||||
}
|
||||
|
||||
func (x *WorkflowRunEvent) GetWorkflowRunId() string {
|
||||
if x != nil {
|
||||
return x.WorkflowRunId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *WorkflowRunEvent) GetEventType() WorkflowRunEventType {
|
||||
if x != nil {
|
||||
return x.EventType
|
||||
}
|
||||
return WorkflowRunEventType_WORKFLOW_RUN_EVENT_TYPE_FINISHED
|
||||
}
|
||||
|
||||
func (x *WorkflowRunEvent) GetEventTimestamp() *timestamppb.Timestamp {
|
||||
if x != nil {
|
||||
return x.EventTimestamp
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *WorkflowRunEvent) GetResults() []*StepRunResult {
|
||||
if x != nil {
|
||||
return x.Results
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type StepRunResult struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
StepRunId string `protobuf:"bytes,1,opt,name=stepRunId,proto3" json:"stepRunId,omitempty"`
|
||||
StepReadableId string `protobuf:"bytes,2,opt,name=stepReadableId,proto3" json:"stepReadableId,omitempty"`
|
||||
JobRunId string `protobuf:"bytes,3,opt,name=jobRunId,proto3" json:"jobRunId,omitempty"`
|
||||
Error *string `protobuf:"bytes,4,opt,name=error,proto3,oneof" json:"error,omitempty"`
|
||||
Output *string `protobuf:"bytes,5,opt,name=output,proto3,oneof" json:"output,omitempty"`
|
||||
}
|
||||
|
||||
func (x *StepRunResult) Reset() {
|
||||
*x = StepRunResult{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[13]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *StepRunResult) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*StepRunResult) ProtoMessage() {}
|
||||
|
||||
func (x *StepRunResult) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[13]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use StepRunResult.ProtoReflect.Descriptor instead.
|
||||
func (*StepRunResult) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{13}
|
||||
}
|
||||
|
||||
func (x *StepRunResult) GetStepRunId() string {
|
||||
if x != nil {
|
||||
return x.StepRunId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StepRunResult) GetStepReadableId() string {
|
||||
if x != nil {
|
||||
return x.StepReadableId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StepRunResult) GetJobRunId() string {
|
||||
if x != nil {
|
||||
return x.JobRunId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StepRunResult) GetError() string {
|
||||
if x != nil && x.Error != nil {
|
||||
return *x.Error
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *StepRunResult) GetOutput() string {
|
||||
if x != nil && x.Output != nil {
|
||||
return *x.Output
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type OverridesData struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
@@ -1184,7 +1426,7 @@ type OverridesData struct {
|
||||
func (x *OverridesData) Reset() {
|
||||
*x = OverridesData{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[11]
|
||||
mi := &file_dispatcher_proto_msgTypes[14]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1197,7 +1439,7 @@ func (x *OverridesData) String() string {
|
||||
func (*OverridesData) ProtoMessage() {}
|
||||
|
||||
func (x *OverridesData) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[11]
|
||||
mi := &file_dispatcher_proto_msgTypes[14]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1210,7 +1452,7 @@ func (x *OverridesData) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use OverridesData.ProtoReflect.Descriptor instead.
|
||||
func (*OverridesData) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{11}
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{14}
|
||||
}
|
||||
|
||||
func (x *OverridesData) GetStepRunId() string {
|
||||
@@ -1250,7 +1492,7 @@ type OverridesDataResponse struct {
|
||||
func (x *OverridesDataResponse) Reset() {
|
||||
*x = OverridesDataResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[12]
|
||||
mi := &file_dispatcher_proto_msgTypes[15]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1263,7 +1505,7 @@ func (x *OverridesDataResponse) String() string {
|
||||
func (*OverridesDataResponse) ProtoMessage() {}
|
||||
|
||||
func (x *OverridesDataResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[12]
|
||||
mi := &file_dispatcher_proto_msgTypes[15]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1276,7 +1518,7 @@ func (x *OverridesDataResponse) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use OverridesDataResponse.ProtoReflect.Descriptor instead.
|
||||
func (*OverridesDataResponse) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{12}
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{15}
|
||||
}
|
||||
|
||||
type HeartbeatRequest struct {
|
||||
@@ -1293,7 +1535,7 @@ type HeartbeatRequest struct {
|
||||
func (x *HeartbeatRequest) Reset() {
|
||||
*x = HeartbeatRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[13]
|
||||
mi := &file_dispatcher_proto_msgTypes[16]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1306,7 +1548,7 @@ func (x *HeartbeatRequest) String() string {
|
||||
func (*HeartbeatRequest) ProtoMessage() {}
|
||||
|
||||
func (x *HeartbeatRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[13]
|
||||
mi := &file_dispatcher_proto_msgTypes[16]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1319,7 +1561,7 @@ func (x *HeartbeatRequest) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use HeartbeatRequest.ProtoReflect.Descriptor instead.
|
||||
func (*HeartbeatRequest) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{13}
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{16}
|
||||
}
|
||||
|
||||
func (x *HeartbeatRequest) GetWorkerId() string {
|
||||
@@ -1345,7 +1587,7 @@ type HeartbeatResponse struct {
|
||||
func (x *HeartbeatResponse) Reset() {
|
||||
*x = HeartbeatResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_dispatcher_proto_msgTypes[14]
|
||||
mi := &file_dispatcher_proto_msgTypes[17]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
@@ -1358,7 +1600,7 @@ func (x *HeartbeatResponse) String() string {
|
||||
func (*HeartbeatResponse) ProtoMessage() {}
|
||||
|
||||
func (x *HeartbeatResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_dispatcher_proto_msgTypes[14]
|
||||
mi := &file_dispatcher_proto_msgTypes[17]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
@@ -1371,7 +1613,7 @@ func (x *HeartbeatResponse) ProtoReflect() protoreflect.Message {
|
||||
|
||||
// Deprecated: Use HeartbeatResponse.ProtoReflect.Descriptor instead.
|
||||
func (*HeartbeatResponse) Descriptor() ([]byte, []int) {
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{14}
|
||||
return file_dispatcher_proto_rawDescGZIP(), []int{17}
|
||||
}
|
||||
|
||||
var File_dispatcher_proto protoreflect.FileDescriptor
|
||||
@@ -1484,140 +1726,180 @@ var file_dispatcher_proto_rawDesc = []byte{
|
||||
0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75,
|
||||
0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
|
||||
0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b,
|
||||
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x22, 0xa5, 0x03, 0x0a, 0x0d, 0x57, 0x6f,
|
||||
0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77,
|
||||
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x22, 0x46, 0x0a, 0x1e, 0x53, 0x75, 0x62,
|
||||
0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
|
||||
0x52, 0x75, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77,
|
||||
0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49,
|
||||
0x64, 0x12, 0x31, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70,
|
||||
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0d, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
||||
0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
|
||||
0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
|
||||
0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x12, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
||||
0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65,
|
||||
0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
|
||||
0x63, 0x65, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x6f,
|
||||
0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x42, 0x0a, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54,
|
||||
0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a,
|
||||
0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66,
|
||||
0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e,
|
||||
0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76,
|
||||
0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x16,
|
||||
0x0a, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06,
|
||||
0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x12, 0x25, 0x0a, 0x0b, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65,
|
||||
0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0b, 0x73,
|
||||
0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x88, 0x01, 0x01, 0x12, 0x23, 0x0a,
|
||||
0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28,
|
||||
0x05, 0x48, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x88,
|
||||
0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69,
|
||||
0x65, 0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e,
|
||||
0x74, 0x22, 0x7f, 0x0a, 0x0d, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61,
|
||||
0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64,
|
||||
0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
|
||||
0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20,
|
||||
0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x61,
|
||||
0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61,
|
||||
0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44,
|
||||
0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6c, 0x0a, 0x10, 0x48,
|
||||
0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
|
||||
0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x68,
|
||||
0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
|
||||
0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
|
||||
0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0b, 0x68, 0x65,
|
||||
0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41, 0x74, 0x22, 0x13, 0x0a, 0x11, 0x48, 0x65, 0x61,
|
||||
0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x4e,
|
||||
0x0a, 0x0a, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x0e,
|
||||
0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x00,
|
||||
0x12, 0x13, 0x0a, 0x0f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f,
|
||||
0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x47,
|
||||
0x45, 0x54, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x02, 0x2a, 0xa2,
|
||||
0x01, 0x0a, 0x17, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52,
|
||||
0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
|
||||
0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c,
|
||||
0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f,
|
||||
0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x22,
|
||||
0x0a, 0x1e, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e,
|
||||
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44,
|
||||
0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f,
|
||||
0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45,
|
||||
0x44, 0x10, 0x03, 0x2a, 0x8a, 0x01, 0x0a, 0x13, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69,
|
||||
0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x53,
|
||||
0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55,
|
||||
0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50,
|
||||
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52,
|
||||
0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56,
|
||||
0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54,
|
||||
0x45, 0x44, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45,
|
||||
0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03,
|
||||
0x2a, 0x65, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65,
|
||||
0x12, 0x19, 0x0a, 0x15, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50,
|
||||
0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x52,
|
||||
0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x45,
|
||||
0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55,
|
||||
0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f,
|
||||
0x57, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x02, 0x2a, 0xfe, 0x01, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f,
|
||||
0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a,
|
||||
0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f,
|
||||
0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1f,
|
||||
0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54,
|
||||
0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12,
|
||||
0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e,
|
||||
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44,
|
||||
0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45,
|
||||
0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44,
|
||||
0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45,
|
||||
0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c,
|
||||
0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43,
|
||||
0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d,
|
||||
0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x05, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f,
|
||||
0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
|
||||
0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x10, 0x06, 0x32, 0xd1, 0x04, 0x0a, 0x0a, 0x44, 0x69, 0x73,
|
||||
0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x08, 0x52, 0x65, 0x67, 0x69, 0x73,
|
||||
0x74, 0x65, 0x72, 0x12, 0x16, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69,
|
||||
0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x57, 0x6f,
|
||||
0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x33, 0x0a, 0x06, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e,
|
||||
0x12, 0x14, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65,
|
||||
0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x30, 0x01, 0x12, 0x35, 0x0a, 0x08, 0x4c,
|
||||
0x69, 0x73, 0x74, 0x65, 0x6e, 0x56, 0x32, 0x12, 0x14, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72,
|
||||
0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e,
|
||||
0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00,
|
||||
0x30, 0x01, 0x12, 0x34, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12,
|
||||
0x11, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x1a, 0x12, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65,
|
||||
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x19, 0x53, 0x75, 0x62, 0x73,
|
||||
0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45,
|
||||
0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
|
||||
0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74,
|
||||
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66,
|
||||
0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3f, 0x0a, 0x13,
|
||||
0x53, 0x65, 0x6e, 0x64, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76,
|
||||
0x65, 0x6e, 0x74, 0x12, 0x10, 0x2e, 0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||
0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76,
|
||||
0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a,
|
||||
0x17, 0x53, 0x65, 0x6e, 0x64, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x14, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70,
|
||||
0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14,
|
||||
0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3c, 0x0a, 0x10, 0x50, 0x75, 0x74, 0x4f, 0x76, 0x65,
|
||||
0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x0e, 0x2e, 0x4f, 0x76, 0x65,
|
||||
0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x1a, 0x16, 0x2e, 0x4f, 0x76, 0x65,
|
||||
0x64, 0x22, 0xa5, 0x03, 0x0a, 0x0d, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76,
|
||||
0x65, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
|
||||
0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b,
|
||||
0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x31, 0x0a, 0x0c, 0x72, 0x65, 0x73,
|
||||
0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||
0x0d, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c,
|
||||
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x09,
|
||||
0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||
0x12, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54,
|
||||
0x79, 0x70, 0x65, 0x52, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e,
|
||||
0x0a, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x18, 0x04, 0x20, 0x01,
|
||||
0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x42,
|
||||
0x0a, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
|
||||
0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e,
|
||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
|
||||
0x6d, 0x70, 0x52, 0x0e, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
|
||||
0x6d, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50, 0x61, 0x79, 0x6c, 0x6f,
|
||||
0x61, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x50,
|
||||
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70,
|
||||
0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x68, 0x61, 0x6e, 0x67, 0x75, 0x70, 0x12, 0x25,
|
||||
0x0a, 0x0b, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x08, 0x20,
|
||||
0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69,
|
||||
0x65, 0x73, 0x88, 0x01, 0x01, 0x12, 0x23, 0x0a, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x43, 0x6f,
|
||||
0x75, 0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x0a, 0x72, 0x65, 0x74,
|
||||
0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x88, 0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x73,
|
||||
0x74, 0x65, 0x70, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x73, 0x42, 0x0d, 0x0a, 0x0b, 0x5f, 0x72,
|
||||
0x65, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xdb, 0x01, 0x0a, 0x10, 0x57, 0x6f,
|
||||
0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24,
|
||||
0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18,
|
||||
0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52,
|
||||
0x75, 0x6e, 0x49, 0x64, 0x12, 0x33, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
|
||||
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c,
|
||||
0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09,
|
||||
0x65, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x42, 0x0a, 0x0e, 0x65, 0x76, 0x65,
|
||||
0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||
0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
||||
0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x0e, 0x65,
|
||||
0x76, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x28, 0x0a,
|
||||
0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e,
|
||||
0x2e, 0x53, 0x74, 0x65, 0x70, 0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07,
|
||||
0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0xbe, 0x01, 0x0a, 0x0d, 0x53, 0x74, 0x65, 0x70,
|
||||
0x52, 0x75, 0x6e, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65,
|
||||
0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74,
|
||||
0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x0e, 0x73, 0x74, 0x65, 0x70, 0x52,
|
||||
0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x0e, 0x73, 0x74, 0x65, 0x70, 0x52, 0x65, 0x61, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12,
|
||||
0x1a, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x08, 0x6a, 0x6f, 0x62, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x05, 0x65,
|
||||
0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x05, 0x65, 0x72,
|
||||
0x72, 0x6f, 0x72, 0x88, 0x01, 0x01, 0x12, 0x1b, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74,
|
||||
0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x48, 0x01, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74,
|
||||
0x88, 0x01, 0x01, 0x42, 0x08, 0x0a, 0x06, 0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x42, 0x09, 0x0a,
|
||||
0x07, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x22, 0x7f, 0x0a, 0x0d, 0x4f, 0x76, 0x65, 0x72,
|
||||
0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x74, 0x65,
|
||||
0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74,
|
||||
0x65, 0x70, 0x52, 0x75, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18,
|
||||
0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x76,
|
||||
0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
|
||||
0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e,
|
||||
0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x65,
|
||||
0x72, 0x46, 0x69, 0x6c, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x17, 0x0a, 0x15, 0x4f, 0x76, 0x65,
|
||||
0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
|
||||
0x73, 0x65, 0x22, 0x00, 0x12, 0x46, 0x0a, 0x0b, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72,
|
||||
0x69, 0x62, 0x65, 0x12, 0x19, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75,
|
||||
0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a,
|
||||
0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
|
||||
0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x47, 0x5a, 0x45,
|
||||
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68,
|
||||
0x65, 0x74, 0x2d, 0x64, 0x65, 0x76, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2f, 0x69,
|
||||
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73,
|
||||
0x2f, 0x64, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x2f, 0x63, 0x6f, 0x6e, 0x74,
|
||||
0x72, 0x61, 0x63, 0x74, 0x73, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
0x73, 0x65, 0x22, 0x6c, 0x0a, 0x10, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52,
|
||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
|
||||
0x49, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72,
|
||||
0x49, 0x64, 0x12, 0x3c, 0x0a, 0x0b, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41,
|
||||
0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
|
||||
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74,
|
||||
0x61, 0x6d, 0x70, 0x52, 0x0b, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x41, 0x74,
|
||||
0x22, 0x13, 0x0a, 0x11, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73,
|
||||
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x4e, 0x0a, 0x0a, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54,
|
||||
0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x0e, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x45,
|
||||
0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x41, 0x4e, 0x43, 0x45,
|
||||
0x4c, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x17, 0x0a, 0x13,
|
||||
0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x47, 0x45, 0x54, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f,
|
||||
0x4b, 0x45, 0x59, 0x10, 0x02, 0x2a, 0xa2, 0x01, 0x0a, 0x17, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b,
|
||||
0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70,
|
||||
0x65, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45,
|
||||
0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57,
|
||||
0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59,
|
||||
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52,
|
||||
0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x4b,
|
||||
0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f,
|
||||
0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1f, 0x0a, 0x1b, 0x47, 0x52, 0x4f,
|
||||
0x55, 0x50, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50,
|
||||
0x45, 0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x8a, 0x01, 0x0a, 0x13, 0x53,
|
||||
0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x54, 0x79,
|
||||
0x70, 0x65, 0x12, 0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54,
|
||||
0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12,
|
||||
0x1b, 0x0a, 0x17, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
|
||||
0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19,
|
||||
0x53, 0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
|
||||
0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1a, 0x0a, 0x16, 0x53,
|
||||
0x54, 0x45, 0x50, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46,
|
||||
0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x2a, 0x65, 0x0a, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75,
|
||||
0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x45, 0x53, 0x4f, 0x55,
|
||||
0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e,
|
||||
0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54,
|
||||
0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x45, 0x50, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x01, 0x12, 0x1e,
|
||||
0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f,
|
||||
0x57, 0x4f, 0x52, 0x4b, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x52, 0x55, 0x4e, 0x10, 0x02, 0x2a, 0xfe,
|
||||
0x01, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74,
|
||||
0x54, 0x79, 0x70, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45,
|
||||
0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e,
|
||||
0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43,
|
||||
0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41,
|
||||
0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52,
|
||||
0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x4f,
|
||||
0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53,
|
||||
0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,
|
||||
0x5f, 0x46, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x52, 0x45, 0x53,
|
||||
0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45,
|
||||
0x5f, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x21, 0x0a, 0x1d,
|
||||
0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54,
|
||||
0x59, 0x50, 0x45, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x44, 0x5f, 0x4f, 0x55, 0x54, 0x10, 0x05, 0x12,
|
||||
0x1e, 0x0a, 0x1a, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, 0x5f, 0x45, 0x56, 0x45, 0x4e,
|
||||
0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x10, 0x06, 0x2a,
|
||||
0x3c, 0x0a, 0x14, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76,
|
||||
0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x57, 0x4f, 0x52, 0x4b, 0x46,
|
||||
0x4c, 0x4f, 0x57, 0x5f, 0x52, 0x55, 0x4e, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x59,
|
||||
0x50, 0x45, 0x5f, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x45, 0x44, 0x10, 0x00, 0x32, 0xa6, 0x05,
|
||||
0x0a, 0x0a, 0x44, 0x69, 0x73, 0x70, 0x61, 0x74, 0x63, 0x68, 0x65, 0x72, 0x12, 0x3d, 0x0a, 0x08,
|
||||
0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x12, 0x16, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65,
|
||||
0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
|
||||
0x1a, 0x17, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65,
|
||||
0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x33, 0x0a, 0x06, 0x4c,
|
||||
0x69, 0x73, 0x74, 0x65, 0x6e, 0x12, 0x14, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69,
|
||||
0x73, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x41, 0x73,
|
||||
0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x30, 0x01,
|
||||
0x12, 0x35, 0x0a, 0x08, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x56, 0x32, 0x12, 0x14, 0x2e, 0x57,
|
||||
0x6f, 0x72, 0x6b, 0x65, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65,
|
||||
0x73, 0x74, 0x1a, 0x0f, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x41, 0x63, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x22, 0x00, 0x30, 0x01, 0x12, 0x34, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74,
|
||||
0x62, 0x65, 0x61, 0x74, 0x12, 0x11, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74,
|
||||
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x12, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62,
|
||||
0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a,
|
||||
0x19, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b,
|
||||
0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x53, 0x75, 0x62,
|
||||
0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77,
|
||||
0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x0e, 0x2e,
|
||||
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30,
|
||||
0x01, 0x12, 0x53, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f,
|
||||
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x73, 0x12, 0x1f, 0x2e, 0x53,
|
||||
0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x54, 0x6f, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c,
|
||||
0x6f, 0x77, 0x52, 0x75, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x11, 0x2e,
|
||||
0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x75, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74,
|
||||
0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x3f, 0x0a, 0x13, 0x53, 0x65, 0x6e, 0x64, 0x53, 0x74,
|
||||
0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x10, 0x2e,
|
||||
0x53, 0x74, 0x65, 0x70, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a,
|
||||
0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73,
|
||||
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x47, 0x0a, 0x17, 0x53, 0x65, 0x6e, 0x64, 0x47,
|
||||
0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65,
|
||||
0x6e, 0x74, 0x12, 0x14, 0x2e, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x74,
|
||||
0x69, 0x6f, 0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x14, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f,
|
||||
0x6e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||
0x12, 0x3c, 0x0a, 0x10, 0x50, 0x75, 0x74, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73,
|
||||
0x44, 0x61, 0x74, 0x61, 0x12, 0x0e, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73,
|
||||
0x44, 0x61, 0x74, 0x61, 0x1a, 0x16, 0x2e, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73,
|
||||
0x44, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x46,
|
||||
0x0a, 0x0b, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x19, 0x2e,
|
||||
0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
|
||||
0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65,
|
||||
0x72, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70,
|
||||
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x47, 0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62,
|
||||
0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2d, 0x64, 0x65, 0x76,
|
||||
0x2f, 0x68, 0x61, 0x74, 0x63, 0x68, 0x65, 0x74, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
|
||||
0x6c, 0x2f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x2f, 0x64, 0x69, 0x73, 0x70, 0x61,
|
||||
0x74, 0x63, 0x68, 0x65, 0x72, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x73, 0x62,
|
||||
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
@@ -1632,64 +1914,73 @@ func file_dispatcher_proto_rawDescGZIP() []byte {
|
||||
return file_dispatcher_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_dispatcher_proto_enumTypes = make([]protoimpl.EnumInfo, 5)
|
||||
var file_dispatcher_proto_msgTypes = make([]protoimpl.MessageInfo, 15)
|
||||
var file_dispatcher_proto_enumTypes = make([]protoimpl.EnumInfo, 6)
|
||||
var file_dispatcher_proto_msgTypes = make([]protoimpl.MessageInfo, 18)
|
||||
var file_dispatcher_proto_goTypes = []interface{}{
|
||||
(ActionType)(0), // 0: ActionType
|
||||
(GroupKeyActionEventType)(0), // 1: GroupKeyActionEventType
|
||||
(StepActionEventType)(0), // 2: StepActionEventType
|
||||
(ResourceType)(0), // 3: ResourceType
|
||||
(ResourceEventType)(0), // 4: ResourceEventType
|
||||
(*WorkerRegisterRequest)(nil), // 5: WorkerRegisterRequest
|
||||
(*WorkerRegisterResponse)(nil), // 6: WorkerRegisterResponse
|
||||
(*AssignedAction)(nil), // 7: AssignedAction
|
||||
(*WorkerListenRequest)(nil), // 8: WorkerListenRequest
|
||||
(*WorkerUnsubscribeRequest)(nil), // 9: WorkerUnsubscribeRequest
|
||||
(*WorkerUnsubscribeResponse)(nil), // 10: WorkerUnsubscribeResponse
|
||||
(*GroupKeyActionEvent)(nil), // 11: GroupKeyActionEvent
|
||||
(*StepActionEvent)(nil), // 12: StepActionEvent
|
||||
(*ActionEventResponse)(nil), // 13: ActionEventResponse
|
||||
(*SubscribeToWorkflowEventsRequest)(nil), // 14: SubscribeToWorkflowEventsRequest
|
||||
(*WorkflowEvent)(nil), // 15: WorkflowEvent
|
||||
(*OverridesData)(nil), // 16: OverridesData
|
||||
(*OverridesDataResponse)(nil), // 17: OverridesDataResponse
|
||||
(*HeartbeatRequest)(nil), // 18: HeartbeatRequest
|
||||
(*HeartbeatResponse)(nil), // 19: HeartbeatResponse
|
||||
(*timestamppb.Timestamp)(nil), // 20: google.protobuf.Timestamp
|
||||
(WorkflowRunEventType)(0), // 5: WorkflowRunEventType
|
||||
(*WorkerRegisterRequest)(nil), // 6: WorkerRegisterRequest
|
||||
(*WorkerRegisterResponse)(nil), // 7: WorkerRegisterResponse
|
||||
(*AssignedAction)(nil), // 8: AssignedAction
|
||||
(*WorkerListenRequest)(nil), // 9: WorkerListenRequest
|
||||
(*WorkerUnsubscribeRequest)(nil), // 10: WorkerUnsubscribeRequest
|
||||
(*WorkerUnsubscribeResponse)(nil), // 11: WorkerUnsubscribeResponse
|
||||
(*GroupKeyActionEvent)(nil), // 12: GroupKeyActionEvent
|
||||
(*StepActionEvent)(nil), // 13: StepActionEvent
|
||||
(*ActionEventResponse)(nil), // 14: ActionEventResponse
|
||||
(*SubscribeToWorkflowEventsRequest)(nil), // 15: SubscribeToWorkflowEventsRequest
|
||||
(*SubscribeToWorkflowRunsRequest)(nil), // 16: SubscribeToWorkflowRunsRequest
|
||||
(*WorkflowEvent)(nil), // 17: WorkflowEvent
|
||||
(*WorkflowRunEvent)(nil), // 18: WorkflowRunEvent
|
||||
(*StepRunResult)(nil), // 19: StepRunResult
|
||||
(*OverridesData)(nil), // 20: OverridesData
|
||||
(*OverridesDataResponse)(nil), // 21: OverridesDataResponse
|
||||
(*HeartbeatRequest)(nil), // 22: HeartbeatRequest
|
||||
(*HeartbeatResponse)(nil), // 23: HeartbeatResponse
|
||||
(*timestamppb.Timestamp)(nil), // 24: google.protobuf.Timestamp
|
||||
}
|
||||
var file_dispatcher_proto_depIdxs = []int32{
|
||||
0, // 0: AssignedAction.actionType:type_name -> ActionType
|
||||
20, // 1: GroupKeyActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
24, // 1: GroupKeyActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
1, // 2: GroupKeyActionEvent.eventType:type_name -> GroupKeyActionEventType
|
||||
20, // 3: StepActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
24, // 3: StepActionEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
2, // 4: StepActionEvent.eventType:type_name -> StepActionEventType
|
||||
3, // 5: WorkflowEvent.resourceType:type_name -> ResourceType
|
||||
4, // 6: WorkflowEvent.eventType:type_name -> ResourceEventType
|
||||
20, // 7: WorkflowEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
20, // 8: HeartbeatRequest.heartbeatAt:type_name -> google.protobuf.Timestamp
|
||||
5, // 9: Dispatcher.Register:input_type -> WorkerRegisterRequest
|
||||
8, // 10: Dispatcher.Listen:input_type -> WorkerListenRequest
|
||||
8, // 11: Dispatcher.ListenV2:input_type -> WorkerListenRequest
|
||||
18, // 12: Dispatcher.Heartbeat:input_type -> HeartbeatRequest
|
||||
14, // 13: Dispatcher.SubscribeToWorkflowEvents:input_type -> SubscribeToWorkflowEventsRequest
|
||||
12, // 14: Dispatcher.SendStepActionEvent:input_type -> StepActionEvent
|
||||
11, // 15: Dispatcher.SendGroupKeyActionEvent:input_type -> GroupKeyActionEvent
|
||||
16, // 16: Dispatcher.PutOverridesData:input_type -> OverridesData
|
||||
9, // 17: Dispatcher.Unsubscribe:input_type -> WorkerUnsubscribeRequest
|
||||
6, // 18: Dispatcher.Register:output_type -> WorkerRegisterResponse
|
||||
7, // 19: Dispatcher.Listen:output_type -> AssignedAction
|
||||
7, // 20: Dispatcher.ListenV2:output_type -> AssignedAction
|
||||
19, // 21: Dispatcher.Heartbeat:output_type -> HeartbeatResponse
|
||||
15, // 22: Dispatcher.SubscribeToWorkflowEvents:output_type -> WorkflowEvent
|
||||
13, // 23: Dispatcher.SendStepActionEvent:output_type -> ActionEventResponse
|
||||
13, // 24: Dispatcher.SendGroupKeyActionEvent:output_type -> ActionEventResponse
|
||||
17, // 25: Dispatcher.PutOverridesData:output_type -> OverridesDataResponse
|
||||
10, // 26: Dispatcher.Unsubscribe:output_type -> WorkerUnsubscribeResponse
|
||||
18, // [18:27] is the sub-list for method output_type
|
||||
9, // [9:18] is the sub-list for method input_type
|
||||
9, // [9:9] is the sub-list for extension type_name
|
||||
9, // [9:9] is the sub-list for extension extendee
|
||||
0, // [0:9] is the sub-list for field type_name
|
||||
24, // 7: WorkflowEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
5, // 8: WorkflowRunEvent.eventType:type_name -> WorkflowRunEventType
|
||||
24, // 9: WorkflowRunEvent.eventTimestamp:type_name -> google.protobuf.Timestamp
|
||||
19, // 10: WorkflowRunEvent.results:type_name -> StepRunResult
|
||||
24, // 11: HeartbeatRequest.heartbeatAt:type_name -> google.protobuf.Timestamp
|
||||
6, // 12: Dispatcher.Register:input_type -> WorkerRegisterRequest
|
||||
9, // 13: Dispatcher.Listen:input_type -> WorkerListenRequest
|
||||
9, // 14: Dispatcher.ListenV2:input_type -> WorkerListenRequest
|
||||
22, // 15: Dispatcher.Heartbeat:input_type -> HeartbeatRequest
|
||||
15, // 16: Dispatcher.SubscribeToWorkflowEvents:input_type -> SubscribeToWorkflowEventsRequest
|
||||
16, // 17: Dispatcher.SubscribeToWorkflowRuns:input_type -> SubscribeToWorkflowRunsRequest
|
||||
13, // 18: Dispatcher.SendStepActionEvent:input_type -> StepActionEvent
|
||||
12, // 19: Dispatcher.SendGroupKeyActionEvent:input_type -> GroupKeyActionEvent
|
||||
20, // 20: Dispatcher.PutOverridesData:input_type -> OverridesData
|
||||
10, // 21: Dispatcher.Unsubscribe:input_type -> WorkerUnsubscribeRequest
|
||||
7, // 22: Dispatcher.Register:output_type -> WorkerRegisterResponse
|
||||
8, // 23: Dispatcher.Listen:output_type -> AssignedAction
|
||||
8, // 24: Dispatcher.ListenV2:output_type -> AssignedAction
|
||||
23, // 25: Dispatcher.Heartbeat:output_type -> HeartbeatResponse
|
||||
17, // 26: Dispatcher.SubscribeToWorkflowEvents:output_type -> WorkflowEvent
|
||||
18, // 27: Dispatcher.SubscribeToWorkflowRuns:output_type -> WorkflowRunEvent
|
||||
14, // 28: Dispatcher.SendStepActionEvent:output_type -> ActionEventResponse
|
||||
14, // 29: Dispatcher.SendGroupKeyActionEvent:output_type -> ActionEventResponse
|
||||
21, // 30: Dispatcher.PutOverridesData:output_type -> OverridesDataResponse
|
||||
11, // 31: Dispatcher.Unsubscribe:output_type -> WorkerUnsubscribeResponse
|
||||
22, // [22:32] is the sub-list for method output_type
|
||||
12, // [12:22] is the sub-list for method input_type
|
||||
12, // [12:12] is the sub-list for extension type_name
|
||||
12, // [12:12] is the sub-list for extension extendee
|
||||
0, // [0:12] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_dispatcher_proto_init() }
|
||||
@@ -1819,7 +2110,7 @@ func file_dispatcher_proto_init() {
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*WorkflowEvent); i {
|
||||
switch v := v.(*SubscribeToWorkflowRunsRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@@ -1831,7 +2122,7 @@ func file_dispatcher_proto_init() {
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*OverridesData); i {
|
||||
switch v := v.(*WorkflowEvent); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@@ -1843,7 +2134,7 @@ func file_dispatcher_proto_init() {
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*OverridesDataResponse); i {
|
||||
switch v := v.(*WorkflowRunEvent); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@@ -1855,7 +2146,7 @@ func file_dispatcher_proto_init() {
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*HeartbeatRequest); i {
|
||||
switch v := v.(*StepRunResult); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
@@ -1867,6 +2158,42 @@ func file_dispatcher_proto_init() {
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*OverridesData); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*OverridesDataResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*HeartbeatRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
|
||||
switch v := v.(*HeartbeatResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
@@ -1880,14 +2207,15 @@ func file_dispatcher_proto_init() {
|
||||
}
|
||||
}
|
||||
file_dispatcher_proto_msgTypes[0].OneofWrappers = []interface{}{}
|
||||
file_dispatcher_proto_msgTypes[10].OneofWrappers = []interface{}{}
|
||||
file_dispatcher_proto_msgTypes[11].OneofWrappers = []interface{}{}
|
||||
file_dispatcher_proto_msgTypes[13].OneofWrappers = []interface{}{}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_dispatcher_proto_rawDesc,
|
||||
NumEnums: 5,
|
||||
NumMessages: 15,
|
||||
NumEnums: 6,
|
||||
NumMessages: 18,
|
||||
NumExtensions: 0,
|
||||
NumServices: 1,
|
||||
},
|
||||
|
||||
@@ -30,6 +30,7 @@ type DispatcherClient interface {
|
||||
// Heartbeat is a method for workers to send heartbeats to the dispatcher
|
||||
Heartbeat(ctx context.Context, in *HeartbeatRequest, opts ...grpc.CallOption) (*HeartbeatResponse, error)
|
||||
SubscribeToWorkflowEvents(ctx context.Context, in *SubscribeToWorkflowEventsRequest, opts ...grpc.CallOption) (Dispatcher_SubscribeToWorkflowEventsClient, error)
|
||||
SubscribeToWorkflowRuns(ctx context.Context, opts ...grpc.CallOption) (Dispatcher_SubscribeToWorkflowRunsClient, error)
|
||||
SendStepActionEvent(ctx context.Context, in *StepActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error)
|
||||
SendGroupKeyActionEvent(ctx context.Context, in *GroupKeyActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error)
|
||||
PutOverridesData(ctx context.Context, in *OverridesData, opts ...grpc.CallOption) (*OverridesDataResponse, error)
|
||||
@@ -158,6 +159,37 @@ func (x *dispatcherSubscribeToWorkflowEventsClient) Recv() (*WorkflowEvent, erro
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *dispatcherClient) SubscribeToWorkflowRuns(ctx context.Context, opts ...grpc.CallOption) (Dispatcher_SubscribeToWorkflowRunsClient, error) {
|
||||
stream, err := c.cc.NewStream(ctx, &Dispatcher_ServiceDesc.Streams[3], "/Dispatcher/SubscribeToWorkflowRuns", opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
x := &dispatcherSubscribeToWorkflowRunsClient{stream}
|
||||
return x, nil
|
||||
}
|
||||
|
||||
type Dispatcher_SubscribeToWorkflowRunsClient interface {
|
||||
Send(*SubscribeToWorkflowRunsRequest) error
|
||||
Recv() (*WorkflowRunEvent, error)
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
type dispatcherSubscribeToWorkflowRunsClient struct {
|
||||
grpc.ClientStream
|
||||
}
|
||||
|
||||
func (x *dispatcherSubscribeToWorkflowRunsClient) Send(m *SubscribeToWorkflowRunsRequest) error {
|
||||
return x.ClientStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *dispatcherSubscribeToWorkflowRunsClient) Recv() (*WorkflowRunEvent, error) {
|
||||
m := new(WorkflowRunEvent)
|
||||
if err := x.ClientStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func (c *dispatcherClient) SendStepActionEvent(ctx context.Context, in *StepActionEvent, opts ...grpc.CallOption) (*ActionEventResponse, error) {
|
||||
out := new(ActionEventResponse)
|
||||
err := c.cc.Invoke(ctx, "/Dispatcher/SendStepActionEvent", in, out, opts...)
|
||||
@@ -206,6 +238,7 @@ type DispatcherServer interface {
|
||||
// Heartbeat is a method for workers to send heartbeats to the dispatcher
|
||||
Heartbeat(context.Context, *HeartbeatRequest) (*HeartbeatResponse, error)
|
||||
SubscribeToWorkflowEvents(*SubscribeToWorkflowEventsRequest, Dispatcher_SubscribeToWorkflowEventsServer) error
|
||||
SubscribeToWorkflowRuns(Dispatcher_SubscribeToWorkflowRunsServer) error
|
||||
SendStepActionEvent(context.Context, *StepActionEvent) (*ActionEventResponse, error)
|
||||
SendGroupKeyActionEvent(context.Context, *GroupKeyActionEvent) (*ActionEventResponse, error)
|
||||
PutOverridesData(context.Context, *OverridesData) (*OverridesDataResponse, error)
|
||||
@@ -232,6 +265,9 @@ func (UnimplementedDispatcherServer) Heartbeat(context.Context, *HeartbeatReques
|
||||
func (UnimplementedDispatcherServer) SubscribeToWorkflowEvents(*SubscribeToWorkflowEventsRequest, Dispatcher_SubscribeToWorkflowEventsServer) error {
|
||||
return status.Errorf(codes.Unimplemented, "method SubscribeToWorkflowEvents not implemented")
|
||||
}
|
||||
func (UnimplementedDispatcherServer) SubscribeToWorkflowRuns(Dispatcher_SubscribeToWorkflowRunsServer) error {
|
||||
return status.Errorf(codes.Unimplemented, "method SubscribeToWorkflowRuns not implemented")
|
||||
}
|
||||
func (UnimplementedDispatcherServer) SendStepActionEvent(context.Context, *StepActionEvent) (*ActionEventResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method SendStepActionEvent not implemented")
|
||||
}
|
||||
@@ -356,6 +392,32 @@ func (x *dispatcherSubscribeToWorkflowEventsServer) Send(m *WorkflowEvent) error
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func _Dispatcher_SubscribeToWorkflowRuns_Handler(srv interface{}, stream grpc.ServerStream) error {
|
||||
return srv.(DispatcherServer).SubscribeToWorkflowRuns(&dispatcherSubscribeToWorkflowRunsServer{stream})
|
||||
}
|
||||
|
||||
type Dispatcher_SubscribeToWorkflowRunsServer interface {
|
||||
Send(*WorkflowRunEvent) error
|
||||
Recv() (*SubscribeToWorkflowRunsRequest, error)
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
type dispatcherSubscribeToWorkflowRunsServer struct {
|
||||
grpc.ServerStream
|
||||
}
|
||||
|
||||
func (x *dispatcherSubscribeToWorkflowRunsServer) Send(m *WorkflowRunEvent) error {
|
||||
return x.ServerStream.SendMsg(m)
|
||||
}
|
||||
|
||||
func (x *dispatcherSubscribeToWorkflowRunsServer) Recv() (*SubscribeToWorkflowRunsRequest, error) {
|
||||
m := new(SubscribeToWorkflowRunsRequest)
|
||||
if err := x.ServerStream.RecvMsg(m); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return m, nil
|
||||
}
|
||||
|
||||
func _Dispatcher_SendStepActionEvent_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(StepActionEvent)
|
||||
if err := dec(in); err != nil {
|
||||
@@ -476,6 +538,12 @@ var Dispatcher_ServiceDesc = grpc.ServiceDesc{
|
||||
Handler: _Dispatcher_SubscribeToWorkflowEvents_Handler,
|
||||
ServerStreams: true,
|
||||
},
|
||||
{
|
||||
StreamName: "SubscribeToWorkflowRuns",
|
||||
Handler: _Dispatcher_SubscribeToWorkflowRuns_Handler,
|
||||
ServerStreams: true,
|
||||
ClientStreams: true,
|
||||
},
|
||||
},
|
||||
Metadata: "dispatcher.proto",
|
||||
}
|
||||
|
||||
@@ -142,17 +142,18 @@ func New(fs ...DispatcherOpt) (*DispatcherImpl, error) {
|
||||
}
|
||||
|
||||
func (d *DispatcherImpl) Start() (func() error, error) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
// register the dispatcher by creating a new dispatcher in the database
|
||||
dispatcher, err := d.repo.Dispatcher().CreateNewDispatcher(&repository.CreateDispatcherOpts{
|
||||
dispatcher, err := d.repo.Dispatcher().CreateNewDispatcher(ctx, &repository.CreateDispatcherOpts{
|
||||
ID: d.dispatcherId,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
cancel()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
_, err = d.s.NewJob(
|
||||
gocron.DurationJob(time.Second*5),
|
||||
gocron.NewTask(
|
||||
@@ -212,17 +213,20 @@ func (d *DispatcherImpl) Start() (func() error, error) {
|
||||
return true
|
||||
})
|
||||
|
||||
err = d.repo.Dispatcher().Delete(dispatcherId)
|
||||
if err := d.s.Shutdown(); err != nil {
|
||||
return fmt.Errorf("could not shutdown scheduler: %w", err)
|
||||
}
|
||||
|
||||
deleteCtx, deleteCancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer deleteCancel()
|
||||
|
||||
err = d.repo.Dispatcher().Delete(deleteCtx, dispatcherId)
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not delete dispatcher: %w", err)
|
||||
}
|
||||
|
||||
d.l.Debug().Msgf("deleted dispatcher %s", dispatcherId)
|
||||
|
||||
if err := d.s.Shutdown(); err != nil {
|
||||
return fmt.Errorf("could not shutdown scheduler: %w", err)
|
||||
}
|
||||
|
||||
d.l.Debug().Msgf("dispatcher has shutdown")
|
||||
return nil
|
||||
}
|
||||
@@ -278,7 +282,7 @@ func (d *DispatcherImpl) handleGroupKeyActionAssignedTask(ctx context.Context, t
|
||||
}
|
||||
|
||||
// load the workflow run from the database
|
||||
workflowRun, err := d.repo.WorkflowRun().GetWorkflowRunById(metadata.TenantId, payload.WorkflowRunId)
|
||||
workflowRun, err := d.repo.WorkflowRun().GetWorkflowRunById(ctx, metadata.TenantId, payload.WorkflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get workflow run: %w", err)
|
||||
@@ -292,7 +296,7 @@ func (d *DispatcherImpl) handleGroupKeyActionAssignedTask(ctx context.Context, t
|
||||
return fmt.Errorf("could not get group key run")
|
||||
}
|
||||
|
||||
sqlcGroupKeyRun, err := d.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(metadata.TenantId, groupKeyRunId)
|
||||
sqlcGroupKeyRun, err := d.repo.GetGroupKeyRun().GetGroupKeyRunForEngine(ctx, metadata.TenantId, groupKeyRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get group key run for engine: %w", err)
|
||||
@@ -334,7 +338,7 @@ func (d *DispatcherImpl) handleStepRunAssignedTask(ctx context.Context, task *ms
|
||||
}
|
||||
|
||||
// load the step run from the database
|
||||
stepRun, err := d.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
|
||||
stepRun, err := d.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get step run: %w", err)
|
||||
@@ -378,7 +382,7 @@ func (d *DispatcherImpl) handleStepRunCancelled(ctx context.Context, task *msgqu
|
||||
}
|
||||
|
||||
// load the step run from the database
|
||||
stepRun, err := d.repo.StepRun().GetStepRunForEngine(metadata.TenantId, payload.StepRunId)
|
||||
stepRun, err := d.repo.StepRun().GetStepRunForEngine(ctx, metadata.TenantId, payload.StepRunId)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("could not get step run: %w", err)
|
||||
@@ -402,7 +406,7 @@ func (d *DispatcherImpl) runUpdateHeartbeat(ctx context.Context) func() {
|
||||
now := time.Now().UTC()
|
||||
|
||||
// update the heartbeat
|
||||
_, err := d.repo.Dispatcher().UpdateDispatcher(d.dispatcherId, &repository.UpdateDispatcherOpts{
|
||||
_, err := d.repo.Dispatcher().UpdateDispatcher(ctx, d.dispatcherId, &repository.UpdateDispatcherOpts{
|
||||
LastHeartbeatAt: &now,
|
||||
})
|
||||
|
||||
|
||||
@@ -2,12 +2,17 @@ package dispatcher
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/rs/zerolog"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
"google.golang.org/protobuf/types/known/timestamppb"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/datautils"
|
||||
"github.com/hatchet-dev/hatchet/internal/msgqueue"
|
||||
@@ -144,7 +149,7 @@ func (s *DispatcherImpl) Register(ctx context.Context, request *contracts.Worker
|
||||
}
|
||||
|
||||
// create a worker in the database
|
||||
worker, err := s.repo.Worker().CreateNewWorker(tenantId, opts)
|
||||
worker, err := s.repo.Worker().CreateNewWorker(ctx, tenantId, opts)
|
||||
|
||||
if err != nil {
|
||||
s.l.Error().Err(err).Msgf("could not create worker for tenant %s", tenantId)
|
||||
@@ -170,7 +175,9 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
|
||||
|
||||
s.l.Debug().Msgf("Received subscribe request from ID: %s", request.WorkerId)
|
||||
|
||||
worker, err := s.repo.Worker().GetWorkerForEngine(tenantId, request.WorkerId)
|
||||
ctx := stream.Context()
|
||||
|
||||
worker, err := s.repo.Worker().GetWorkerForEngine(ctx, tenantId, request.WorkerId)
|
||||
|
||||
if err != nil {
|
||||
s.l.Error().Err(err).Msgf("could not get worker %s", request.WorkerId)
|
||||
@@ -179,7 +186,7 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
|
||||
|
||||
// check the worker's dispatcher against the current dispatcher. if they don't match, then update the worker
|
||||
if worker.DispatcherId.Valid && sqlchelpers.UUIDToStr(worker.DispatcherId) != s.dispatcherId {
|
||||
_, err = s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
_, err = s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
DispatcherId: &s.dispatcherId,
|
||||
})
|
||||
|
||||
@@ -203,8 +210,6 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
|
||||
s.workers.Delete(request.WorkerId)
|
||||
}()
|
||||
|
||||
ctx := stream.Context()
|
||||
|
||||
// update the worker with a last heartbeat time every 5 seconds as long as the worker is connected
|
||||
go func() {
|
||||
timer := time.NewTicker(100 * time.Millisecond)
|
||||
@@ -225,7 +230,7 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
|
||||
if now := time.Now().UTC(); lastHeartbeat.Add(4 * time.Second).Before(now) {
|
||||
s.l.Debug().Msgf("updating worker %s heartbeat", request.WorkerId)
|
||||
|
||||
_, err := s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
LastHeartbeatAt: &now,
|
||||
})
|
||||
|
||||
@@ -259,9 +264,11 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
|
||||
tenant := stream.Context().Value("tenant").(*dbsqlc.Tenant)
|
||||
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
|
||||
|
||||
ctx := stream.Context()
|
||||
|
||||
s.l.Debug().Msgf("Received subscribe request from ID: %s", request.WorkerId)
|
||||
|
||||
worker, err := s.repo.Worker().GetWorkerForEngine(tenantId, request.WorkerId)
|
||||
worker, err := s.repo.Worker().GetWorkerForEngine(ctx, tenantId, request.WorkerId)
|
||||
|
||||
if err != nil {
|
||||
s.l.Error().Err(err).Msgf("could not get worker %s", request.WorkerId)
|
||||
@@ -270,7 +277,7 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
|
||||
|
||||
// check the worker's dispatcher against the current dispatcher. if they don't match, then update the worker
|
||||
if worker.DispatcherId.Valid && sqlchelpers.UUIDToStr(worker.DispatcherId) != s.dispatcherId {
|
||||
_, err = s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
_, err = s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
DispatcherId: &s.dispatcherId,
|
||||
})
|
||||
|
||||
@@ -295,7 +302,7 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
|
||||
|
||||
inactive := db.WorkerStatusInactive
|
||||
|
||||
_, err := s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
Status: &inactive,
|
||||
})
|
||||
|
||||
@@ -304,8 +311,6 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
|
||||
}
|
||||
}()
|
||||
|
||||
ctx := stream.Context()
|
||||
|
||||
// Keep the connection alive for sending messages
|
||||
for {
|
||||
select {
|
||||
@@ -335,7 +340,7 @@ func (s *DispatcherImpl) Heartbeat(ctx context.Context, req *contracts.Heartbeat
|
||||
s.l.Warn().Msgf("heartbeat time is greater than expected heartbeat interval")
|
||||
}
|
||||
|
||||
_, err := s.repo.Worker().UpdateWorker(tenantId, req.WorkerId, &repository.UpdateWorkerOpts{
|
||||
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, req.WorkerId, &repository.UpdateWorkerOpts{
|
||||
// use the system time for heartbeat
|
||||
LastHeartbeatAt: &heartbeatAt,
|
||||
})
|
||||
@@ -411,6 +416,219 @@ func (s *DispatcherImpl) SubscribeToWorkflowEvents(request *contracts.SubscribeT
|
||||
return nil
|
||||
}
|
||||
|
||||
// map of workflow run ids to whether the workflow runs are finished and have sent a message
|
||||
// that the workflow run is finished
|
||||
type workflowRunAcks struct {
|
||||
acks map[string]bool
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
func (w *workflowRunAcks) addWorkflowRun(id string) {
|
||||
w.mu.Lock()
|
||||
defer w.mu.Unlock()
|
||||
|
||||
w.acks[id] = false
|
||||
}
|
||||
|
||||
func (w *workflowRunAcks) getNonAckdWorkflowRuns() []string {
|
||||
w.mu.RLock()
|
||||
defer w.mu.RUnlock()
|
||||
|
||||
ids := make([]string, 0, len(w.acks))
|
||||
|
||||
for id := range w.acks {
|
||||
if !w.acks[id] {
|
||||
ids = append(ids, id)
|
||||
}
|
||||
}
|
||||
|
||||
return ids
|
||||
}
|
||||
|
||||
func (w *workflowRunAcks) ackWorkflowRun(id string) {
|
||||
w.mu.Lock()
|
||||
defer w.mu.Unlock()
|
||||
|
||||
w.acks[id] = true
|
||||
}
|
||||
|
||||
type sendTimeFilter struct {
|
||||
mu sync.Mutex
|
||||
}
|
||||
|
||||
func (s *sendTimeFilter) canSend() bool {
|
||||
if !s.mu.TryLock() {
|
||||
return false
|
||||
}
|
||||
|
||||
go func() {
|
||||
time.Sleep(time.Second - 10*time.Millisecond)
|
||||
s.mu.Unlock()
|
||||
}()
|
||||
|
||||
return true
|
||||
}
|
||||
|
||||
// SubscribeToWorkflowEvents registers workflow events with the dispatcher
|
||||
func (s *DispatcherImpl) SubscribeToWorkflowRuns(server contracts.Dispatcher_SubscribeToWorkflowRunsServer) error {
|
||||
tenant := server.Context().Value("tenant").(*dbsqlc.Tenant)
|
||||
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
|
||||
|
||||
s.l.Debug().Msgf("Received subscribe request for tenant: %s", tenantId)
|
||||
|
||||
acks := &workflowRunAcks{
|
||||
acks: make(map[string]bool),
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(server.Context())
|
||||
defer cancel()
|
||||
|
||||
// subscribe to the task queue for the tenant
|
||||
q, err := msgqueue.TenantEventConsumerQueue(tenantId)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
wg := sync.WaitGroup{}
|
||||
|
||||
sendEvent := func(e *contracts.WorkflowRunEvent) error {
|
||||
// send the task to the client
|
||||
err := server.Send(e)
|
||||
|
||||
if err != nil {
|
||||
cancel() // FIXME is this necessary?
|
||||
s.l.Error().Err(err).Msgf("could not send workflow event to client")
|
||||
return err
|
||||
}
|
||||
|
||||
acks.ackWorkflowRun(e.WorkflowRunId)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
immediateSendFilter := &sendTimeFilter{}
|
||||
iterSendFilter := &sendTimeFilter{}
|
||||
|
||||
iter := func(workflowRunIds []string) error {
|
||||
limit := 1000
|
||||
|
||||
workflowRuns, err := s.repo.WorkflowRun().ListWorkflowRuns(ctx, tenantId, &repository.ListWorkflowRunsOpts{
|
||||
Ids: workflowRunIds,
|
||||
Limit: &limit,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
s.l.Error().Err(err).Msg("could not get workflow runs")
|
||||
return nil
|
||||
}
|
||||
|
||||
events, err := s.toWorkflowRunEvent(tenantId, workflowRuns.Rows)
|
||||
|
||||
if err != nil {
|
||||
s.l.Error().Err(err).Msg("could not convert workflow run to event")
|
||||
return nil
|
||||
} else if events == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
for _, event := range events {
|
||||
err := sendEvent(event)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// start a new goroutine to handle client-side streaming
|
||||
go func() {
|
||||
for {
|
||||
req, err := server.Recv()
|
||||
|
||||
if err != nil {
|
||||
cancel()
|
||||
if errors.Is(err, io.EOF) || status.Code(err) == codes.Canceled {
|
||||
return
|
||||
}
|
||||
|
||||
s.l.Error().Err(err).Msg("could not receive message from client")
|
||||
return
|
||||
}
|
||||
|
||||
acks.addWorkflowRun(req.WorkflowRunId)
|
||||
|
||||
if immediateSendFilter.canSend() {
|
||||
if err := iter([]string{req.WorkflowRunId}); err != nil {
|
||||
s.l.Error().Err(err).Msg("could not iterate over workflow runs")
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
f := func(task *msgqueue.Message) error {
|
||||
wg.Add(1)
|
||||
defer wg.Done()
|
||||
|
||||
workflowRunIds := acks.getNonAckdWorkflowRuns()
|
||||
|
||||
if matchedWorkflowRunId, ok := s.isMatchingWorkflowRun(task, workflowRunIds...); ok {
|
||||
if immediateSendFilter.canSend() {
|
||||
if err := iter([]string{matchedWorkflowRunId}); err != nil {
|
||||
s.l.Error().Err(err).Msg("could not iterate over workflow runs")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// subscribe to the task queue for the tenant
|
||||
cleanupQueue, err := s.mq.Subscribe(q, msgqueue.NoOpHook, f)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// new goroutine to poll every second for finished workflow runs which are not ackd
|
||||
go func() {
|
||||
ticker := time.NewTicker(1 * time.Second)
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
if !iterSendFilter.canSend() {
|
||||
continue
|
||||
}
|
||||
|
||||
workflowRunIds := acks.getNonAckdWorkflowRuns()
|
||||
|
||||
if len(workflowRunIds) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
if err := iter(workflowRunIds); err != nil {
|
||||
s.l.Error().Err(err).Msg("could not iterate over workflow runs")
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
<-ctx.Done()
|
||||
|
||||
if err := cleanupQueue(); err != nil {
|
||||
return fmt.Errorf("could not cleanup queue: %w", err)
|
||||
}
|
||||
|
||||
waitFor(&wg, 60*time.Second, s.l)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func waitFor(wg *sync.WaitGroup, timeout time.Duration, l *zerolog.Logger) {
|
||||
done := make(chan struct{})
|
||||
|
||||
@@ -470,7 +688,7 @@ func (s *DispatcherImpl) PutOverridesData(ctx context.Context, request *contract
|
||||
opts.CallerFile = &request.CallerFilename
|
||||
}
|
||||
|
||||
_, err := s.repo.StepRun().UpdateStepRunOverridesData(tenantId, request.StepRunId, opts)
|
||||
_, err := s.repo.StepRun().UpdateStepRunOverridesData(ctx, tenantId, request.StepRunId, opts)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -488,7 +706,7 @@ func (s *DispatcherImpl) Unsubscribe(ctx context.Context, request *contracts.Wor
|
||||
|
||||
inactive := db.WorkerStatusInactive
|
||||
|
||||
_, err := s.repo.Worker().UpdateWorker(tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
_, err := s.repo.Worker().UpdateWorker(ctx, tenantId, request.WorkerId, &repository.UpdateWorkerOpts{
|
||||
Status: &inactive,
|
||||
})
|
||||
|
||||
@@ -716,7 +934,7 @@ func (s *DispatcherImpl) handleGetGroupKeyRunFailed(ctx context.Context, request
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenantId, workflowRunId string) (*contracts.WorkflowEvent, error) {
|
||||
func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenantId string, workflowRunIds ...string) (*contracts.WorkflowEvent, error) {
|
||||
workflowEvent := &contracts.WorkflowEvent{}
|
||||
|
||||
var stepRunId string
|
||||
@@ -764,13 +982,13 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
|
||||
|
||||
if workflowEvent.ResourceType == contracts.ResourceType_RESOURCE_TYPE_STEP_RUN {
|
||||
// determine if this step run matches the workflow run id
|
||||
stepRun, err := s.repo.StepRun().GetStepRunForEngine(tenantId, stepRunId)
|
||||
stepRun, err := s.repo.StepRun().GetStepRunForEngine(context.Background(), tenantId, stepRunId)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if sqlchelpers.UUIDToStr(stepRun.WorkflowRunId) != workflowRunId {
|
||||
if !contains(workflowRunIds, sqlchelpers.UUIDToStr(stepRun.WorkflowRunId)) {
|
||||
// this is an expected error, so we don't return it
|
||||
return nil, nil
|
||||
}
|
||||
@@ -784,7 +1002,7 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
|
||||
return nil, err
|
||||
}
|
||||
|
||||
streamEvent, err := s.repo.StreamEvent().GetStreamEvent(tenantId, streamEventId)
|
||||
streamEvent, err := s.repo.StreamEvent().GetStreamEvent(context.Background(), tenantId, streamEventId)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -794,7 +1012,7 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
|
||||
}
|
||||
|
||||
} else if workflowEvent.ResourceType == contracts.ResourceType_RESOURCE_TYPE_WORKFLOW_RUN {
|
||||
if workflowEvent.ResourceId != workflowRunId {
|
||||
if !contains(workflowRunIds, workflowEvent.ResourceId) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
@@ -803,3 +1021,104 @@ func (s *DispatcherImpl) tenantTaskToWorkflowEvent(task *msgqueue.Message, tenan
|
||||
|
||||
return workflowEvent, nil
|
||||
}
|
||||
|
||||
func (s *DispatcherImpl) isMatchingWorkflowRun(task *msgqueue.Message, workflowRunIds ...string) (string, bool) {
|
||||
if task.ID != "workflow-run-finished" {
|
||||
return "", false
|
||||
}
|
||||
|
||||
workflowRunId := task.Payload["workflow_run_id"].(string)
|
||||
|
||||
if contains(workflowRunIds, workflowRunId) {
|
||||
return workflowRunId, true
|
||||
}
|
||||
|
||||
return "", false
|
||||
}
|
||||
|
||||
func (s *DispatcherImpl) toWorkflowRunEvent(tenantId string, workflowRuns []*dbsqlc.ListWorkflowRunsRow) ([]*contracts.WorkflowRunEvent, error) {
|
||||
workflowRunIds := make([]string, 0)
|
||||
|
||||
for _, workflowRun := range workflowRuns {
|
||||
if workflowRun.WorkflowRun.Status != dbsqlc.WorkflowRunStatusFAILED && workflowRun.WorkflowRun.Status != dbsqlc.WorkflowRunStatusSUCCEEDED {
|
||||
continue
|
||||
}
|
||||
|
||||
workflowRunId := sqlchelpers.UUIDToStr(workflowRun.WorkflowRun.ID)
|
||||
|
||||
workflowRunIds = append(workflowRunIds, workflowRunId)
|
||||
}
|
||||
|
||||
res := make([]*contracts.WorkflowRunEvent, 0)
|
||||
|
||||
// get step run results for each workflow run
|
||||
mappedStepRunResults, err := s.getStepResultsForWorkflowRun(tenantId, workflowRunIds)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for workflowRunId, stepRunResults := range mappedStepRunResults {
|
||||
res = append(res, &contracts.WorkflowRunEvent{
|
||||
WorkflowRunId: workflowRunId,
|
||||
EventType: contracts.WorkflowRunEventType_WORKFLOW_RUN_EVENT_TYPE_FINISHED,
|
||||
EventTimestamp: timestamppb.Now(),
|
||||
Results: stepRunResults,
|
||||
})
|
||||
}
|
||||
|
||||
return res, nil
|
||||
}
|
||||
|
||||
func (s *DispatcherImpl) getStepResultsForWorkflowRun(tenantId string, workflowRunIds []string) (map[string][]*contracts.StepRunResult, error) {
|
||||
stepRuns, err := s.repo.StepRun().ListStepRuns(context.Background(), tenantId, &repository.ListStepRunsOpts{
|
||||
WorkflowRunIds: workflowRunIds,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
res := make(map[string][]*contracts.StepRunResult)
|
||||
|
||||
for _, stepRun := range stepRuns {
|
||||
resStepRun := &contracts.StepRunResult{
|
||||
StepRunId: sqlchelpers.UUIDToStr(stepRun.StepRun.ID),
|
||||
StepReadableId: stepRun.StepReadableId.String,
|
||||
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
|
||||
}
|
||||
|
||||
if stepRun.StepRun.Error.Valid {
|
||||
resStepRun.Error = &stepRun.StepRun.Error.String
|
||||
}
|
||||
|
||||
if stepRun.StepRun.CancelledReason.Valid {
|
||||
errString := fmt.Sprintf("this step run was cancelled due to %s", stepRun.StepRun.CancelledReason.String)
|
||||
resStepRun.Error = &errString
|
||||
}
|
||||
|
||||
if stepRun.StepRun.Output != nil {
|
||||
resStepRun.Output = repository.StringPtr(string(stepRun.StepRun.Output))
|
||||
}
|
||||
|
||||
workflowRunId := sqlchelpers.UUIDToStr(stepRun.WorkflowRunId)
|
||||
|
||||
if currResults, ok := res[workflowRunId]; ok {
|
||||
res[workflowRunId] = append(currResults, resStepRun)
|
||||
} else {
|
||||
res[workflowRunId] = []*contracts.StepRunResult{resStepRun}
|
||||
}
|
||||
}
|
||||
|
||||
return res, nil
|
||||
}
|
||||
|
||||
func contains(s []string, e string) bool {
|
||||
for _, a := range s {
|
||||
if a == e {
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
@@ -33,7 +33,7 @@ func (a *GRPCAuthN) Middleware(ctx context.Context) (context.Context, error) {
|
||||
return nil, forbidden
|
||||
}
|
||||
|
||||
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(token)
|
||||
tenantId, err := a.config.Auth.JWTManager.ValidateTenantToken(ctx, token)
|
||||
|
||||
if err != nil {
|
||||
a.l.Debug().Err(err).Msgf("error validating tenant token: %s", err)
|
||||
@@ -42,7 +42,7 @@ func (a *GRPCAuthN) Middleware(ctx context.Context) (context.Context, error) {
|
||||
}
|
||||
|
||||
// get the tenant id
|
||||
queriedTenant, err := a.config.EngineRepository.Tenant().GetTenantByID(tenantId)
|
||||
queriedTenant, err := a.config.EngineRepository.Tenant().GetTenantByID(ctx, tenantId)
|
||||
|
||||
if err != nil {
|
||||
a.l.Debug().Err(err).Msgf("error getting tenant by id: %s", err)
|
||||
|
||||
@@ -41,7 +41,7 @@ func (i *IngestorImpl) ReplaySingleEvent(ctx context.Context, req *contracts.Rep
|
||||
|
||||
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
|
||||
|
||||
oldEvent, err := i.eventRepository.GetEventForEngine(tenantId, req.EventId)
|
||||
oldEvent, err := i.eventRepository.GetEventForEngine(ctx, tenantId, req.EventId)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@@ -79,7 +79,7 @@ func (i *IngestorImpl) PutStreamEvent(ctx context.Context, req *contracts.PutStr
|
||||
metadata = []byte(req.Metadata)
|
||||
}
|
||||
|
||||
streamEvent, err := i.streamEventRepository.PutStreamEvent(tenantId, &repository.CreateStreamEventOpts{
|
||||
streamEvent, err := i.streamEventRepository.PutStreamEvent(ctx, tenantId, &repository.CreateStreamEventOpts{
|
||||
StepRunId: req.StepRunId,
|
||||
CreatedAt: createdAt,
|
||||
Message: req.Message,
|
||||
@@ -122,7 +122,7 @@ func (i *IngestorImpl) PutLog(ctx context.Context, req *contracts.PutLogRequest)
|
||||
metadata = []byte(req.Metadata)
|
||||
}
|
||||
|
||||
_, err := i.logRepository.PutLog(tenantId, &repository.CreateLogLineOpts{
|
||||
_, err := i.logRepository.PutLog(ctx, tenantId, &repository.CreateLogLineOpts{
|
||||
StepRunId: req.StepRunId,
|
||||
CreatedAt: createdAt,
|
||||
Message: req.Message,
|
||||
|
||||
@@ -16,9 +16,12 @@ import (
|
||||
|
||||
func (t *TickerImpl) runPollCronSchedules(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: polling cron schedules")
|
||||
|
||||
crons, err := t.repo.Ticker().PollCronSchedules(t.tickerId)
|
||||
crons, err := t.repo.Ticker().PollCronSchedules(ctx, t.tickerId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not poll cron schedules")
|
||||
@@ -98,9 +101,12 @@ func (t *TickerImpl) handleScheduleCron(ctx context.Context, cron *dbsqlc.PollCr
|
||||
|
||||
func (t *TickerImpl) runCronWorkflow(ctx context.Context, tenantId, workflowVersionId, cron, cronParentId string, input []byte) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: running workflow %s", workflowVersionId)
|
||||
|
||||
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(tenantId, workflowVersionId)
|
||||
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(ctx, tenantId, workflowVersionId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not get workflow version")
|
||||
@@ -122,7 +128,7 @@ func (t *TickerImpl) runCronWorkflow(ctx context.Context, tenantId, workflowVers
|
||||
return
|
||||
}
|
||||
|
||||
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(tenantId, workflowRunId)
|
||||
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(ctx, tenantId, workflowRunId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not list job runs for workflow run")
|
||||
|
||||
@@ -2,6 +2,7 @@ package ticker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/datautils"
|
||||
"github.com/hatchet-dev/hatchet/internal/msgqueue"
|
||||
@@ -11,9 +12,12 @@ import (
|
||||
|
||||
func (t *TickerImpl) runPollGetGroupKeyRuns(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: polling get group key runs")
|
||||
|
||||
getGroupKeyRuns, err := t.repo.Ticker().PollGetGroupKeyRuns(t.tickerId)
|
||||
getGroupKeyRuns, err := t.repo.Ticker().PollGetGroupKeyRuns(ctx, t.tickerId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not poll get group key runs")
|
||||
|
||||
@@ -16,9 +16,12 @@ import (
|
||||
|
||||
func (t *TickerImpl) runPollSchedules(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: polling workflow schedules")
|
||||
|
||||
scheduledWorkflows, err := t.repo.Ticker().PollScheduledWorkflows(t.tickerId)
|
||||
scheduledWorkflows, err := t.repo.Ticker().PollScheduledWorkflows(ctx, t.tickerId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not poll workflow schedules")
|
||||
@@ -111,9 +114,12 @@ func (t *TickerImpl) handleScheduleWorkflow(ctx context.Context, scheduledWorkfl
|
||||
|
||||
func (t *TickerImpl) runScheduledWorkflow(ctx context.Context, tenantId, workflowVersionId, scheduledWorkflowId string, scheduled *dbsqlc.PollScheduledWorkflowsRow) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: running workflow %s", workflowVersionId)
|
||||
|
||||
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(tenantId, workflowVersionId)
|
||||
workflowVersion, err := t.repo.Workflow().GetWorkflowVersionById(ctx, tenantId, workflowVersionId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not get workflow version")
|
||||
@@ -157,7 +163,7 @@ func (t *TickerImpl) runScheduledWorkflow(ctx context.Context, tenantId, workflo
|
||||
return
|
||||
}
|
||||
|
||||
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(tenantId, workflowRunId)
|
||||
jobRuns, err := t.repo.JobRun().ListJobRunsForWorkflowRun(ctx, tenantId, workflowRunId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not list job runs for workflow run")
|
||||
|
||||
@@ -2,6 +2,7 @@ package ticker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/internal/datautils"
|
||||
"github.com/hatchet-dev/hatchet/internal/msgqueue"
|
||||
@@ -11,9 +12,12 @@ import (
|
||||
|
||||
func (t *TickerImpl) runPollStepRuns(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: polling step runs")
|
||||
|
||||
stepRuns, err := t.repo.Ticker().PollStepRuns(t.tickerId)
|
||||
stepRuns, err := t.repo.Ticker().PollStepRuns(ctx, t.tickerId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not poll step runs")
|
||||
|
||||
@@ -112,7 +112,7 @@ func (t *TickerImpl) Start() (func() error, error) {
|
||||
t.l.Debug().Msgf("starting ticker %s", t.tickerId)
|
||||
|
||||
// register the ticker
|
||||
_, err := t.repo.Ticker().CreateNewTicker(&repository.CreateTickerOpts{
|
||||
_, err := t.repo.Ticker().CreateNewTicker(ctx, &repository.CreateTickerOpts{
|
||||
ID: t.tickerId,
|
||||
})
|
||||
|
||||
@@ -186,7 +186,7 @@ func (t *TickerImpl) Start() (func() error, error) {
|
||||
_, err = t.s.NewJob(
|
||||
gocron.DurationJob(time.Minute*5),
|
||||
gocron.NewTask(
|
||||
t.runStreamEventCleanup(),
|
||||
t.runStreamEventCleanup(ctx),
|
||||
),
|
||||
)
|
||||
|
||||
@@ -202,18 +202,21 @@ func (t *TickerImpl) Start() (func() error, error) {
|
||||
|
||||
cancel()
|
||||
|
||||
if err := t.s.Shutdown(); err != nil {
|
||||
return fmt.Errorf("could not shutdown scheduler: %w", err)
|
||||
}
|
||||
|
||||
deleteCtx, deleteCancel := context.WithTimeout(context.Background(), 5*time.Second)
|
||||
defer deleteCancel()
|
||||
|
||||
// delete the ticker
|
||||
err = t.repo.Ticker().Delete(t.tickerId)
|
||||
err = t.repo.Ticker().Delete(deleteCtx, t.tickerId)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not delete ticker")
|
||||
return err
|
||||
}
|
||||
|
||||
if err := t.s.Shutdown(); err != nil {
|
||||
return fmt.Errorf("could not shutdown scheduler: %w", err)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -227,7 +230,7 @@ func (t *TickerImpl) runUpdateHeartbeat(ctx context.Context) func() {
|
||||
now := time.Now().UTC()
|
||||
|
||||
// update the heartbeat
|
||||
_, err := t.repo.Ticker().UpdateTicker(t.tickerId, &repository.UpdateTickerOpts{
|
||||
_, err := t.repo.Ticker().UpdateTicker(ctx, t.tickerId, &repository.UpdateTickerOpts{
|
||||
LastHeartbeatAt: &now,
|
||||
})
|
||||
|
||||
@@ -237,11 +240,14 @@ func (t *TickerImpl) runUpdateHeartbeat(ctx context.Context) func() {
|
||||
}
|
||||
}
|
||||
|
||||
func (t *TickerImpl) runStreamEventCleanup() func() {
|
||||
func (t *TickerImpl) runStreamEventCleanup(ctx context.Context) func() {
|
||||
return func() {
|
||||
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
|
||||
defer cancel()
|
||||
|
||||
t.l.Debug().Msgf("ticker: cleaning up stream event")
|
||||
|
||||
err := t.repo.StreamEvent().CleanupStreamEvents()
|
||||
err := t.repo.StreamEvent().CleanupStreamEvents(ctx)
|
||||
|
||||
if err != nil {
|
||||
t.l.Err(err).Msg("could not cleanup stream events")
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package testutils
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"os"
|
||||
"path"
|
||||
@@ -72,7 +73,7 @@ func Prepare(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(tenantId, "default")
|
||||
defaultTok, err := serverConf.Auth.JWTManager.GenerateTenantToken(context.Background(), tenantId, "default")
|
||||
if err != nil {
|
||||
t.Fatalf("could not generate default token: %v", err)
|
||||
}
|
||||
|
||||
@@ -3,32 +3,236 @@ package client
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/rs/zerolog"
|
||||
"golang.org/x/sync/errgroup"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/codes"
|
||||
"google.golang.org/grpc/status"
|
||||
|
||||
dispatchercontracts "github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
|
||||
"github.com/hatchet-dev/hatchet/internal/validator"
|
||||
)
|
||||
|
||||
type RunEvent *dispatchercontracts.WorkflowEvent
|
||||
type WorkflowEvent *dispatchercontracts.WorkflowEvent
|
||||
type WorkflowRunEvent *dispatchercontracts.WorkflowRunEvent
|
||||
|
||||
type StreamEvent struct {
|
||||
Message []byte
|
||||
}
|
||||
|
||||
type RunHandler func(event RunEvent) error
|
||||
type RunHandler func(event WorkflowEvent) error
|
||||
type StreamHandler func(event StreamEvent) error
|
||||
type WorkflowRunEventHandler func(event WorkflowRunEvent) error
|
||||
|
||||
type WorkflowRunsListener struct {
|
||||
constructor func(context.Context) (dispatchercontracts.Dispatcher_SubscribeToWorkflowRunsClient, error)
|
||||
|
||||
client dispatchercontracts.Dispatcher_SubscribeToWorkflowRunsClient
|
||||
clientMu sync.RWMutex
|
||||
|
||||
l *zerolog.Logger
|
||||
|
||||
// map of workflow run ids to a list of handlers
|
||||
handlers sync.Map
|
||||
}
|
||||
|
||||
func (r *subscribeClientImpl) newWorkflowRunsListener(
|
||||
ctx context.Context,
|
||||
) (*WorkflowRunsListener, error) {
|
||||
constructor := func(ctx context.Context) (dispatchercontracts.Dispatcher_SubscribeToWorkflowRunsClient, error) {
|
||||
return r.client.SubscribeToWorkflowRuns(r.ctx.newContext(ctx))
|
||||
}
|
||||
|
||||
w := &WorkflowRunsListener{
|
||||
constructor: constructor,
|
||||
l: r.l,
|
||||
}
|
||||
|
||||
err := w.retrySubscribe(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return w, nil
|
||||
}
|
||||
|
||||
func (w *WorkflowRunsListener) retrySubscribe(ctx context.Context) error {
|
||||
w.clientMu.Lock()
|
||||
defer w.clientMu.Unlock()
|
||||
|
||||
retries := 0
|
||||
|
||||
for retries < DefaultActionListenerRetryCount {
|
||||
if retries > 0 {
|
||||
time.Sleep(DefaultActionListenerRetryInterval)
|
||||
}
|
||||
|
||||
client, err := w.constructor(ctx)
|
||||
|
||||
if err != nil {
|
||||
retries++
|
||||
w.l.Error().Err(err).Msgf("could not resubscribe to the listener")
|
||||
continue
|
||||
}
|
||||
|
||||
w.client = client
|
||||
|
||||
// listen for all the same workflow runs
|
||||
var rangeErr error
|
||||
|
||||
w.handlers.Range(func(key, value interface{}) bool {
|
||||
workflowRunId := key.(string)
|
||||
|
||||
err := w.client.Send(&dispatchercontracts.SubscribeToWorkflowRunsRequest{
|
||||
WorkflowRunId: workflowRunId,
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
w.l.Error().Err(err).Msgf("could not subscribe to the worker")
|
||||
rangeErr = err
|
||||
return false
|
||||
}
|
||||
|
||||
return true
|
||||
})
|
||||
|
||||
if rangeErr != nil {
|
||||
continue
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
return fmt.Errorf("could not subscribe to the worker after %d retries", retries)
|
||||
}
|
||||
|
||||
type threadSafeHandlers struct {
|
||||
handlers []WorkflowRunEventHandler
|
||||
mu sync.RWMutex
|
||||
}
|
||||
|
||||
func (l *WorkflowRunsListener) AddWorkflowRun(
|
||||
workflowRunId string,
|
||||
handler WorkflowRunEventHandler,
|
||||
) error {
|
||||
handlers, _ := l.handlers.LoadOrStore(workflowRunId, &threadSafeHandlers{
|
||||
handlers: []WorkflowRunEventHandler{},
|
||||
})
|
||||
|
||||
h := handlers.(*threadSafeHandlers)
|
||||
|
||||
h.mu.Lock()
|
||||
h.handlers = append(h.handlers, handler)
|
||||
l.handlers.Store(workflowRunId, h)
|
||||
h.mu.Unlock()
|
||||
|
||||
err := l.retrySend(workflowRunId)
|
||||
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *WorkflowRunsListener) retrySend(workflowRunId string) error {
|
||||
l.clientMu.RLock()
|
||||
defer l.clientMu.RUnlock()
|
||||
|
||||
if l.client == nil {
|
||||
return fmt.Errorf("client is not connected")
|
||||
}
|
||||
|
||||
for i := 0; i < DefaultActionListenerRetryCount; i++ {
|
||||
err := l.client.Send(&dispatchercontracts.SubscribeToWorkflowRunsRequest{
|
||||
WorkflowRunId: workflowRunId,
|
||||
})
|
||||
|
||||
if err == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
time.Sleep(DefaultActionListenerRetryInterval)
|
||||
}
|
||||
|
||||
return fmt.Errorf("could not send to the worker after %d retries", DefaultActionListenerRetryCount)
|
||||
}
|
||||
|
||||
func (l *WorkflowRunsListener) Listen(ctx context.Context) error {
|
||||
for {
|
||||
l.clientMu.RLock()
|
||||
event, err := l.client.Recv()
|
||||
l.clientMu.RUnlock()
|
||||
|
||||
if err != nil {
|
||||
if errors.Is(err, io.EOF) || status.Code(err) == codes.Canceled {
|
||||
return nil
|
||||
}
|
||||
|
||||
retryErr := l.retrySubscribe(ctx)
|
||||
|
||||
if retryErr != nil {
|
||||
return retryErr
|
||||
}
|
||||
|
||||
continue
|
||||
}
|
||||
|
||||
if err := l.handleWorkflowRun(event); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (l *WorkflowRunsListener) Close() error {
|
||||
return l.client.CloseSend()
|
||||
}
|
||||
|
||||
func (l *WorkflowRunsListener) handleWorkflowRun(event *dispatchercontracts.WorkflowRunEvent) error {
|
||||
// find all handlers for this workflow run
|
||||
handlers, ok := l.handlers.Load(event.WorkflowRunId)
|
||||
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
|
||||
eg := errgroup.Group{}
|
||||
|
||||
h := handlers.(*threadSafeHandlers)
|
||||
|
||||
h.mu.RLock()
|
||||
|
||||
for _, handler := range h.handlers {
|
||||
handlerCp := handler
|
||||
|
||||
eg.Go(func() error {
|
||||
return handlerCp(event)
|
||||
})
|
||||
}
|
||||
|
||||
h.mu.RUnlock()
|
||||
|
||||
err := eg.Wait()
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
type SubscribeClient interface {
|
||||
On(ctx context.Context, workflowRunId string, handler RunHandler) error
|
||||
|
||||
Stream(ctx context.Context, workflowRunId string, handler StreamHandler) error
|
||||
|
||||
SubscribeToWorkflowRunEvents(ctx context.Context) (*WorkflowRunsListener, error)
|
||||
}
|
||||
|
||||
type ClientEventListener interface {
|
||||
OnRunEvent(ctx context.Context, event *RunEvent) error
|
||||
OnWorkflowEvent(ctx context.Context, event *WorkflowEvent) error
|
||||
}
|
||||
|
||||
type subscribeClientImpl struct {
|
||||
@@ -111,3 +315,29 @@ func (r *subscribeClientImpl) Stream(ctx context.Context, workflowRunId string,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (r *subscribeClientImpl) SubscribeToWorkflowRunEvents(ctx context.Context) (*WorkflowRunsListener, error) {
|
||||
l, err := r.newWorkflowRunsListener(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
go func() {
|
||||
defer func() {
|
||||
err := l.Close()
|
||||
|
||||
if err != nil {
|
||||
r.l.Error().Err(err).Msg("failed to close workflow run events listener")
|
||||
}
|
||||
}()
|
||||
|
||||
err := l.Listen(ctx)
|
||||
|
||||
if err != nil {
|
||||
r.l.Error().Err(err).Msg("failed to listen for workflow run events")
|
||||
}
|
||||
}()
|
||||
|
||||
return l, nil
|
||||
}
|
||||
|
||||
@@ -1,36 +1,37 @@
|
||||
package worker
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/rs/zerolog"
|
||||
|
||||
"github.com/hatchet-dev/hatchet/pkg/client"
|
||||
"github.com/hatchet-dev/hatchet/pkg/client/rest"
|
||||
|
||||
dispatchercontracts "github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
|
||||
)
|
||||
|
||||
type ChildWorkflow struct {
|
||||
workflowRunId string
|
||||
client client.Client
|
||||
l *zerolog.Logger
|
||||
// client client.Client
|
||||
l *zerolog.Logger
|
||||
listener *client.WorkflowRunsListener
|
||||
}
|
||||
|
||||
type ChildWorkflowResult struct {
|
||||
workflowRun *rest.WorkflowRun
|
||||
workflowRun *dispatchercontracts.WorkflowRunEvent
|
||||
}
|
||||
|
||||
func (r *ChildWorkflowResult) StepOutput(key string, v interface{}) error {
|
||||
var outputBytes []byte
|
||||
for _, jobRun := range *r.workflowRun.JobRuns {
|
||||
for _, stepRun := range *jobRun.StepRuns {
|
||||
if stepRun.Step.ReadableId == key && stepRun.Output != nil {
|
||||
outputBytes = []byte(*stepRun.Output)
|
||||
for _, stepRunResult := range r.workflowRun.Results {
|
||||
if stepRunResult.StepReadableId == key {
|
||||
if stepRunResult.Error != nil {
|
||||
return fmt.Errorf("step run failed: %s", *stepRunResult.Error)
|
||||
}
|
||||
|
||||
if stepRunResult.Output != nil {
|
||||
outputBytes = []byte(*stepRunResult.Output)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -47,89 +48,28 @@ func (r *ChildWorkflowResult) StepOutput(key string, v interface{}) error {
|
||||
}
|
||||
|
||||
func (c *ChildWorkflow) Result() (*ChildWorkflowResult, error) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
resChan := make(chan *ChildWorkflowResult)
|
||||
workflowErrChan := make(chan error)
|
||||
errChan := make(chan error)
|
||||
|
||||
f := func() {
|
||||
resp, err := c.client.API().WorkflowRunGetWithResponse(
|
||||
ctx,
|
||||
uuid.MustParse(c.client.TenantId()),
|
||||
uuid.MustParse(c.workflowRunId),
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
errChan <- fmt.Errorf("failed to get workflow run: %w", err)
|
||||
return
|
||||
}
|
||||
|
||||
if workflowRun := resp.JSON200; workflowRun != nil {
|
||||
if workflowRun.Status == rest.SUCCEEDED {
|
||||
// write the workflow run to the channel
|
||||
resChan <- &ChildWorkflowResult{
|
||||
workflowRun: workflowRun,
|
||||
}
|
||||
}
|
||||
|
||||
if workflowRun.Status == rest.FAILED || workflowRun.Status == rest.CANCELLED {
|
||||
// write the error to the channel
|
||||
workflowErrChan <- fmt.Errorf("workflow run failed with status %s", workflowRun.Status)
|
||||
}
|
||||
} else {
|
||||
errChan <- fmt.Errorf("request failed with status %d", resp.StatusCode())
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// start two goroutines: one which polls the API for the workflow run result, and one which listens for
|
||||
// workflow finished events
|
||||
go func() {
|
||||
f()
|
||||
|
||||
// poll the API for the workflow run result
|
||||
ticker := time.NewTicker(5 * time.Second)
|
||||
|
||||
for {
|
||||
err := c.listener.AddWorkflowRun(
|
||||
c.workflowRunId,
|
||||
func(event client.WorkflowRunEvent) error {
|
||||
// non-blocking send
|
||||
select {
|
||||
case <-ticker.C:
|
||||
f()
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case resChan <- &ChildWorkflowResult{
|
||||
workflowRun: event,
|
||||
}: // continue
|
||||
default:
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
go func() {
|
||||
// listen for workflow finished events
|
||||
err := c.client.Subscribe().On(
|
||||
ctx,
|
||||
c.workflowRunId,
|
||||
func(event client.RunEvent) error {
|
||||
if event.ResourceType == dispatchercontracts.ResourceType_RESOURCE_TYPE_WORKFLOW_RUN {
|
||||
if event.EventType == dispatchercontracts.ResourceEventType_RESOURCE_EVENT_TYPE_COMPLETED {
|
||||
f()
|
||||
}
|
||||
}
|
||||
return nil
|
||||
},
|
||||
)
|
||||
|
||||
return nil
|
||||
},
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
errChan <- fmt.Errorf("failed to listen for workflow events: %w", err)
|
||||
}
|
||||
}()
|
||||
|
||||
select {
|
||||
case res := <-resChan:
|
||||
return res, nil
|
||||
case err := <-workflowErrChan:
|
||||
return nil, err
|
||||
case err := <-errChan:
|
||||
c.l.Err(err).Msg("error occurred")
|
||||
return nil, err
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to listen for workflow events: %w", err)
|
||||
}
|
||||
|
||||
res := <-resChan
|
||||
|
||||
return res, nil
|
||||
}
|
||||
|
||||
@@ -72,8 +72,10 @@ type hatchetContext struct {
|
||||
c client.Client
|
||||
l *zerolog.Logger
|
||||
|
||||
i int
|
||||
indexMu sync.Mutex
|
||||
i int
|
||||
indexMu sync.Mutex
|
||||
listener *client.WorkflowRunsListener
|
||||
listenerMu sync.Mutex
|
||||
}
|
||||
|
||||
func newHatchetContext(
|
||||
@@ -176,7 +178,32 @@ type SpawnWorkflowOpts struct {
|
||||
Key *string
|
||||
}
|
||||
|
||||
func (h *hatchetContext) saveOrLoadListener() (*client.WorkflowRunsListener, error) {
|
||||
h.listenerMu.Lock()
|
||||
defer h.listenerMu.Unlock()
|
||||
|
||||
if h.listener != nil {
|
||||
return h.listener, nil
|
||||
}
|
||||
|
||||
listener, err := h.client().Subscribe().SubscribeToWorkflowRunEvents(h)
|
||||
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to subscribe to workflow run events: %w", err)
|
||||
}
|
||||
|
||||
h.listener = listener
|
||||
|
||||
return listener, nil
|
||||
}
|
||||
|
||||
func (h *hatchetContext) SpawnWorkflow(workflowName string, input any, opts *SpawnWorkflowOpts) (*ChildWorkflow, error) {
|
||||
listener, err := h.saveOrLoadListener()
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
workflowRunId, err := h.client().Admin().RunChildWorkflow(
|
||||
workflowName,
|
||||
input,
|
||||
@@ -197,8 +224,8 @@ func (h *hatchetContext) SpawnWorkflow(workflowName string, input any, opts *Spa
|
||||
|
||||
return &ChildWorkflow{
|
||||
workflowRunId: workflowRunId,
|
||||
client: h.client(),
|
||||
l: h.l,
|
||||
listener: listener,
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
||||
@@ -407,6 +407,8 @@ func (w *Worker) startStepRun(ctx context.Context, assignedAction *client.Action
|
||||
// the service-specific middleware
|
||||
return w.middlewares.runAll(hCtx, func(ctx HatchetContext) error {
|
||||
return svc.mws.runAll(ctx, func(ctx HatchetContext) error {
|
||||
defer cancel()
|
||||
|
||||
args := []any{ctx}
|
||||
|
||||
if arg != nil {
|
||||
|
||||
Reference in New Issue
Block a user