mirror of
https://github.com/hatchet-dev/hatchet.git
synced 2025-12-16 22:35:11 -06:00
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:
@@ -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 {
|
||||
|
||||
@@ -34,6 +34,7 @@ export default {
|
||||
type: "separator",
|
||||
},
|
||||
docker: "Running with Docker",
|
||||
"troubleshooting-workers": "Troubleshooting",
|
||||
compute: {
|
||||
title: "Managed Compute",
|
||||
type: "page",
|
||||
|
||||
29
frontend/docs/pages/home/troubleshooting-workers.mdx
Normal file
29
frontend/docs/pages/home/troubleshooting-workers.mdx
Normal 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.
|
||||
@@ -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
|
||||
}
|
||||
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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
|
||||
|
||||
210
internal/services/dispatcher/subscribed_worker.go
Normal file
210
internal/services/dispatcher/subscribed_worker.go
Normal 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,
|
||||
})
|
||||
}
|
||||
267
internal/services/dispatcher/subscribed_worker_v1.go
Normal file
267
internal/services/dispatcher/subscribed_worker_v1.go
Normal 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
|
||||
}
|
||||
@@ -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")
|
||||
|
||||
Reference in New Issue
Block a user