2024-01-22 09:25:36 -08:00
import { Container } from 'typedi' ;
import { Flags , type Config } from '@oclif/core' ;
2022-04-08 14:32:08 -07:00
import express from 'express' ;
import http from 'http' ;
2023-01-27 05:56:56 -08:00
import type PCancelable from 'p-cancelable' ;
2023-02-10 05:59:20 -08:00
import { WorkflowExecute } from 'n8n-core' ;
2024-02-21 05:47:02 -08:00
import type { ExecutionStatus , IExecuteResponsePromiseData , INodeTypes , IRun } from 'n8n-workflow' ;
import { Workflow , sleep , ApplicationError } from 'n8n-workflow' ;
2021-02-08 23:59:32 -08:00
2022-11-09 06:25:00 -08:00
import * as Db from '@/Db' ;
import * as ResponseHelper from '@/ResponseHelper' ;
import * as WebhookHelpers from '@/WebhookHelpers' ;
import * as WorkflowExecuteAdditionalData from '@/WorkflowExecuteAdditionalData' ;
import config from '@/config' ;
2023-10-13 02:53:59 -07:00
import type { Job , JobId , JobResponse , WebhookResponse } from '@/Queue' ;
2023-03-16 07:34:13 -07:00
import { Queue } from '@/Queue' ;
2024-06-18 01:50:39 -07:00
import { N8N_VERSION , inTest } from '@/constants' ;
2023-11-10 06:04:26 -08:00
import { ExecutionRepository } from '@db/repositories/execution.repository' ;
import { WorkflowRepository } from '@db/repositories/workflow.repository' ;
2023-08-17 06:42:57 -07:00
import type { ICredentialsOverwrite } from '@/Interfaces' ;
import { CredentialsOverwrites } from '@/CredentialsOverwrites' ;
import { rawBodyReader , bodyParser } from '@/middlewares' ;
2024-02-06 01:08:46 -08:00
import { MessageEventBus } from '@/eventbus/MessageEventBus/MessageEventBus' ;
2023-10-06 04:58:11 -07:00
import type { RedisServicePubSubSubscriber } from '@/services/redis/RedisServicePubSubSubscriber' ;
import { EventMessageGeneric } from '@/eventbus/EventMessageClasses/EventMessageGeneric' ;
import { OrchestrationHandlerWorkerService } from '@/services/orchestration/worker/orchestration.handler.worker.service' ;
import { OrchestrationWorkerService } from '@/services/orchestration/worker/orchestration.worker.service' ;
2024-01-22 09:25:36 -08:00
import type { WorkerJobStatusSummary } from '@/services/orchestration/worker/types' ;
2023-11-28 01:19:27 -08:00
import { ServiceUnavailableError } from '@/errors/response-errors/service-unavailable.error' ;
2024-01-22 09:25:36 -08:00
import { BaseCommand } from './BaseCommand' ;
2024-04-05 11:03:49 -07:00
import { MaxStalledCountError } from '@/errors/max-stalled-count.error' ;
2024-07-03 01:29:25 -07:00
import { AuditEventRelay } from '@/eventbus/audit-event-relay.service' ;
2021-02-08 23:59:32 -08:00
2023-02-10 05:59:20 -08:00
export class Worker extends BaseCommand {
2021-02-08 23:59:32 -08:00
static description = '\nStarts a n8n worker' ;
2022-12-29 03:20:43 -08:00
static examples = [ '$ n8n worker --concurrency=5' ] ;
2021-02-08 23:59:32 -08:00
static flags = {
2024-01-22 09:25:36 -08:00
help : Flags.help ( { char : 'h' } ) ,
concurrency : Flags.integer ( {
2021-02-08 23:59:32 -08:00
default : 10 ,
description : 'How many jobs can run in parallel.' ,
} ) ,
} ;
static runningJobs : {
[ key : string ] : PCancelable < IRun > ;
} = { } ;
2023-10-06 08:52:27 -07:00
static runningJobsSummary : {
[ jobId : string ] : WorkerJobStatusSummary ;
} = { } ;
2023-10-13 02:53:59 -07:00
static jobQueue : Queue ;
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
redisSubscriber : RedisServicePubSubSubscriber ;
2021-02-08 23:59:32 -08:00
/ * *
2022-11-08 08:06:00 -08:00
* Stop n8n in a graceful way .
2021-02-08 23:59:32 -08:00
* Make for example sure that all the webhooks from third party services
* get removed .
* /
2023-02-10 05:59:20 -08:00
async stopProcess() {
2023-10-25 07:35:22 -07:00
this . logger . info ( 'Stopping n8n...' ) ;
2021-02-08 23:59:32 -08:00
try {
2023-12-17 23:23:10 -08:00
await this . externalHooks ? . run ( 'n8n.stop' , [ ] ) ;
2021-02-08 23:59:32 -08:00
2024-05-31 06:41:00 -07:00
const hardStopTimeMs = Date . now ( ) + this . gracefulShutdownTimeoutInS * 1000 ;
2021-02-08 23:59:32 -08:00
// Wait for active workflow executions to finish
let count = 0 ;
while ( Object . keys ( Worker . runningJobs ) . length !== 0 ) {
if ( count ++ % 4 === 0 ) {
2024-05-31 06:41:00 -07:00
const waitLeft = Math . ceil ( ( hardStopTimeMs - Date . now ( ) ) / 1000 ) ;
2023-10-25 07:35:22 -07:00
this . logger . info (
2021-05-01 20:43:01 -07:00
` Waiting for ${
Object . keys ( Worker . runningJobs ) . length
2024-05-31 06:41:00 -07:00
} active executions to finish . . . ( max wait $ { waitLeft } more seconds ) ` ,
2021-05-01 20:43:01 -07:00
) ;
2021-02-08 23:59:32 -08:00
}
2023-07-31 02:00:48 -07:00
2022-11-08 08:06:00 -08:00
await sleep ( 500 ) ;
2021-02-08 23:59:32 -08:00
}
} catch ( error ) {
2023-02-10 05:59:20 -08:00
await this . exitWithCrash ( 'There was an error shutting down n8n.' , error ) ;
2021-02-08 23:59:32 -08:00
}
2023-02-10 05:59:20 -08:00
await this . exitSuccessFully ( ) ;
2021-02-08 23:59:32 -08:00
}
2023-03-16 07:34:13 -07:00
async runJob ( job : Job , nodeTypes : INodeTypes ) : Promise < JobResponse > {
2022-09-09 06:14:49 -07:00
const { executionId , loadStaticData } = job . data ;
2023-11-01 05:51:13 -07:00
const executionRepository = Container . get ( ExecutionRepository ) ;
const fullExecutionData = await executionRepository . findSingleExecution ( executionId , {
includeData : true ,
unflattenData : true ,
} ) ;
2022-03-25 07:10:31 -07:00
2023-06-20 10:13:18 -07:00
if ( ! fullExecutionData ) {
2023-10-25 07:35:22 -07:00
this . logger . error (
2022-09-09 06:14:49 -07:00
` Worker failed to find data of execution " ${ executionId } " in database. Cannot continue. ` ,
{ executionId } ,
2022-05-30 03:05:43 -07:00
) ;
2023-11-29 03:25:10 -08:00
throw new ApplicationError (
'Unable to find data of execution in database. Aborting execution.' ,
{ extra : { executionId } } ,
2022-05-30 03:05:43 -07:00
) ;
2022-03-25 07:10:31 -07:00
}
2024-01-16 01:53:17 -08:00
const workflowId = fullExecutionData . workflowData . id ;
2023-12-14 09:13:12 -08:00
2023-10-25 07:35:22 -07:00
this . logger . info (
2023-02-10 05:59:20 -08:00
` Start job: ${ job . id } (Workflow ID: ${ workflowId } | Execution: ${ executionId } ) ` ,
2021-05-01 20:43:01 -07:00
) ;
2023-11-01 05:51:13 -07:00
await executionRepository . updateStatus ( executionId , 'running' ) ;
2021-08-29 11:58:11 -07:00
2023-06-20 10:13:18 -07:00
let { staticData } = fullExecutionData . workflowData ;
2022-09-09 06:14:49 -07:00
if ( loadStaticData ) {
2023-11-10 06:04:26 -08:00
const workflowData = await Container . get ( WorkflowRepository ) . findOne ( {
2021-02-08 23:59:32 -08:00
select : [ 'id' , 'staticData' ] ,
2023-01-13 09:12:22 -08:00
where : {
2023-02-10 05:59:20 -08:00
id : workflowId ,
2023-01-13 09:12:22 -08:00
} ,
} ) ;
if ( workflowData === null ) {
2023-10-25 07:35:22 -07:00
this . logger . error (
2022-03-25 07:10:31 -07:00
'Worker execution failed because workflow could not be found in database.' ,
2023-02-10 05:59:20 -08:00
{ workflowId , executionId } ,
2021-02-08 23:59:32 -08:00
) ;
2023-11-29 03:25:10 -08:00
throw new ApplicationError ( 'Workflow could not be found' , { extra : { workflowId } } ) ;
2021-02-08 23:59:32 -08:00
}
staticData = workflowData . staticData ;
}
2023-06-20 10:13:18 -07:00
const workflowSettings = fullExecutionData . workflowData . settings ? ? { } ;
2023-03-24 05:11:48 -07:00
let workflowTimeout = workflowSettings . executionTimeout ? ? config . getEnv ( 'executions.timeout' ) ; // initialize with default
2021-04-17 07:44:07 -07:00
let executionTimeoutTimestamp : number | undefined ;
if ( workflowTimeout > 0 ) {
2022-04-08 10:37:27 -07:00
workflowTimeout = Math . min ( workflowTimeout , config . getEnv ( 'executions.maxTimeout' ) ) ;
2021-04-17 07:44:07 -07:00
executionTimeoutTimestamp = Date . now ( ) + workflowTimeout * 1000 ;
}
2021-02-08 23:59:32 -08:00
const workflow = new Workflow ( {
2023-02-10 05:59:20 -08:00
id : workflowId ,
2023-06-20 10:13:18 -07:00
name : fullExecutionData.workflowData.name ,
nodes : fullExecutionData.workflowData.nodes ,
connections : fullExecutionData.workflowData.connections ,
active : fullExecutionData.workflowData.active ,
2021-02-08 23:59:32 -08:00
nodeTypes ,
staticData ,
2023-06-20 10:13:18 -07:00
settings : fullExecutionData.workflowData.settings ,
2021-02-08 23:59:32 -08:00
} ) ;
2021-08-29 11:58:11 -07:00
2021-08-20 09:57:30 -07:00
const additionalData = await WorkflowExecuteAdditionalData . getBase (
2024-05-17 01:53:15 -07:00
undefined ,
2021-08-20 09:57:30 -07:00
undefined ,
executionTimeoutTimestamp ,
) ;
2021-03-10 06:51:18 -08:00
additionalData . hooks = WorkflowExecuteAdditionalData . getWorkflowHooksWorkerExecuter (
2023-06-20 10:13:18 -07:00
fullExecutionData . mode ,
2021-03-10 06:51:18 -08:00
job . data . executionId ,
2023-06-20 10:13:18 -07:00
fullExecutionData . workflowData ,
refactor(core): Move event and telemetry handling into workers in queue mode (#7138)
# Motivation
In Queue mode, finished executions would cause the main instance to
always pull all execution data from the database, unflatten it and then
use it to send out event log events and telemetry events, as well as
required returns to Respond to Webhook nodes etc.
This could cause OOM errors when the data was large, since it had to be
fully unpacked and transformed on the main instance’s side, using up a
lot of memory (and time).
This PR attempts to limit this behaviour to only happen in those
required cases where the data has to be forwarded to some waiting
webhook, for example.
# Changes
Execution data is only required in cases, where the active execution has
a `postExecutePromise` attached to it. These usually forward the data to
some other endpoint (e.g. a listening webhook connection).
By adding a helper `getPostExecutePromiseCount()`, we can decide that in
cases where there is nothing listening at all, there is no reason to
pull the data on the main instance.
Previously, there would always be postExecutePromises because the
telemetry events were called. Now, these have been moved into the
workers, which have been given the various InternalHooks calls to their
hook function arrays, so they themselves issue these telemetry and event
calls.
This results in all event log messages to now be logged on the worker’s
event log, as well as the worker’s eventbus being the one to send out
the events to destinations. The main event log does…pretty much nothing.
We are not logging executions on the main event log any more, because
this would require all events to be replicated 1:1 from the workers to
the main instance(s) (this IS possible and implemented, see the worker’s
`replicateToRedisEventLogFunction` - but it is not enabled to reduce the
amount of traffic over redis).
Partial events in the main log could confuse the recovery process and
would result in, ironically, the recovery corrupting the execution data
by considering them crashed.
# Refactor
I have also used the opportunity to reduce duplicate code and move some
of the hook functionality into
`packages/cli/src/executionLifecycleHooks/shared/sharedHookFunctions.ts`
in preparation for a future full refactor of the hooks
2023-09-13 22:58:15 -07:00
{
retryOf : fullExecutionData.retryOf as string ,
} ,
2021-03-10 06:51:18 -08:00
) ;
2021-11-05 09:45:51 -07:00
additionalData . hooks . hookFunctions . sendResponse = [
async ( response : IExecuteResponsePromiseData ) : Promise < void > = > {
2023-03-16 07:34:13 -07:00
const progress : WebhookResponse = {
2022-09-09 06:14:49 -07:00
executionId ,
2021-11-05 09:45:51 -07:00
response : WebhookHelpers.encodeWebhookResponse ( response ) ,
2022-09-09 06:14:49 -07:00
} ;
await job . progress ( progress ) ;
2021-11-05 09:45:51 -07:00
} ,
] ;
2022-09-09 06:14:49 -07:00
additionalData . executionId = executionId ;
2021-02-08 23:59:32 -08:00
2023-02-17 01:54:07 -08:00
additionalData . setExecutionStatus = ( status : ExecutionStatus ) = > {
// Can't set the status directly in the queued worker, but it will happen in InternalHook.onWorkflowPostExecute
2023-10-25 07:35:22 -07:00
this . logger . debug ( ` Queued worker execution status for ${ executionId } is " ${ status } " ` ) ;
2023-02-17 01:54:07 -08:00
} ;
2021-02-08 23:59:32 -08:00
let workflowExecute : WorkflowExecute ;
let workflowRun : PCancelable < IRun > ;
2023-06-20 10:13:18 -07:00
if ( fullExecutionData . data !== undefined ) {
2021-02-08 23:59:32 -08:00
workflowExecute = new WorkflowExecute (
additionalData ,
2023-06-20 10:13:18 -07:00
fullExecutionData . mode ,
fullExecutionData . data ,
2021-02-08 23:59:32 -08:00
) ;
workflowRun = workflowExecute . processRunExecutionData ( workflow ) ;
} else {
// Execute all nodes
// Can execute without webhook so go on
2023-06-20 10:13:18 -07:00
workflowExecute = new WorkflowExecute ( additionalData , fullExecutionData . mode ) ;
2021-02-08 23:59:32 -08:00
workflowRun = workflowExecute . run ( workflow ) ;
}
Worker . runningJobs [ job . id ] = workflowRun ;
2023-10-06 08:52:27 -07:00
Worker . runningJobsSummary [ job . id ] = {
jobId : job.id.toString ( ) ,
executionId ,
workflowId : fullExecutionData.workflowId ? ? '' ,
workflowName : fullExecutionData.workflowData.name ,
mode : fullExecutionData.mode ,
startedAt : fullExecutionData.startedAt ,
retryOf : fullExecutionData.retryOf ? ? '' ,
status : fullExecutionData.status ,
} ;
2021-02-08 23:59:32 -08:00
// Wait till the execution is finished
2021-10-21 16:24:43 -07:00
await workflowRun ;
2021-02-08 23:59:32 -08:00
delete Worker . runningJobs [ job . id ] ;
2023-10-06 08:52:27 -07:00
delete Worker . runningJobsSummary [ job . id ] ;
2021-02-08 23:59:32 -08:00
refactor(core): Move event and telemetry handling into workers in queue mode (#7138)
# Motivation
In Queue mode, finished executions would cause the main instance to
always pull all execution data from the database, unflatten it and then
use it to send out event log events and telemetry events, as well as
required returns to Respond to Webhook nodes etc.
This could cause OOM errors when the data was large, since it had to be
fully unpacked and transformed on the main instance’s side, using up a
lot of memory (and time).
This PR attempts to limit this behaviour to only happen in those
required cases where the data has to be forwarded to some waiting
webhook, for example.
# Changes
Execution data is only required in cases, where the active execution has
a `postExecutePromise` attached to it. These usually forward the data to
some other endpoint (e.g. a listening webhook connection).
By adding a helper `getPostExecutePromiseCount()`, we can decide that in
cases where there is nothing listening at all, there is no reason to
pull the data on the main instance.
Previously, there would always be postExecutePromises because the
telemetry events were called. Now, these have been moved into the
workers, which have been given the various InternalHooks calls to their
hook function arrays, so they themselves issue these telemetry and event
calls.
This results in all event log messages to now be logged on the worker’s
event log, as well as the worker’s eventbus being the one to send out
the events to destinations. The main event log does…pretty much nothing.
We are not logging executions on the main event log any more, because
this would require all events to be replicated 1:1 from the workers to
the main instance(s) (this IS possible and implemented, see the worker’s
`replicateToRedisEventLogFunction` - but it is not enabled to reduce the
amount of traffic over redis).
Partial events in the main log could confuse the recovery process and
would result in, ironically, the recovery corrupting the execution data
by considering them crashed.
# Refactor
I have also used the opportunity to reduce duplicate code and move some
of the hook functionality into
`packages/cli/src/executionLifecycleHooks/shared/sharedHookFunctions.ts`
in preparation for a future full refactor of the hooks
2023-09-13 22:58:15 -07:00
// do NOT call workflowExecuteAfter hook here, since it is being called from processSuccessExecution()
// already!
2021-02-08 23:59:32 -08:00
return {
success : true ,
} ;
}
2024-01-22 09:25:36 -08:00
constructor ( argv : string [ ] , cmdConfig : Config ) {
2023-09-26 04:58:06 -07:00
super ( argv , cmdConfig ) ;
2024-01-16 09:25:53 -08:00
if ( ! process . env . N8N_ENCRYPTION_KEY ) {
throw new ApplicationError (
2024-05-27 02:50:57 -07:00
'Missing encryption key. Worker started without the required N8N_ENCRYPTION_KEY env var. More information: https://docs.n8n.io/hosting/configuration/configuration-examples/encryption-key/' ,
2024-01-16 09:25:53 -08:00
) ;
}
2023-09-26 04:58:06 -07:00
this . setInstanceType ( 'worker' ) ;
this . setInstanceQueueModeId ( ) ;
}
2023-02-10 05:59:20 -08:00
async init() {
2024-02-01 02:10:40 -08:00
const { QUEUE_WORKER_TIMEOUT } = process . env ;
if ( QUEUE_WORKER_TIMEOUT ) {
this . gracefulShutdownTimeoutInS =
parseInt ( QUEUE_WORKER_TIMEOUT , 10 ) || config . default ( 'queue.bull.gracefulShutdownTimeout' ) ;
2023-12-18 04:04:19 -08:00
this . logger . warn (
'QUEUE_WORKER_TIMEOUT has been deprecated. Rename it to N8N_GRACEFUL_SHUTDOWN_TIMEOUT.' ,
) ;
}
2023-02-10 05:59:20 -08:00
await this . initCrashJournal ( ) ;
2023-09-26 04:58:06 -07:00
2023-02-10 05:59:20 -08:00
this . logger . debug ( 'Starting n8n worker...' ) ;
2023-09-26 04:58:06 -07:00
this . logger . debug ( ` Queue mode id: ${ this . queueModeId } ` ) ;
await super . init ( ) ;
await this . initLicense ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'License init complete' ) ;
2023-09-22 08:22:12 -07:00
await this . initBinaryDataService ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'Binary data service init complete' ) ;
2023-02-10 05:59:20 -08:00
await this . initExternalHooks ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'External hooks init complete' ) ;
2023-08-25 01:33:46 -07:00
await this . initExternalSecrets ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'External secrets init complete' ) ;
2023-09-07 05:44:19 -07:00
await this . initEventBus ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'Event bus init complete' ) ;
2023-09-07 05:44:19 -07:00
await this . initQueue ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'Queue init complete' ) ;
2023-10-06 04:58:11 -07:00
await this . initOrchestration ( ) ;
this . logger . debug ( 'Orchestration init complete' ) ;
2024-06-11 01:02:23 -07:00
await Container . get ( MessageEventBus ) . send (
2023-10-06 04:58:11 -07:00
new EventMessageGeneric ( {
eventName : 'n8n.worker.started' ,
payload : {
workerId : this.queueModeId ,
} ,
} ) ,
) ;
2023-02-10 05:59:20 -08:00
}
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
async initEventBus() {
2024-01-26 03:21:15 -08:00
await Container . get ( MessageEventBus ) . initialize ( {
2023-09-26 04:58:06 -07:00
workerId : this.queueModeId ,
2023-09-07 05:44:19 -07:00
} ) ;
2024-07-03 01:29:25 -07:00
Container . get ( AuditEventRelay ) . init ( ) ;
2023-09-07 05:44:19 -07:00
}
/ * *
* Initializes the redis connection
* A publishing connection to redis is created to publish events to the event log
* A subscription connection to redis is created to subscribe to commands from the main process
* The subscription connection adds a handler to handle the command messages
* /
2023-10-06 04:58:11 -07:00
async initOrchestration() {
await Container . get ( OrchestrationWorkerService ) . init ( ) ;
await Container . get ( OrchestrationHandlerWorkerService ) . initWithOptions ( {
queueModeId : this.queueModeId ,
redisPublisher : Container.get ( OrchestrationWorkerService ) . redisPublisher ,
getRunningJobIds : ( ) = > Object . keys ( Worker . runningJobs ) ,
2023-10-06 08:52:27 -07:00
getRunningJobsSummary : ( ) = > Object . values ( Worker . runningJobsSummary ) ,
2023-10-06 04:58:11 -07:00
} ) ;
2023-09-07 05:44:19 -07:00
}
async initQueue() {
2024-01-22 09:25:36 -08:00
const { flags } = await this . parse ( Worker ) ;
2021-02-08 23:59:32 -08:00
2023-02-10 05:59:20 -08:00
const redisConnectionTimeoutLimit = config . getEnv ( 'queue.bull.redis.timeoutThreshold' ) ;
2021-02-08 23:59:32 -08:00
2023-10-05 04:37:25 -07:00
this . logger . debug (
` Opening Redis connection to listen to messages with timeout ${ redisConnectionTimeoutLimit } ` ,
) ;
2023-10-13 02:53:59 -07:00
Worker . jobQueue = Container . get ( Queue ) ;
await Worker . jobQueue . init ( ) ;
2023-10-05 04:37:25 -07:00
this . logger . debug ( 'Queue singleton ready' ) ;
2024-06-12 06:05:43 -07:00
const envConcurrency = config . getEnv ( 'executions.concurrency.productionLimit' ) ;
const concurrency = envConcurrency !== - 1 ? envConcurrency : flags.concurrency ;
2024-01-17 07:08:50 -08:00
void Worker . jobQueue . process (
2024-06-12 06:05:43 -07:00
concurrency ,
2024-01-17 07:08:50 -08:00
async ( job ) = > await this . runJob ( job , this . nodeTypes ) ,
2023-05-23 17:01:45 -07:00
) ;
2023-02-10 05:59:20 -08:00
2023-10-13 02:53:59 -07:00
Worker . jobQueue . getBullObjectInstance ( ) . on ( 'global:progress' , ( jobId : JobId , progress ) = > {
2023-02-10 05:59:20 -08:00
// Progress of a job got updated which does get used
// to communicate that a job got canceled.
if ( progress === - 1 ) {
// Job has to get canceled
if ( Worker . runningJobs [ jobId ] !== undefined ) {
// Job is processed by current worker so cancel
Worker . runningJobs [ jobId ] . cancel ( ) ;
delete Worker . runningJobs [ jobId ] ;
}
}
} ) ;
2022-11-23 07:20:28 -08:00
2023-02-10 05:59:20 -08:00
let lastTimer = 0 ;
let cumulativeTimeout = 0 ;
2023-10-13 02:53:59 -07:00
Worker . jobQueue . getBullObjectInstance ( ) . on ( 'error' , ( error : Error ) = > {
2023-02-10 05:59:20 -08:00
if ( error . toString ( ) . includes ( 'ECONNREFUSED' ) ) {
const now = Date . now ( ) ;
if ( now - lastTimer > 30000 ) {
// Means we had no timeout at all or last timeout was temporary and we recovered
lastTimer = now ;
cumulativeTimeout = 0 ;
} else {
cumulativeTimeout += now - lastTimer ;
lastTimer = now ;
if ( cumulativeTimeout > redisConnectionTimeoutLimit ) {
this . logger . error (
` Unable to connect to Redis after ${ redisConnectionTimeoutLimit } . Exiting process. ` ,
) ;
process . exit ( 1 ) ;
}
}
this . logger . warn ( 'Redis unavailable - trying to reconnect...' ) ;
} else if ( error . toString ( ) . includes ( 'Error initializing Lua scripts' ) ) {
// This is a non-recoverable error
// Happens when worker starts and Redis is unavailable
// Even if Redis comes back online, worker will be zombie
this . logger . error ( 'Error initializing worker.' ) ;
process . exit ( 2 ) ;
} else {
this . logger . error ( 'Error from queue: ' , error ) ;
2024-04-05 11:03:49 -07:00
if ( error . message . includes ( 'job stalled more than maxStalledCount' ) ) {
throw new MaxStalledCountError ( error ) ;
}
2023-02-10 05:59:20 -08:00
throw error ;
}
} ) ;
2023-09-07 05:44:19 -07:00
}
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
async setupHealthMonitor() {
const port = config . getEnv ( 'queue.health.port' ) ;
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
const app = express ( ) ;
app . disable ( 'x-powered-by' ) ;
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
const server = http . createServer ( app ) ;
2021-02-09 14:32:40 -08:00
2023-09-07 05:44:19 -07:00
app . get (
'/healthz' ,
2023-07-31 02:00:48 -07:00
2024-05-31 05:06:13 -07:00
async ( _req : express.Request , res : express.Response ) = > {
2023-10-25 07:35:22 -07:00
this . logger . debug ( 'Health check started!' ) ;
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
const connection = Db . getConnection ( ) ;
2021-10-21 16:25:31 -07:00
2023-09-07 05:44:19 -07:00
try {
if ( ! connection . isInitialized ) {
// Connection is not active
2023-11-29 03:25:10 -08:00
throw new ApplicationError ( 'No active database connection' ) ;
2023-02-10 05:59:20 -08:00
}
2023-09-07 05:44:19 -07:00
// DB ping
await connection . query ( 'SELECT 1' ) ;
} catch ( e ) {
2023-10-25 07:35:22 -07:00
this . logger . error ( 'No Database connection!' , e as Error ) ;
2023-11-28 01:19:27 -08:00
const error = new ServiceUnavailableError ( 'No Database connection!' ) ;
2023-09-07 05:44:19 -07:00
return ResponseHelper . sendErrorResponse ( res , error ) ;
}
2022-01-07 08:14:59 -08:00
2023-09-07 05:44:19 -07:00
// Just to be complete, generally will the worker stop automatically
// if it loses the connection to redis
try {
// Redis ping
2023-10-13 02:53:59 -07:00
await Worker . jobQueue . ping ( ) ;
2023-09-07 05:44:19 -07:00
} catch ( e ) {
2023-10-25 07:35:22 -07:00
this . logger . error ( 'No Redis connection!' , e as Error ) ;
2023-11-28 01:19:27 -08:00
const error = new ServiceUnavailableError ( 'No Redis connection!' ) ;
2023-09-07 05:44:19 -07:00
return ResponseHelper . sendErrorResponse ( res , error ) ;
}
2021-12-23 13:29:04 -08:00
2023-09-07 05:44:19 -07:00
// Everything fine
const responseData = {
status : 'ok' ,
} ;
2021-02-08 23:59:32 -08:00
2023-10-25 07:35:22 -07:00
this . logger . debug ( 'Health check completed successfully!' ) ;
2021-02-08 23:59:32 -08:00
2023-09-07 05:44:19 -07:00
ResponseHelper . sendSuccessResponse ( res , responseData , true , 200 ) ;
} ,
) ;
2022-02-11 04:09:27 -08:00
2023-09-07 05:44:19 -07:00
let presetCredentialsLoaded = false ;
const endpointPresetCredentials = config . getEnv ( 'credentials.overwrite.endpoint' ) ;
if ( endpointPresetCredentials !== '' ) {
// POST endpoint to set preset credentials
app . post (
` / ${ endpointPresetCredentials } ` ,
rawBodyReader ,
bodyParser ,
async ( req : express.Request , res : express.Response ) = > {
if ( ! presetCredentialsLoaded ) {
const body = req . body as ICredentialsOverwrite ;
if ( req . contentType !== 'application/json' ) {
2023-08-17 06:42:57 -07:00
ResponseHelper . sendErrorResponse (
res ,
2023-09-07 05:44:19 -07:00
new Error (
'Body must be a valid JSON, make sure the content-type is application/json' ,
) ,
2023-08-17 06:42:57 -07:00
) ;
2023-09-07 05:44:19 -07:00
return ;
2023-08-17 06:42:57 -07:00
}
2023-09-07 05:44:19 -07:00
2023-10-09 07:09:23 -07:00
Container . get ( CredentialsOverwrites ) . setData ( body ) ;
2023-09-07 05:44:19 -07:00
presetCredentialsLoaded = true ;
ResponseHelper . sendSuccessResponse ( res , { success : true } , true , 200 ) ;
} else {
ResponseHelper . sendErrorResponse ( res , new Error ( 'Preset credentials can be set once' ) ) ;
}
} ,
) ;
}
server . on ( 'error' , ( error : Error & { code : string } ) = > {
if ( error . code === 'EADDRINUSE' ) {
this . logger . error (
` n8n's port ${ port } is already in use. Do you have the n8n main process running on that port? ` ,
2023-08-17 06:42:57 -07:00
) ;
2023-09-07 05:44:19 -07:00
process . exit ( 1 ) ;
2023-08-17 06:42:57 -07:00
}
2023-09-07 05:44:19 -07:00
} ) ;
2022-02-11 04:09:27 -08:00
2023-09-07 05:44:19 -07:00
await new Promise < void > ( ( resolve ) = > server . listen ( port , ( ) = > resolve ( ) ) ) ;
2023-12-17 23:23:10 -08:00
await this . externalHooks ? . run ( 'worker.ready' ) ;
2023-09-07 05:44:19 -07:00
this . logger . info ( ` \ nn8n worker health check via, port ${ port } ` ) ;
}
async run() {
2024-01-22 09:25:36 -08:00
const { flags } = await this . parse ( Worker ) ;
2023-08-17 06:42:57 -07:00
2023-09-07 05:44:19 -07:00
this . logger . info ( '\nn8n worker is now ready' ) ;
this . logger . info ( ` * Version: ${ N8N_VERSION } ` ) ;
this . logger . info ( ` * Concurrency: ${ flags . concurrency } ` ) ;
this . logger . info ( '' ) ;
if ( config . getEnv ( 'queue.health.active' ) ) {
await this . setupHealthMonitor ( ) ;
2023-02-10 05:59:20 -08:00
}
2023-02-11 07:46:10 -08:00
2024-05-31 06:41:00 -07:00
if ( process . stdout . isTTY ) {
process . stdin . setRawMode ( true ) ;
process . stdin . resume ( ) ;
process . stdin . setEncoding ( 'utf8' ) ;
process . stdin . on ( 'data' , ( key : string ) = > {
if ( key . charCodeAt ( 0 ) === 3 ) process . kill ( process . pid , 'SIGINT' ) ; // ctrl+c
} ) ;
}
2023-02-11 07:46:10 -08:00
// Make sure that the process does not close
2024-06-18 01:50:39 -07:00
if ( ! inTest ) await new Promise ( ( ) = > { } ) ;
2023-02-10 05:59:20 -08:00
}
async catch ( error : Error ) {
await this . exitWithCrash ( 'Worker exiting due to an error.' , error ) ;
2021-02-08 23:59:32 -08:00
}
}