fix: prevent large worker gRPC stream backlogs (#2597)

* fix: prevent large worker backlogs

* add config value

* add doc for troubleshooting
This commit is contained in:
abelanger5
2025-12-03 17:15:43 -05:00
committed by GitHub
parent 2dfdd9dd0c
commit d071a1c36b
9 changed files with 571 additions and 431 deletions

View File

@@ -445,6 +445,7 @@ func runV0Config(ctx context.Context, sc *server.ServerConfig) ([]Teardown, erro
dispatcher.WithEntitlementsRepository(sc.EntitlementRepository),
dispatcher.WithCache(cacheInstance),
dispatcher.WithPayloadSizeThreshold(sc.Runtime.GRPCMaxMsgSize),
dispatcher.WithDefaultMaxWorkerBacklogSize(int64(sc.Runtime.GRPCWorkerStreamMaxBacklogSize)),
)
if err != nil {
@@ -905,6 +906,7 @@ func runV1Config(ctx context.Context, sc *server.ServerConfig) ([]Teardown, erro
dispatcher.WithEntitlementsRepository(sc.EntitlementRepository),
dispatcher.WithCache(cacheInstance),
dispatcher.WithPayloadSizeThreshold(sc.Runtime.GRPCMaxMsgSize),
dispatcher.WithDefaultMaxWorkerBacklogSize(int64(sc.Runtime.GRPCWorkerStreamMaxBacklogSize)),
)
if err != nil {

View File

@@ -34,6 +34,7 @@ export default {
type: "separator",
},
docker: "Running with Docker",
"troubleshooting-workers": "Troubleshooting",
compute: {
title: "Managed Compute",
type: "page",

View File

@@ -0,0 +1,29 @@
import { Tabs, Callout } from "nextra/components";
# Troubleshooting Hatchet Workers
This guide aims to document common issues when deploying Hatchet workers.
## Could not send task to worker
If you see this error in the event history of a task, it could mean several things:
1. The worker is closing its network connection while the task is being sent. This could be caused by the worker crashing or going offline.
2. The payload is too large for the worker to accept or the Hatchet engine to send. The default maximum payload size is 4MB. Consider reducing the size of the input data or output data of your tasks.
3. The worker has a large backlog of tasks in-flight on the network connection and is rejecting new tasks. This can occur if workers are geographically distant from the Hatchet engine or if there are network issues causing delays. Hatchet Cloud runs by default in `us-west-2` (Oregon, USA), so consider deploying your workers in a region close to that for the best performance.
If you are self-hosting, you can increase the maximum backlog size via the `SERVER_GRPC_WORKER_STREAM_MAX_BACKLOG_SIZE` environment variable in your Hatchet engine configuration. The default is 20.
## No workers visible in dashboard
If you have deployed workers but they are not visible in the Hatchet dashboard, it is likely that:
1. Your API token is invalid or incorrect. Ensure that the token you are using to start the worker matches the token generated in the Hatchet dashboard for your tenant.
2. Worker heartbeats are not reaching the Hatchet engine. You will see noisy logs in the worker output if this is the case.
## Phantom workers active in dashboard
This is often due to workers still running in your deployed environment. We see this most often with very long termination periods for workers, or in local development environments where worker processes are leaking. If you are in a local development environment, you can usually view running Hatchet worker processes via `ps -a | grep worker` (or whatever your entrypoint binary is called) and kill them manually.

View File

@@ -41,20 +41,21 @@ type Dispatcher interface {
type DispatcherImpl struct {
contracts.UnimplementedDispatcherServer
s gocron.Scheduler
mq msgqueue.MessageQueue
mqv1 msgqueuev1.MessageQueue
pubBuffer *msgqueuev1.MQPubBuffer
sharedReader *msgqueue.SharedTenantReader
sharedNonBufferedReaderv1 *msgqueuev1.SharedTenantReader
sharedBufferedReaderv1 *msgqueuev1.SharedBufferedTenantReader
l *zerolog.Logger
dv datautils.DataDecoderValidator
v validator.Validator
repo repository.EngineRepository
repov1 v1.Repository
cache cache.Cacheable
payloadSizeThreshold int
s gocron.Scheduler
mq msgqueue.MessageQueue
mqv1 msgqueuev1.MessageQueue
pubBuffer *msgqueuev1.MQPubBuffer
sharedReader *msgqueue.SharedTenantReader
sharedNonBufferedReaderv1 *msgqueuev1.SharedTenantReader
sharedBufferedReaderv1 *msgqueuev1.SharedBufferedTenantReader
l *zerolog.Logger
dv datautils.DataDecoderValidator
v validator.Validator
repo repository.EngineRepository
repov1 v1.Repository
cache cache.Cacheable
payloadSizeThreshold int
defaultMaxWorkerBacklogSize int64
entitlements repository.EntitlementsRepository
@@ -127,17 +128,18 @@ func (w *workers) Delete(workerId string) {
type DispatcherOpt func(*DispatcherOpts)
type DispatcherOpts struct {
mq msgqueue.MessageQueue
mqv1 msgqueuev1.MessageQueue
l *zerolog.Logger
dv datautils.DataDecoderValidator
repo repository.EngineRepository
repov1 v1.Repository
entitlements repository.EntitlementsRepository
dispatcherId string
alerter hatcheterrors.Alerter
cache cache.Cacheable
payloadSizeThreshold int
mq msgqueue.MessageQueue
mqv1 msgqueuev1.MessageQueue
l *zerolog.Logger
dv datautils.DataDecoderValidator
repo repository.EngineRepository
repov1 v1.Repository
entitlements repository.EntitlementsRepository
dispatcherId string
alerter hatcheterrors.Alerter
cache cache.Cacheable
payloadSizeThreshold int
defaultMaxWorkerBacklogSize int64
}
func defaultDispatcherOpts() *DispatcherOpts {
@@ -145,11 +147,12 @@ func defaultDispatcherOpts() *DispatcherOpts {
alerter := hatcheterrors.NoOpAlerter{}
return &DispatcherOpts{
l: &logger,
dv: datautils.NewDataDecoderValidator(),
dispatcherId: uuid.New().String(),
alerter: alerter,
payloadSizeThreshold: 3 * 1024 * 1024,
l: &logger,
dv: datautils.NewDataDecoderValidator(),
dispatcherId: uuid.New().String(),
alerter: alerter,
payloadSizeThreshold: 3 * 1024 * 1024,
defaultMaxWorkerBacklogSize: 20,
}
}
@@ -219,6 +222,12 @@ func WithPayloadSizeThreshold(threshold int) DispatcherOpt {
}
}
func WithDefaultMaxWorkerBacklogSize(size int64) DispatcherOpt {
return func(opts *DispatcherOpts) {
opts.defaultMaxWorkerBacklogSize = size
}
}
func New(fs ...DispatcherOpt) (*DispatcherImpl, error) {
opts := defaultDispatcherOpts()
@@ -266,21 +275,22 @@ func New(fs ...DispatcherOpt) (*DispatcherImpl, error) {
pubBuffer := msgqueuev1.NewMQPubBuffer(opts.mqv1)
return &DispatcherImpl{
mq: opts.mq,
mqv1: opts.mqv1,
pubBuffer: pubBuffer,
l: opts.l,
dv: opts.dv,
v: validator.NewDefaultValidator(),
repo: opts.repo,
repov1: opts.repov1,
entitlements: opts.entitlements,
dispatcherId: opts.dispatcherId,
workers: &workers{},
s: s,
a: a,
cache: opts.cache,
payloadSizeThreshold: opts.payloadSizeThreshold,
mq: opts.mq,
mqv1: opts.mqv1,
pubBuffer: pubBuffer,
l: opts.l,
dv: opts.dv,
v: validator.NewDefaultValidator(),
repo: opts.repo,
repov1: opts.repov1,
entitlements: opts.entitlements,
dispatcherId: opts.dispatcherId,
workers: &workers{},
s: s,
a: a,
cache: opts.cache,
payloadSizeThreshold: opts.payloadSizeThreshold,
defaultMaxWorkerBacklogSize: opts.defaultMaxWorkerBacklogSize,
}, nil
}

View File

@@ -9,13 +9,10 @@ import (
"time"
"github.com/hashicorp/go-multierror"
"go.opentelemetry.io/otel/codes"
"golang.org/x/sync/errgroup"
"google.golang.org/grpc"
msgqueuev1 "github.com/hatchet-dev/hatchet/internal/msgqueue/v1"
"github.com/hatchet-dev/hatchet/internal/queueutils"
"github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
tasktypesv1 "github.com/hatchet-dev/hatchet/internal/services/shared/tasktypes/v1"
"github.com/hatchet-dev/hatchet/pkg/repository/postgres/sqlchelpers"
v1 "github.com/hatchet-dev/hatchet/pkg/repository/v1"
@@ -23,211 +20,6 @@ import (
"github.com/hatchet-dev/hatchet/pkg/telemetry"
)
func (worker *subscribedWorker) StartTaskFromBulk(
ctx context.Context,
tenantId string,
task *v1.V1TaskWithPayload,
) error {
if ctx.Err() != nil {
return fmt.Errorf("context done before starting task: %w", ctx.Err())
}
ctx, span := telemetry.NewSpan(ctx, "start-step-run-from-bulk") // nolint:ineffassign
defer span.End()
inputBytes := []byte{}
if task.Payload != nil {
inputBytes = task.Payload
}
action := populateAssignedAction(tenantId, task.V1Task, task.RetryCount)
action.ActionType = contracts.ActionType_START_STEP_RUN
action.ActionPayload = string(inputBytes)
err := worker.sendToWorker(ctx, action)
if err != nil {
// if the context is done, we return nil, because the worker took too long to receive the message, and we're not
// sure if the worker received it or not. this is equivalent to a network drop, and would be resolved by worker-side
// acks, which we don't currently have.
if errors.Is(err, context.DeadlineExceeded) {
return nil
}
return fmt.Errorf("could not send start action to worker: %w", err)
}
return nil
}
func (worker *subscribedWorker) sendToWorker(
ctx context.Context,
action *contracts.AssignedAction,
) error {
ctx, span := telemetry.NewSpan(ctx, "send-to-worker") // nolint:ineffassign
defer span.End()
telemetry.WithAttributes(
span,
telemetry.AttributeKV{
Key: "worker.id",
Value: worker.workerId,
},
)
telemetry.WithAttributes(
span,
telemetry.AttributeKV{
Key: "payload.size_bytes",
Value: len(action.ActionPayload),
},
)
_, encodeSpan := telemetry.NewSpan(ctx, "encode-action")
msg := &grpc.PreparedMsg{}
err := msg.Encode(worker.stream, action)
if err != nil {
encodeSpan.RecordError(err)
encodeSpan.End()
return fmt.Errorf("could not encode action: %w", err)
}
encodeSpan.End()
lockBegin := time.Now()
_, lockSpan := telemetry.NewSpan(ctx, "acquire-worker-stream-lock")
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
lockSpan.End()
telemetry.WithAttributes(span, telemetry.AttributeKV{
Key: "lock.duration_ms",
Value: time.Since(lockBegin).Milliseconds(),
})
_, streamSpan := telemetry.NewSpan(ctx, "send-worker-stream")
defer streamSpan.End()
sendMsgBegin := time.Now()
sentCh := make(chan error, 1)
go func() {
defer close(sentCh)
err = worker.stream.SendMsg(msg)
if err != nil {
span.RecordError(err)
}
if time.Since(sendMsgBegin) > 50*time.Millisecond {
span.SetStatus(codes.Error, "flow control detected")
span.RecordError(fmt.Errorf("send took too long, we may be in flow control: %s", time.Since(sendMsgBegin)))
}
sentCh <- err
}()
select {
case <-ctx.Done():
return fmt.Errorf("context done before send could complete: %w", ctx.Err())
case err = <-sentCh:
return err
}
}
func (worker *subscribedWorker) CancelTask(
ctx context.Context,
tenantId string,
task *sqlcv1.V1Task,
retryCount int32,
) error {
if ctx.Err() != nil {
return fmt.Errorf("context done before cancelling task: %w", ctx.Err())
}
ctx, span := telemetry.NewSpan(ctx, "cancel-task") // nolint:ineffassign
defer span.End()
action := populateAssignedAction(tenantId, task, retryCount)
action.ActionType = contracts.ActionType_CANCEL_STEP_RUN
sentCh := make(chan error, 1)
go func() {
defer close(sentCh)
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
sentCh <- worker.stream.Send(action)
}()
select {
case <-ctx.Done():
return fmt.Errorf("context done before send could complete: %w", ctx.Err())
case err := <-sentCh:
if err != nil {
span.RecordError(err)
return fmt.Errorf("could not send cancel action to worker: %w", err)
}
}
return nil
}
func populateAssignedAction(tenantID string, task *sqlcv1.V1Task, retryCount int32) *contracts.AssignedAction {
workflowId := sqlchelpers.UUIDToStr(task.WorkflowID)
workflowVersionId := sqlchelpers.UUIDToStr(task.WorkflowVersionID)
action := &contracts.AssignedAction{
TenantId: tenantID,
JobId: sqlchelpers.UUIDToStr(task.StepID), // FIXME
JobName: task.StepReadableID,
JobRunId: sqlchelpers.UUIDToStr(task.ExternalID), // FIXME
StepId: sqlchelpers.UUIDToStr(task.StepID),
StepRunId: sqlchelpers.UUIDToStr(task.ExternalID),
ActionId: task.ActionID,
StepName: task.StepReadableID,
WorkflowRunId: sqlchelpers.UUIDToStr(task.WorkflowRunID),
RetryCount: retryCount,
Priority: task.Priority.Int32,
WorkflowId: &workflowId,
WorkflowVersionId: &workflowVersionId,
}
if task.AdditionalMetadata != nil {
metadataStr := string(task.AdditionalMetadata)
action.AdditionalMetadata = &metadataStr
}
if task.ParentTaskExternalID.Valid {
parentId := sqlchelpers.UUIDToStr(task.ParentTaskExternalID)
action.ParentWorkflowRunId = &parentId
}
if task.ChildIndex.Valid {
i := int32(task.ChildIndex.Int64) // nolint: gosec
action.ChildWorkflowIndex = &i
}
if task.ChildKey.Valid {
key := task.ChildKey.String
action.ChildWorkflowKey = &key
}
return action
}
func (d *DispatcherImpl) handleTaskBulkAssignedTask(ctx context.Context, msg *msgqueuev1.Message) error {
ctx, span := telemetry.NewSpanWithCarrier(ctx, "task-assigned-bulk", msg.OtelCarrier)
defer span.End()

View File

@@ -33,182 +33,6 @@ import (
"github.com/hatchet-dev/hatchet/pkg/telemetry"
)
type subscribedWorker struct {
// stream is the server side of the RPC stream
stream contracts.Dispatcher_ListenServer
// finished is used to signal closure of a client subscribing goroutine
finished chan<- bool
sendMu sync.Mutex
workerId string
}
func (worker *subscribedWorker) StartStepRun(
ctx context.Context,
tenantId string,
stepRun *dbsqlc.GetStepRunForEngineRow,
stepRunData *dbsqlc.GetStepRunDataForEngineRow,
) error {
ctx, span := telemetry.NewSpan(ctx, "start-step-run") // nolint:ineffassign
defer span.End()
inputBytes := []byte{}
if stepRunData.Input != nil {
inputBytes = stepRunData.Input
}
stepName := stepRun.StepReadableId.String
action := &contracts.AssignedAction{
TenantId: tenantId,
JobId: sqlchelpers.UUIDToStr(stepRun.JobId),
JobName: stepRun.JobName,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
StepId: sqlchelpers.UUIDToStr(stepRun.StepId),
StepRunId: sqlchelpers.UUIDToStr(stepRun.SRID),
ActionType: contracts.ActionType_START_STEP_RUN,
ActionId: stepRun.ActionId,
ActionPayload: string(inputBytes),
StepName: stepName,
WorkflowRunId: sqlchelpers.UUIDToStr(stepRun.WorkflowRunId),
RetryCount: stepRun.SRRetryCount,
// NOTE: This is the default because this method is unused
Priority: 1,
}
if stepRunData.AdditionalMetadata != nil {
metadataStr := string(stepRunData.AdditionalMetadata)
action.AdditionalMetadata = &metadataStr
}
if stepRunData.ChildIndex.Valid {
action.ChildWorkflowIndex = &stepRunData.ChildIndex.Int32
}
if stepRunData.ChildKey.Valid {
action.ChildWorkflowKey = &stepRunData.ChildKey.String
}
if stepRunData.ParentId.Valid {
parentId := sqlchelpers.UUIDToStr(stepRunData.ParentId)
action.ParentWorkflowRunId = &parentId
}
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(action)
}
func (worker *subscribedWorker) StartStepRunFromBulk(
ctx context.Context,
tenantId string,
stepRun *dbsqlc.GetStepRunBulkDataForEngineRow,
) error {
ctx, span := telemetry.NewSpan(ctx, "start-step-run-from-bulk") // nolint:ineffassign
defer span.End()
inputBytes := []byte{}
if stepRun.Input != nil {
inputBytes = stepRun.Input
}
stepName := stepRun.StepReadableId.String
action := &contracts.AssignedAction{
TenantId: tenantId,
JobId: sqlchelpers.UUIDToStr(stepRun.JobId),
JobName: stepRun.JobName,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
StepId: sqlchelpers.UUIDToStr(stepRun.StepId),
StepRunId: sqlchelpers.UUIDToStr(stepRun.SRID),
ActionType: contracts.ActionType_START_STEP_RUN,
ActionId: stepRun.ActionId,
ActionPayload: string(inputBytes),
StepName: stepName,
WorkflowRunId: sqlchelpers.UUIDToStr(stepRun.WorkflowRunId),
RetryCount: stepRun.SRRetryCount,
Priority: stepRun.Priority,
}
if stepRun.AdditionalMetadata != nil {
metadataStr := string(stepRun.AdditionalMetadata)
action.AdditionalMetadata = &metadataStr
}
if stepRun.ChildIndex.Valid {
action.ChildWorkflowIndex = &stepRun.ChildIndex.Int32
}
if stepRun.ChildKey.Valid {
action.ChildWorkflowKey = &stepRun.ChildKey.String
}
if stepRun.ParentId.Valid {
parentId := sqlchelpers.UUIDToStr(stepRun.ParentId)
action.ParentWorkflowRunId = &parentId
}
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(action)
}
func (worker *subscribedWorker) StartGroupKeyAction(
ctx context.Context,
tenantId string,
getGroupKeyRun *dbsqlc.GetGroupKeyRunForEngineRow,
) error {
ctx, span := telemetry.NewSpan(ctx, "start-group-key-action") // nolint:ineffassign
defer span.End()
inputData := getGroupKeyRun.GetGroupKeyRun.Input
workflowRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.WorkflowRunId)
getGroupKeyRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.GetGroupKeyRun.ID)
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(&contracts.AssignedAction{
TenantId: tenantId,
WorkflowRunId: workflowRunId,
GetGroupKeyRunId: getGroupKeyRunId,
ActionType: contracts.ActionType_START_GET_GROUP_KEY,
ActionId: getGroupKeyRun.ActionId,
ActionPayload: string(inputData),
})
}
func (worker *subscribedWorker) CancelStepRun(
ctx context.Context,
tenantId string,
stepRun *dbsqlc.GetStepRunForEngineRow,
) error {
ctx, span := telemetry.NewSpan(ctx, "cancel-step-run") // nolint:ineffassign
defer span.End()
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(&contracts.AssignedAction{
TenantId: tenantId,
JobId: sqlchelpers.UUIDToStr(stepRun.JobId),
JobName: stepRun.JobName,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
StepId: sqlchelpers.UUIDToStr(stepRun.StepId),
StepRunId: sqlchelpers.UUIDToStr(stepRun.SRID),
ActionType: contracts.ActionType_CANCEL_STEP_RUN,
StepName: stepRun.StepReadableId.String,
WorkflowRunId: sqlchelpers.UUIDToStr(stepRun.WorkflowRunId),
RetryCount: stepRun.SRRetryCount,
})
}
func (s *DispatcherImpl) Register(ctx context.Context, request *contracts.WorkerRegisterRequest) (*contracts.WorkerRegisterResponse, error) {
tenant := ctx.Value("tenant").(*dbsqlc.Tenant)
tenantId := sqlchelpers.UUIDToStr(tenant.ID)
@@ -360,7 +184,7 @@ func (s *DispatcherImpl) Listen(request *contracts.WorkerListenRequest, stream c
fin := make(chan bool)
s.workers.Add(request.WorkerId, sessionId, &subscribedWorker{stream: stream, finished: fin, workerId: request.WorkerId})
s.workers.Add(request.WorkerId, sessionId, newSubscribedWorker(stream, fin, request.WorkerId, 20))
defer func() {
// non-blocking send
@@ -482,7 +306,7 @@ func (s *DispatcherImpl) ListenV2(request *contracts.WorkerListenRequest, stream
fin := make(chan bool)
s.workers.Add(request.WorkerId, sessionId, &subscribedWorker{stream: stream, finished: fin, workerId: request.WorkerId})
s.workers.Add(request.WorkerId, sessionId, newSubscribedWorker(stream, fin, request.WorkerId, s.defaultMaxWorkerBacklogSize))
defer func() {
// non-blocking send

View File

@@ -0,0 +1,210 @@
package dispatcher
import (
"context"
"sync"
"github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
"github.com/hatchet-dev/hatchet/pkg/repository/postgres/dbsqlc"
"github.com/hatchet-dev/hatchet/pkg/repository/postgres/sqlchelpers"
"github.com/hatchet-dev/hatchet/pkg/telemetry"
)
type subscribedWorker struct {
// stream is the server side of the RPC stream
stream contracts.Dispatcher_ListenServer
// finished is used to signal closure of a client subscribing goroutine
finished chan<- bool
sendMu sync.Mutex
workerId string
backlogSize int64
backlogSizeMu sync.Mutex
maxBacklogSize int64
}
func newSubscribedWorker(
stream contracts.Dispatcher_ListenServer,
fin chan<- bool,
workerId string,
maxBacklogSize int64,
) *subscribedWorker {
if maxBacklogSize <= 0 {
maxBacklogSize = 20
}
return &subscribedWorker{
stream: stream,
finished: fin,
workerId: workerId,
maxBacklogSize: maxBacklogSize,
}
}
func (worker *subscribedWorker) StartStepRun(
ctx context.Context,
tenantId string,
stepRun *dbsqlc.GetStepRunForEngineRow,
stepRunData *dbsqlc.GetStepRunDataForEngineRow,
) error {
_, span := telemetry.NewSpan(ctx, "start-step-run")
defer span.End()
inputBytes := []byte{}
if stepRunData.Input != nil {
inputBytes = stepRunData.Input
}
stepName := stepRun.StepReadableId.String
action := &contracts.AssignedAction{
TenantId: tenantId,
JobId: sqlchelpers.UUIDToStr(stepRun.JobId),
JobName: stepRun.JobName,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
StepId: sqlchelpers.UUIDToStr(stepRun.StepId),
StepRunId: sqlchelpers.UUIDToStr(stepRun.SRID),
ActionType: contracts.ActionType_START_STEP_RUN,
ActionId: stepRun.ActionId,
ActionPayload: string(inputBytes),
StepName: stepName,
WorkflowRunId: sqlchelpers.UUIDToStr(stepRun.WorkflowRunId),
RetryCount: stepRun.SRRetryCount,
// NOTE: This is the default because this method is unused
Priority: 1,
}
if stepRunData.AdditionalMetadata != nil {
metadataStr := string(stepRunData.AdditionalMetadata)
action.AdditionalMetadata = &metadataStr
}
if stepRunData.ChildIndex.Valid {
action.ChildWorkflowIndex = &stepRunData.ChildIndex.Int32
}
if stepRunData.ChildKey.Valid {
action.ChildWorkflowKey = &stepRunData.ChildKey.String
}
if stepRunData.ParentId.Valid {
parentId := sqlchelpers.UUIDToStr(stepRunData.ParentId)
action.ParentWorkflowRunId = &parentId
}
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(action)
}
func (worker *subscribedWorker) StartStepRunFromBulk(
ctx context.Context,
tenantId string,
stepRun *dbsqlc.GetStepRunBulkDataForEngineRow,
) error {
_, span := telemetry.NewSpan(ctx, "start-step-run-from-bulk")
defer span.End()
inputBytes := []byte{}
if stepRun.Input != nil {
inputBytes = stepRun.Input
}
stepName := stepRun.StepReadableId.String
action := &contracts.AssignedAction{
TenantId: tenantId,
JobId: sqlchelpers.UUIDToStr(stepRun.JobId),
JobName: stepRun.JobName,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
StepId: sqlchelpers.UUIDToStr(stepRun.StepId),
StepRunId: sqlchelpers.UUIDToStr(stepRun.SRID),
ActionType: contracts.ActionType_START_STEP_RUN,
ActionId: stepRun.ActionId,
ActionPayload: string(inputBytes),
StepName: stepName,
WorkflowRunId: sqlchelpers.UUIDToStr(stepRun.WorkflowRunId),
RetryCount: stepRun.SRRetryCount,
Priority: stepRun.Priority,
}
if stepRun.AdditionalMetadata != nil {
metadataStr := string(stepRun.AdditionalMetadata)
action.AdditionalMetadata = &metadataStr
}
if stepRun.ChildIndex.Valid {
action.ChildWorkflowIndex = &stepRun.ChildIndex.Int32
}
if stepRun.ChildKey.Valid {
action.ChildWorkflowKey = &stepRun.ChildKey.String
}
if stepRun.ParentId.Valid {
parentId := sqlchelpers.UUIDToStr(stepRun.ParentId)
action.ParentWorkflowRunId = &parentId
}
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(action)
}
func (worker *subscribedWorker) StartGroupKeyAction(
ctx context.Context,
tenantId string,
getGroupKeyRun *dbsqlc.GetGroupKeyRunForEngineRow,
) error {
_, span := telemetry.NewSpan(ctx, "start-group-key-action")
defer span.End()
inputData := getGroupKeyRun.GetGroupKeyRun.Input
workflowRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.WorkflowRunId)
getGroupKeyRunId := sqlchelpers.UUIDToStr(getGroupKeyRun.GetGroupKeyRun.ID)
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(&contracts.AssignedAction{
TenantId: tenantId,
WorkflowRunId: workflowRunId,
GetGroupKeyRunId: getGroupKeyRunId,
ActionType: contracts.ActionType_START_GET_GROUP_KEY,
ActionId: getGroupKeyRun.ActionId,
ActionPayload: string(inputData),
})
}
func (worker *subscribedWorker) CancelStepRun(
ctx context.Context,
tenantId string,
stepRun *dbsqlc.GetStepRunForEngineRow,
) error {
_, span := telemetry.NewSpan(ctx, "cancel-step-run")
defer span.End()
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
return worker.stream.Send(&contracts.AssignedAction{
TenantId: tenantId,
JobId: sqlchelpers.UUIDToStr(stepRun.JobId),
JobName: stepRun.JobName,
JobRunId: sqlchelpers.UUIDToStr(stepRun.JobRunId),
StepId: sqlchelpers.UUIDToStr(stepRun.StepId),
StepRunId: sqlchelpers.UUIDToStr(stepRun.SRID),
ActionType: contracts.ActionType_CANCEL_STEP_RUN,
StepName: stepRun.StepReadableId.String,
WorkflowRunId: sqlchelpers.UUIDToStr(stepRun.WorkflowRunId),
RetryCount: stepRun.SRRetryCount,
})
}

View File

@@ -0,0 +1,267 @@
package dispatcher
import (
"context"
"errors"
"fmt"
"time"
"go.opentelemetry.io/otel/codes"
"google.golang.org/grpc"
"github.com/hatchet-dev/hatchet/internal/services/dispatcher/contracts"
"github.com/hatchet-dev/hatchet/pkg/repository/postgres/sqlchelpers"
v1 "github.com/hatchet-dev/hatchet/pkg/repository/v1"
"github.com/hatchet-dev/hatchet/pkg/repository/v1/sqlcv1"
"github.com/hatchet-dev/hatchet/pkg/telemetry"
)
func (worker *subscribedWorker) StartTaskFromBulk(
ctx context.Context,
tenantId string,
task *v1.V1TaskWithPayload,
) error {
if ctx.Err() != nil {
return fmt.Errorf("context done before starting task: %w", ctx.Err())
}
ctx, span := telemetry.NewSpan(ctx, "start-step-run-from-bulk") // nolint:ineffassign
defer span.End()
inputBytes := []byte{}
if task.Payload != nil {
inputBytes = task.Payload
}
action := populateAssignedAction(tenantId, task.V1Task, task.RetryCount)
action.ActionType = contracts.ActionType_START_STEP_RUN
action.ActionPayload = string(inputBytes)
err := worker.sendToWorker(ctx, action)
if err != nil {
// if the context is done, we return nil, because the worker took too long to receive the message, and we're not
// sure if the worker received it or not. this is equivalent to a network drop, and would be resolved by worker-side
// acks, which we don't currently have.
if errors.Is(err, context.DeadlineExceeded) {
return nil
}
return fmt.Errorf("could not send start action to worker: %w", err)
}
return nil
}
func (worker *subscribedWorker) incBacklogSize(delta int64) bool {
worker.backlogSizeMu.Lock()
defer worker.backlogSizeMu.Unlock()
if worker.backlogSize+delta > worker.maxBacklogSize {
return false
}
worker.backlogSize += delta
return true
}
func (worker *subscribedWorker) decBacklogSize(delta int64) int64 {
worker.backlogSizeMu.Lock()
defer worker.backlogSizeMu.Unlock()
worker.backlogSize -= delta
if worker.backlogSize < 0 {
worker.backlogSize = 0
}
return worker.backlogSize
}
func (worker *subscribedWorker) sendToWorker(
ctx context.Context,
action *contracts.AssignedAction,
) error {
ctx, span := telemetry.NewSpan(ctx, "send-to-worker") // nolint:ineffassign
defer span.End()
telemetry.WithAttributes(
span,
telemetry.AttributeKV{
Key: "worker.id",
Value: worker.workerId,
},
)
telemetry.WithAttributes(
span,
telemetry.AttributeKV{
Key: "payload.size_bytes",
Value: len(action.ActionPayload),
},
)
_, encodeSpan := telemetry.NewSpan(ctx, "encode-action")
msg := &grpc.PreparedMsg{}
err := msg.Encode(worker.stream, action)
if err != nil {
encodeSpan.RecordError(err)
encodeSpan.End()
return fmt.Errorf("could not encode action: %w", err)
}
encodeSpan.End()
incSuccess := worker.incBacklogSize(1)
if !incSuccess {
err := fmt.Errorf("worker backlog size exceeded max of %d", worker.maxBacklogSize)
span.RecordError(err)
span.SetStatus(codes.Error, "worker backlog size exceeded max")
return err
}
lockBegin := time.Now()
_, lockSpan := telemetry.NewSpan(ctx, "acquire-worker-stream-lock")
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
lockSpan.End()
telemetry.WithAttributes(span, telemetry.AttributeKV{
Key: "lock.duration_ms",
Value: time.Since(lockBegin).Milliseconds(),
})
_, streamSpan := telemetry.NewSpan(ctx, "send-worker-stream")
defer streamSpan.End()
sendMsgBegin := time.Now()
sentCh := make(chan error, 1)
go func() {
defer close(sentCh)
defer worker.decBacklogSize(1)
err = worker.stream.SendMsg(msg)
if err != nil {
span.RecordError(err)
}
if time.Since(sendMsgBegin) > 50*time.Millisecond {
span.SetStatus(codes.Error, "flow control detected")
span.RecordError(fmt.Errorf("send took too long, we may be in flow control: %s", time.Since(sendMsgBegin)))
}
sentCh <- err
}()
select {
case <-ctx.Done():
return fmt.Errorf("context done before send could complete: %w", ctx.Err())
case err = <-sentCh:
return err
}
}
func (worker *subscribedWorker) CancelTask(
ctx context.Context,
tenantId string,
task *sqlcv1.V1Task,
retryCount int32,
) error {
if ctx.Err() != nil {
return fmt.Errorf("context done before cancelling task: %w", ctx.Err())
}
ctx, span := telemetry.NewSpan(ctx, "cancel-task") // nolint:ineffassign
defer span.End()
action := populateAssignedAction(tenantId, task, retryCount)
action.ActionType = contracts.ActionType_CANCEL_STEP_RUN
sentCh := make(chan error, 1)
incSuccess := worker.incBacklogSize(1)
if !incSuccess {
err := fmt.Errorf("worker backlog size exceeded max of %d", worker.maxBacklogSize)
span.RecordError(err)
span.SetStatus(codes.Error, "worker backlog size exceeded max")
return err
}
go func() {
defer close(sentCh)
defer worker.decBacklogSize(1)
worker.sendMu.Lock()
defer worker.sendMu.Unlock()
sentCh <- worker.stream.Send(action)
}()
select {
case <-ctx.Done():
return fmt.Errorf("context done before send could complete: %w", ctx.Err())
case err := <-sentCh:
if err != nil {
span.RecordError(err)
return fmt.Errorf("could not send cancel action to worker: %w", err)
}
}
return nil
}
func populateAssignedAction(tenantID string, task *sqlcv1.V1Task, retryCount int32) *contracts.AssignedAction {
workflowId := sqlchelpers.UUIDToStr(task.WorkflowID)
workflowVersionId := sqlchelpers.UUIDToStr(task.WorkflowVersionID)
action := &contracts.AssignedAction{
TenantId: tenantID,
JobId: sqlchelpers.UUIDToStr(task.StepID), // FIXME
JobName: task.StepReadableID,
JobRunId: sqlchelpers.UUIDToStr(task.ExternalID), // FIXME
StepId: sqlchelpers.UUIDToStr(task.StepID),
StepRunId: sqlchelpers.UUIDToStr(task.ExternalID),
ActionId: task.ActionID,
StepName: task.StepReadableID,
WorkflowRunId: sqlchelpers.UUIDToStr(task.WorkflowRunID),
RetryCount: retryCount,
Priority: task.Priority.Int32,
WorkflowId: &workflowId,
WorkflowVersionId: &workflowVersionId,
}
if task.AdditionalMetadata != nil {
metadataStr := string(task.AdditionalMetadata)
action.AdditionalMetadata = &metadataStr
}
if task.ParentTaskExternalID.Valid {
parentId := sqlchelpers.UUIDToStr(task.ParentTaskExternalID)
action.ParentWorkflowRunId = &parentId
}
if task.ChildIndex.Valid {
i := int32(task.ChildIndex.Int64) // nolint: gosec
action.ChildWorkflowIndex = &i
}
if task.ChildKey.Valid {
key := task.ChildKey.String
action.ChildWorkflowKey = &key
}
return action
}

View File

@@ -172,6 +172,10 @@ type ConfigFileRuntime struct {
// GRPCMaxMsgSize is the maximum message size that the grpc server will accept
GRPCMaxMsgSize int `mapstructure:"grpcMaxMsgSize" json:"grpcMaxMsgSize,omitempty" default:"4194304"`
// GRPCWorkerStreamMaxBacklogSize is the maximum number of messages that can be queued for a worker before we start rejecting new
// messages. Default is 20.
GRPCWorkerStreamMaxBacklogSize int `mapstructure:"grpcWorkerStreamMaxBacklogSize" json:"grpcWorkerStreamMaxBacklogSize,omitempty" default:"20"`
// GRPCStaticStreamWindowSize sets the static stream window size for the grpc server. This can help with performance
// with overloaded workers and large messages. Default is 10MB.
GRPCStaticStreamWindowSize int32 `mapstructure:"grpcStaticStreamWindowSize" json:"grpcStaticStreamWindowSize,omitempty" default:"10485760"`
@@ -669,6 +673,7 @@ func BindAllEnv(v *viper.Viper) {
_ = v.BindEnv("runtime.grpcBroadcastAddress", "SERVER_GRPC_BROADCAST_ADDRESS")
_ = v.BindEnv("runtime.grpcInsecure", "SERVER_GRPC_INSECURE")
_ = v.BindEnv("runtime.grpcMaxMsgSize", "SERVER_GRPC_MAX_MSG_SIZE")
_ = v.BindEnv("runtime.grpcWorkerStreamMaxBacklogSize", "SERVER_GRPC_WORKER_STREAM_MAX_BACKLOG_SIZE")
_ = v.BindEnv("runtime.grpcStaticStreamWindowSize", "SERVER_GRPC_STATIC_STREAM_WINDOW_SIZE")
_ = v.BindEnv("runtime.grpcRateLimit", "SERVER_GRPC_RATE_LIMIT")
_ = v.BindEnv("runtime.schedulerConcurrencyRateLimit", "SCHEDULER_CONCURRENCY_RATE_LIMIT")