mirror of
https://github.com/Abdulazizzn/n8n-enterprise-unlocked.git
synced 2025-12-21 11:49:59 +00:00
refactor(core): Introduce RedisClientService (no-changelog) (#9774)
This commit is contained in:
@@ -1,5 +1,5 @@
|
|||||||
import type Bull from 'bull';
|
import type Bull from 'bull';
|
||||||
import { Service } from 'typedi';
|
import Container, { Service } from 'typedi';
|
||||||
import {
|
import {
|
||||||
ApplicationError,
|
ApplicationError,
|
||||||
BINARY_ENCODING,
|
BINARY_ENCODING,
|
||||||
@@ -8,14 +8,6 @@ import {
|
|||||||
type IExecuteResponsePromiseData,
|
type IExecuteResponsePromiseData,
|
||||||
} from 'n8n-workflow';
|
} from 'n8n-workflow';
|
||||||
import { ActiveExecutions } from '@/ActiveExecutions';
|
import { ActiveExecutions } from '@/ActiveExecutions';
|
||||||
|
|
||||||
import {
|
|
||||||
getRedisClusterClient,
|
|
||||||
getRedisClusterNodes,
|
|
||||||
getRedisPrefix,
|
|
||||||
getRedisStandardClient,
|
|
||||||
} from './services/redis/RedisServiceHelper';
|
|
||||||
import type { RedisClientType } from './services/redis/RedisServiceBaseClasses';
|
|
||||||
import config from '@/config';
|
import config from '@/config';
|
||||||
|
|
||||||
export type JobId = Bull.JobId;
|
export type JobId = Bull.JobId;
|
||||||
@@ -44,26 +36,18 @@ export class Queue {
|
|||||||
constructor(private activeExecutions: ActiveExecutions) {}
|
constructor(private activeExecutions: ActiveExecutions) {}
|
||||||
|
|
||||||
async init() {
|
async init() {
|
||||||
const bullPrefix = config.getEnv('queue.bull.prefix');
|
|
||||||
const prefix = getRedisPrefix(bullPrefix);
|
|
||||||
const clusterNodes = getRedisClusterNodes();
|
|
||||||
const usesRedisCluster = clusterNodes.length > 0;
|
|
||||||
|
|
||||||
const { default: Bull } = await import('bull');
|
const { default: Bull } = await import('bull');
|
||||||
|
const { RedisClientService } = await import('@/services/redis/redis-client.service');
|
||||||
|
|
||||||
|
const redisClientService = Container.get(RedisClientService);
|
||||||
|
|
||||||
|
const bullPrefix = config.getEnv('queue.bull.prefix');
|
||||||
|
const prefix = redisClientService.toValidPrefix(bullPrefix);
|
||||||
|
|
||||||
const { default: Redis } = await import('ioredis');
|
|
||||||
// Disabling ready check is necessary as it allows worker to
|
|
||||||
// quickly reconnect to Redis if Redis crashes or is unreachable
|
|
||||||
// for some time. With it enabled, worker might take minutes to realize
|
|
||||||
// redis is back up and resume working.
|
|
||||||
// More here: https://github.com/OptimalBits/bull/issues/890
|
|
||||||
this.jobQueue = new Bull('jobs', {
|
this.jobQueue = new Bull('jobs', {
|
||||||
prefix,
|
prefix,
|
||||||
settings: config.get('queue.bull.settings'),
|
settings: config.get('queue.bull.settings'),
|
||||||
createClient: (type, clientConfig) =>
|
createClient: (type) => redisClientService.createClient({ type: `${type}(bull)` }),
|
||||||
usesRedisCluster
|
|
||||||
? getRedisClusterClient(Redis, clientConfig, (type + '(bull)') as RedisClientType)
|
|
||||||
: getRedisStandardClient(Redis, clientConfig, (type + '(bull)') as RedisClientType),
|
|
||||||
});
|
});
|
||||||
|
|
||||||
this.jobQueue.on('global:progress', (_jobId, progress: WebhookResponse) => {
|
this.jobQueue.on('global:progress', (_jobId, progress: WebhookResponse) => {
|
||||||
|
|||||||
@@ -436,7 +436,7 @@ export const schema = {
|
|||||||
env: 'QUEUE_BULL_REDIS_PORT',
|
env: 'QUEUE_BULL_REDIS_PORT',
|
||||||
},
|
},
|
||||||
timeoutThreshold: {
|
timeoutThreshold: {
|
||||||
doc: 'Redis timeout threshold',
|
doc: 'Max cumulative timeout (in milliseconds) of connection retries before process exit',
|
||||||
format: Number,
|
format: Number,
|
||||||
default: 10000,
|
default: 10000,
|
||||||
env: 'QUEUE_BULL_REDIS_TIMEOUT_THRESHOLD',
|
env: 'QUEUE_BULL_REDIS_TIMEOUT_THRESHOLD',
|
||||||
|
|||||||
@@ -2,6 +2,7 @@
|
|||||||
|
|
||||||
import type { BinaryData } from 'n8n-core';
|
import type { BinaryData } from 'n8n-core';
|
||||||
import type { schema } from './schema';
|
import type { schema } from './schema';
|
||||||
|
import type { RedisOptions } from 'ioredis';
|
||||||
|
|
||||||
// -----------------------------------
|
// -----------------------------------
|
||||||
// transformers
|
// transformers
|
||||||
@@ -74,7 +75,7 @@ type ToReturnType<T extends ConfigOptionPath> = T extends NumericPath
|
|||||||
: unknown;
|
: unknown;
|
||||||
|
|
||||||
type ExceptionPaths = {
|
type ExceptionPaths = {
|
||||||
'queue.bull.redis': object;
|
'queue.bull.redis': RedisOptions;
|
||||||
binaryDataManager: BinaryData.Config;
|
binaryDataManager: BinaryData.Config;
|
||||||
'nodes.exclude': string[] | undefined;
|
'nodes.exclude': string[] | undefined;
|
||||||
'nodes.include': string[] | undefined;
|
'nodes.include': string[] | undefined;
|
||||||
|
|||||||
16
packages/cli/src/services/cache/cache.service.ts
vendored
16
packages/cli/src/services/cache/cache.service.ts
vendored
@@ -1,11 +1,10 @@
|
|||||||
import EventEmitter from 'node:events';
|
import EventEmitter from 'node:events';
|
||||||
|
|
||||||
import { Service } from 'typedi';
|
import Container, { Service } from 'typedi';
|
||||||
import { caching } from 'cache-manager';
|
import { caching } from 'cache-manager';
|
||||||
import { ApplicationError, jsonStringify } from 'n8n-workflow';
|
import { ApplicationError, jsonStringify } from 'n8n-workflow';
|
||||||
|
|
||||||
import config from '@/config';
|
import config from '@/config';
|
||||||
import { getDefaultRedisClient, getRedisPrefix } from '@/services/redis/RedisServiceHelper';
|
|
||||||
import { UncacheableValueError } from '@/errors/cache-errors/uncacheable-value.error';
|
import { UncacheableValueError } from '@/errors/cache-errors/uncacheable-value.error';
|
||||||
import { MalformedRefreshValueError } from '@/errors/cache-errors/malformed-refresh-value.error';
|
import { MalformedRefreshValueError } from '@/errors/cache-errors/malformed-refresh-value.error';
|
||||||
import type {
|
import type {
|
||||||
@@ -29,8 +28,17 @@ export class CacheService extends EventEmitter {
|
|||||||
const useRedis = backend === 'redis' || (backend === 'auto' && mode === 'queue');
|
const useRedis = backend === 'redis' || (backend === 'auto' && mode === 'queue');
|
||||||
|
|
||||||
if (useRedis) {
|
if (useRedis) {
|
||||||
const keyPrefix = `${getRedisPrefix()}:${config.getEnv('cache.redis.prefix')}:`;
|
const { RedisClientService } = await import('../redis/redis-client.service');
|
||||||
const redisClient = await getDefaultRedisClient({ keyPrefix }, 'client(cache)');
|
const redisClientService = Container.get(RedisClientService);
|
||||||
|
|
||||||
|
const prefixBase = config.getEnv('redis.prefix');
|
||||||
|
const cachePrefix = config.getEnv('cache.redis.prefix');
|
||||||
|
const prefix = redisClientService.toValidPrefix(`${prefixBase}:${cachePrefix}:`);
|
||||||
|
|
||||||
|
const redisClient = redisClientService.createClient({
|
||||||
|
type: 'client(cache)',
|
||||||
|
extraOptions: { keyPrefix: prefix },
|
||||||
|
});
|
||||||
|
|
||||||
const { redisStoreUsingClient } = await import('@/services/cache/redis.cache-manager');
|
const { redisStoreUsingClient } = await import('@/services/cache/redis.cache-manager');
|
||||||
const redisStore = redisStoreUsingClient(redisClient, { ttl });
|
const redisStore = redisStoreUsingClient(redisClient, { ttl });
|
||||||
|
|||||||
@@ -2,7 +2,7 @@ import { Container } from 'typedi';
|
|||||||
import { jsonParse } from 'n8n-workflow';
|
import { jsonParse } from 'n8n-workflow';
|
||||||
import { Logger } from '@/Logger';
|
import { Logger } from '@/Logger';
|
||||||
import type { RedisServiceCommandObject } from '../redis/RedisServiceCommands';
|
import type { RedisServiceCommandObject } from '../redis/RedisServiceCommands';
|
||||||
import { COMMAND_REDIS_CHANNEL } from '../redis/RedisServiceHelper';
|
import { COMMAND_REDIS_CHANNEL } from '../redis/RedisConstants';
|
||||||
import * as os from 'os';
|
import * as os from 'os';
|
||||||
|
|
||||||
export interface RedisServiceCommandLastReceived {
|
export interface RedisServiceCommandLastReceived {
|
||||||
|
|||||||
@@ -2,16 +2,17 @@ import { EventEmitter } from 'node:events';
|
|||||||
import config from '@/config';
|
import config from '@/config';
|
||||||
import { Service } from 'typedi';
|
import { Service } from 'typedi';
|
||||||
import { TIME } from '@/constants';
|
import { TIME } from '@/constants';
|
||||||
import { getRedisPrefix } from '@/services/redis/RedisServiceHelper';
|
|
||||||
import { ErrorReporterProxy as EventReporter } from 'n8n-workflow';
|
import { ErrorReporterProxy as EventReporter } from 'n8n-workflow';
|
||||||
import { Logger } from '@/Logger';
|
import { Logger } from '@/Logger';
|
||||||
import { RedisServicePubSubPublisher } from '@/services/redis/RedisServicePubSubPublisher';
|
import { RedisServicePubSubPublisher } from '@/services/redis/RedisServicePubSubPublisher';
|
||||||
|
import { RedisClientService } from '@/services/redis/redis-client.service';
|
||||||
|
|
||||||
@Service()
|
@Service()
|
||||||
export class MultiMainSetup extends EventEmitter {
|
export class MultiMainSetup extends EventEmitter {
|
||||||
constructor(
|
constructor(
|
||||||
private readonly logger: Logger,
|
private readonly logger: Logger,
|
||||||
private readonly redisPublisher: RedisServicePubSubPublisher,
|
private readonly redisPublisher: RedisServicePubSubPublisher,
|
||||||
|
private readonly redisClientService: RedisClientService,
|
||||||
) {
|
) {
|
||||||
super();
|
super();
|
||||||
}
|
}
|
||||||
@@ -20,13 +21,17 @@ export class MultiMainSetup extends EventEmitter {
|
|||||||
return config.getEnv('redis.queueModeId');
|
return config.getEnv('redis.queueModeId');
|
||||||
}
|
}
|
||||||
|
|
||||||
private readonly leaderKey = getRedisPrefix() + ':main_instance_leader';
|
private leaderKey: string;
|
||||||
|
|
||||||
private readonly leaderKeyTtl = config.getEnv('multiMainSetup.ttl');
|
private readonly leaderKeyTtl = config.getEnv('multiMainSetup.ttl');
|
||||||
|
|
||||||
private leaderCheckInterval: NodeJS.Timer | undefined;
|
private leaderCheckInterval: NodeJS.Timer | undefined;
|
||||||
|
|
||||||
async init() {
|
async init() {
|
||||||
|
const prefix = config.getEnv('redis.prefix');
|
||||||
|
const validPrefix = this.redisClientService.toValidPrefix(prefix);
|
||||||
|
this.leaderKey = validPrefix + ':main_instance_leader';
|
||||||
|
|
||||||
await this.tryBecomeLeader(); // prevent initial wait
|
await this.tryBecomeLeader(); // prevent initial wait
|
||||||
|
|
||||||
this.leaderCheckInterval = setInterval(
|
this.leaderCheckInterval = setInterval(
|
||||||
|
|||||||
@@ -1,8 +1,5 @@
|
|||||||
import { Service } from 'typedi';
|
import { Service } from 'typedi';
|
||||||
import {
|
import { COMMAND_REDIS_CHANNEL, WORKER_RESPONSE_REDIS_CHANNEL } from '../../redis/RedisConstants';
|
||||||
COMMAND_REDIS_CHANNEL,
|
|
||||||
WORKER_RESPONSE_REDIS_CHANNEL,
|
|
||||||
} from '../../redis/RedisServiceHelper';
|
|
||||||
import { handleWorkerResponseMessageMain } from './handleWorkerResponseMessageMain';
|
import { handleWorkerResponseMessageMain } from './handleWorkerResponseMessageMain';
|
||||||
import { handleCommandMessageMain } from './handleCommandMessageMain';
|
import { handleCommandMessageMain } from './handleCommandMessageMain';
|
||||||
import { OrchestrationHandlerService } from '../../orchestration.handler.base.service';
|
import { OrchestrationHandlerService } from '../../orchestration.handler.base.service';
|
||||||
|
|||||||
@@ -1,5 +1,5 @@
|
|||||||
import { Service } from 'typedi';
|
import { Service } from 'typedi';
|
||||||
import { COMMAND_REDIS_CHANNEL } from '../../redis/RedisServiceHelper';
|
import { COMMAND_REDIS_CHANNEL } from '../../redis/RedisConstants';
|
||||||
import { OrchestrationHandlerService } from '../../orchestration.handler.base.service';
|
import { OrchestrationHandlerService } from '../../orchestration.handler.base.service';
|
||||||
import { handleCommandMessageWebhook } from './handleCommandMessageWebhook';
|
import { handleCommandMessageWebhook } from './handleCommandMessageWebhook';
|
||||||
|
|
||||||
|
|||||||
@@ -1,7 +1,7 @@
|
|||||||
import { jsonParse } from 'n8n-workflow';
|
import { jsonParse } from 'n8n-workflow';
|
||||||
import Container from 'typedi';
|
import Container from 'typedi';
|
||||||
import type { RedisServiceCommandObject } from '@/services/redis/RedisServiceCommands';
|
import type { RedisServiceCommandObject } from '@/services/redis/RedisServiceCommands';
|
||||||
import { COMMAND_REDIS_CHANNEL } from '@/services/redis/RedisServiceHelper';
|
import { COMMAND_REDIS_CHANNEL } from '@/services/redis/RedisConstants';
|
||||||
import * as os from 'os';
|
import * as os from 'os';
|
||||||
import { License } from '@/License';
|
import { License } from '@/License';
|
||||||
import { MessageEventBus } from '@/eventbus/MessageEventBus/MessageEventBus';
|
import { MessageEventBus } from '@/eventbus/MessageEventBus/MessageEventBus';
|
||||||
|
|||||||
6
packages/cli/src/services/redis/RedisConstants.ts
Normal file
6
packages/cli/src/services/redis/RedisConstants.ts
Normal file
@@ -0,0 +1,6 @@
|
|||||||
|
export const EVENT_BUS_REDIS_STREAM = 'n8n:eventstream';
|
||||||
|
export const COMMAND_REDIS_STREAM = 'n8n:commandstream';
|
||||||
|
export const WORKER_RESPONSE_REDIS_STREAM = 'n8n:workerstream';
|
||||||
|
export const COMMAND_REDIS_CHANNEL = 'n8n.commands';
|
||||||
|
export const WORKER_RESPONSE_REDIS_CHANNEL = 'n8n.worker-response';
|
||||||
|
export const WORKER_RESPONSE_REDIS_LIST = 'n8n:list:worker-response';
|
||||||
@@ -3,7 +3,7 @@ import type { Cluster } from 'ioredis';
|
|||||||
import { Service } from 'typedi';
|
import { Service } from 'typedi';
|
||||||
import config from '@/config';
|
import config from '@/config';
|
||||||
import { Logger } from '@/Logger';
|
import { Logger } from '@/Logger';
|
||||||
import { getDefaultRedisClient } from './RedisServiceHelper';
|
import { RedisClientService } from './redis-client.service';
|
||||||
|
|
||||||
export type RedisClientType =
|
export type RedisClientType =
|
||||||
| 'subscriber'
|
| 'subscriber'
|
||||||
@@ -29,13 +29,16 @@ class RedisServiceBase {
|
|||||||
|
|
||||||
isInitialized = false;
|
isInitialized = false;
|
||||||
|
|
||||||
constructor(protected readonly logger: Logger) {}
|
constructor(
|
||||||
|
protected readonly logger: Logger,
|
||||||
|
private readonly redisClientService: RedisClientService,
|
||||||
|
) {}
|
||||||
|
|
||||||
async init(type: RedisClientType = 'client'): Promise<void> {
|
async init(type: RedisClientType = 'client'): Promise<void> {
|
||||||
if (this.redisClient && this.isInitialized) {
|
if (this.redisClient && this.isInitialized) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
this.redisClient = await getDefaultRedisClient(undefined, type);
|
this.redisClient = this.redisClientService.createClient({ type });
|
||||||
|
|
||||||
this.redisClient.on('close', () => {
|
this.redisClient.on('close', () => {
|
||||||
this.logger.warn('Redis unavailable - trying to reconnect...');
|
this.logger.warn('Redis unavailable - trying to reconnect...');
|
||||||
|
|||||||
@@ -1,151 +0,0 @@
|
|||||||
import type Redis from 'ioredis';
|
|
||||||
import type { Cluster, RedisOptions } from 'ioredis';
|
|
||||||
import config from '@/config';
|
|
||||||
import type { RedisClientType } from './RedisServiceBaseClasses';
|
|
||||||
import Container from 'typedi';
|
|
||||||
import { Logger } from '@/Logger';
|
|
||||||
|
|
||||||
export const EVENT_BUS_REDIS_STREAM = 'n8n:eventstream';
|
|
||||||
export const COMMAND_REDIS_STREAM = 'n8n:commandstream';
|
|
||||||
export const WORKER_RESPONSE_REDIS_STREAM = 'n8n:workerstream';
|
|
||||||
export const COMMAND_REDIS_CHANNEL = 'n8n.commands';
|
|
||||||
export const WORKER_RESPONSE_REDIS_CHANNEL = 'n8n.worker-response';
|
|
||||||
export const WORKER_RESPONSE_REDIS_LIST = 'n8n:list:worker-response';
|
|
||||||
|
|
||||||
export function getRedisClusterNodes(): Array<{ host: string; port: number }> {
|
|
||||||
const clusterNodePairs = config
|
|
||||||
.getEnv('queue.bull.redis.clusterNodes')
|
|
||||||
.split(',')
|
|
||||||
.filter((e) => e);
|
|
||||||
return clusterNodePairs.map((pair) => {
|
|
||||||
const [host, port] = pair.split(':');
|
|
||||||
return { host, port: parseInt(port) };
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
export function getRedisPrefix(customPrefix?: string): string {
|
|
||||||
let prefix = customPrefix ?? config.getEnv('redis.prefix');
|
|
||||||
if (prefix && getRedisClusterNodes().length > 0) {
|
|
||||||
if (!prefix.startsWith('{')) {
|
|
||||||
prefix = '{' + prefix;
|
|
||||||
}
|
|
||||||
if (!prefix.endsWith('}')) {
|
|
||||||
prefix += '}';
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return prefix;
|
|
||||||
}
|
|
||||||
|
|
||||||
export function getRedisStandardClient(
|
|
||||||
redis: typeof Redis,
|
|
||||||
redisOptions?: RedisOptions,
|
|
||||||
redisType?: RedisClientType,
|
|
||||||
): Redis | Cluster {
|
|
||||||
let lastTimer = 0;
|
|
||||||
let cumulativeTimeout = 0;
|
|
||||||
const { host, port, username, password, db }: RedisOptions = config.getEnv('queue.bull.redis');
|
|
||||||
const redisConnectionTimeoutLimit = config.getEnv('queue.bull.redis.timeoutThreshold');
|
|
||||||
const sharedRedisOptions: RedisOptions = {
|
|
||||||
...redisOptions,
|
|
||||||
host,
|
|
||||||
port,
|
|
||||||
username,
|
|
||||||
password,
|
|
||||||
db,
|
|
||||||
enableReadyCheck: false,
|
|
||||||
maxRetriesPerRequest: null,
|
|
||||||
};
|
|
||||||
if (config.getEnv('queue.bull.redis.tls')) sharedRedisOptions.tls = {};
|
|
||||||
|
|
||||||
const logger = Container.get(Logger);
|
|
||||||
logger.debug(
|
|
||||||
`Initialising Redis client${redisType ? ` of type ${redisType}` : ''} connection with host: ${
|
|
||||||
host ?? 'localhost'
|
|
||||||
} and port: ${port ?? '6379'}`,
|
|
||||||
);
|
|
||||||
return new redis({
|
|
||||||
...sharedRedisOptions,
|
|
||||||
retryStrategy: (): number | null => {
|
|
||||||
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) {
|
|
||||||
logger.error(
|
|
||||||
`Unable to connect to Redis after ${redisConnectionTimeoutLimit}. Exiting process.`,
|
|
||||||
);
|
|
||||||
process.exit(1);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return 500;
|
|
||||||
},
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
export function getRedisClusterClient(
|
|
||||||
redis: typeof Redis,
|
|
||||||
redisOptions?: RedisOptions,
|
|
||||||
redisType?: RedisClientType,
|
|
||||||
): Cluster {
|
|
||||||
let lastTimer = 0;
|
|
||||||
let cumulativeTimeout = 0;
|
|
||||||
const clusterNodes = getRedisClusterNodes();
|
|
||||||
const { username, password, db }: RedisOptions = config.getEnv('queue.bull.redis');
|
|
||||||
const redisConnectionTimeoutLimit = config.getEnv('queue.bull.redis.timeoutThreshold');
|
|
||||||
const sharedRedisOptions: RedisOptions = {
|
|
||||||
...redisOptions,
|
|
||||||
username,
|
|
||||||
password,
|
|
||||||
db,
|
|
||||||
enableReadyCheck: false,
|
|
||||||
maxRetriesPerRequest: null,
|
|
||||||
};
|
|
||||||
if (config.getEnv('queue.bull.redis.tls')) sharedRedisOptions.tls = {};
|
|
||||||
|
|
||||||
const logger = Container.get(Logger);
|
|
||||||
logger.debug(
|
|
||||||
`Initialising Redis cluster${
|
|
||||||
redisType ? ` of type ${redisType}` : ''
|
|
||||||
} connection with nodes: ${clusterNodes.map((e) => `${e.host}:${e.port}`).join(',')}`,
|
|
||||||
);
|
|
||||||
return new redis.Cluster(
|
|
||||||
clusterNodes.map((node) => ({ host: node.host, port: node.port })),
|
|
||||||
{
|
|
||||||
redisOptions: sharedRedisOptions,
|
|
||||||
clusterRetryStrategy: (): number | null => {
|
|
||||||
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) {
|
|
||||||
logger.error(
|
|
||||||
`Unable to connect to Redis after ${redisConnectionTimeoutLimit}. Exiting process.`,
|
|
||||||
);
|
|
||||||
process.exit(1);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return 500;
|
|
||||||
},
|
|
||||||
},
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
export async function getDefaultRedisClient(
|
|
||||||
additionalRedisOptions?: RedisOptions,
|
|
||||||
redisType?: RedisClientType,
|
|
||||||
): Promise<Redis | Cluster> {
|
|
||||||
const { default: Redis } = await import('ioredis');
|
|
||||||
const clusterNodes = getRedisClusterNodes();
|
|
||||||
const usesRedisCluster = clusterNodes.length > 0;
|
|
||||||
return usesRedisCluster
|
|
||||||
? getRedisClusterClient(Redis, additionalRedisOptions, redisType)
|
|
||||||
: getRedisStandardClient(Redis, additionalRedisOptions, redisType);
|
|
||||||
}
|
|
||||||
@@ -1,5 +1,5 @@
|
|||||||
import { Service } from 'typedi';
|
import { Service } from 'typedi';
|
||||||
import { COMMAND_REDIS_CHANNEL, WORKER_RESPONSE_REDIS_CHANNEL } from './RedisServiceHelper';
|
import { COMMAND_REDIS_CHANNEL, WORKER_RESPONSE_REDIS_CHANNEL } from './RedisConstants';
|
||||||
import type {
|
import type {
|
||||||
RedisServiceCommandObject,
|
RedisServiceCommandObject,
|
||||||
RedisServiceWorkerResponseObject,
|
RedisServiceWorkerResponseObject,
|
||||||
|
|||||||
@@ -1,5 +1,5 @@
|
|||||||
import { Service } from 'typedi';
|
import { Service } from 'typedi';
|
||||||
import { COMMAND_REDIS_CHANNEL, WORKER_RESPONSE_REDIS_CHANNEL } from './RedisServiceHelper';
|
import { COMMAND_REDIS_CHANNEL, WORKER_RESPONSE_REDIS_CHANNEL } from './RedisConstants';
|
||||||
import { RedisServiceBaseReceiver } from './RedisServiceBaseClasses';
|
import { RedisServiceBaseReceiver } from './RedisServiceBaseClasses';
|
||||||
|
|
||||||
@Service()
|
@Service()
|
||||||
|
|||||||
161
packages/cli/src/services/redis/redis-client.service.ts
Normal file
161
packages/cli/src/services/redis/redis-client.service.ts
Normal file
@@ -0,0 +1,161 @@
|
|||||||
|
import { Service } from 'typedi';
|
||||||
|
import config from '@/config';
|
||||||
|
import { Logger } from '@/Logger';
|
||||||
|
import ioRedis from 'ioredis';
|
||||||
|
import type { Cluster, RedisOptions } from 'ioredis';
|
||||||
|
import type { RedisClientType } from './RedisServiceBaseClasses';
|
||||||
|
import { OnShutdown } from '@/decorators/OnShutdown';
|
||||||
|
|
||||||
|
@Service()
|
||||||
|
export class RedisClientService {
|
||||||
|
private readonly clients = new Set<ioRedis | Cluster>();
|
||||||
|
|
||||||
|
constructor(private readonly logger: Logger) {}
|
||||||
|
|
||||||
|
createClient(arg: { type: RedisClientType; extraOptions?: RedisOptions }) {
|
||||||
|
const client =
|
||||||
|
this.clusterNodes().length > 0
|
||||||
|
? this.createClusterClient(arg)
|
||||||
|
: this.createRegularClient(arg);
|
||||||
|
|
||||||
|
this.clients.add(client);
|
||||||
|
|
||||||
|
return client;
|
||||||
|
}
|
||||||
|
|
||||||
|
@OnShutdown()
|
||||||
|
disconnectClients() {
|
||||||
|
for (const client of this.clients) {
|
||||||
|
client.disconnect();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Ensure prefix is wrapped in curly braces for Redis cluster.
|
||||||
|
* See: https://github.com/OptimalBits/bull/blob/develop/PATTERNS.md
|
||||||
|
*/
|
||||||
|
toValidPrefix(prefix: string) {
|
||||||
|
if (this.clusterNodes().length > 0) {
|
||||||
|
if (!prefix.startsWith('{')) prefix = '{' + prefix;
|
||||||
|
if (!prefix.endsWith('}')) prefix += '}';
|
||||||
|
}
|
||||||
|
|
||||||
|
return prefix;
|
||||||
|
}
|
||||||
|
|
||||||
|
// ----------------------------------
|
||||||
|
// private
|
||||||
|
// ----------------------------------
|
||||||
|
|
||||||
|
private createRegularClient({
|
||||||
|
type,
|
||||||
|
extraOptions,
|
||||||
|
}: {
|
||||||
|
type: RedisClientType;
|
||||||
|
extraOptions?: RedisOptions;
|
||||||
|
}) {
|
||||||
|
const options = this.getOptions({ extraOptions });
|
||||||
|
|
||||||
|
const { host, port } = config.getEnv('queue.bull.redis');
|
||||||
|
|
||||||
|
options.host = host;
|
||||||
|
options.port = port;
|
||||||
|
|
||||||
|
this.logger.debug('[Redis] Initializing regular client', { type, host, port });
|
||||||
|
|
||||||
|
return new ioRedis(options);
|
||||||
|
}
|
||||||
|
|
||||||
|
private createClusterClient({
|
||||||
|
type,
|
||||||
|
extraOptions,
|
||||||
|
}: {
|
||||||
|
type: string;
|
||||||
|
extraOptions?: RedisOptions;
|
||||||
|
}) {
|
||||||
|
const options = this.getOptions({ extraOptions });
|
||||||
|
|
||||||
|
const clusterNodes = this.clusterNodes();
|
||||||
|
|
||||||
|
this.logger.debug('[Redis] Initializing cluster client', { type, clusterNodes });
|
||||||
|
|
||||||
|
return new ioRedis.Cluster(clusterNodes, {
|
||||||
|
redisOptions: options,
|
||||||
|
clusterRetryStrategy: this.retryStrategy(),
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
private getOptions({ extraOptions }: { extraOptions?: RedisOptions }) {
|
||||||
|
const { username, password, db, tls } = config.getEnv('queue.bull.redis');
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Disabling ready check allows quick reconnection to Redis if Redis becomes
|
||||||
|
* temporarily unreachable. With ready check enabled, the client might take
|
||||||
|
* minutes to realize Redis is back up and resume working.
|
||||||
|
*
|
||||||
|
* See:
|
||||||
|
* - https://github.com/OptimalBits/bull/issues/890
|
||||||
|
* - https://github.com/OptimalBits/bull/issues/1873
|
||||||
|
* - https://github.com/OptimalBits/bull/pull/2185
|
||||||
|
*/
|
||||||
|
const options: RedisOptions = {
|
||||||
|
username,
|
||||||
|
password,
|
||||||
|
db,
|
||||||
|
enableReadyCheck: false,
|
||||||
|
maxRetriesPerRequest: null,
|
||||||
|
retryStrategy: this.retryStrategy(),
|
||||||
|
...extraOptions,
|
||||||
|
};
|
||||||
|
|
||||||
|
if (tls) options.tls = {}; // enable TLS with default Node.js settings
|
||||||
|
|
||||||
|
return options;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Strategy to retry connecting to Redis on connection failure.
|
||||||
|
*
|
||||||
|
* Try to reconnect every 500ms. On every failed attempt, increment a timeout
|
||||||
|
* counter - if the cumulative timeout exceeds a limit, exit the process.
|
||||||
|
* Reset the cumulative timeout if >30s between reconnection attempts.
|
||||||
|
*/
|
||||||
|
private retryStrategy() {
|
||||||
|
const RETRY_INTERVAL = 500; // ms
|
||||||
|
const RESET_LENGTH = 30_000; // ms
|
||||||
|
const MAX_TIMEOUT = config.getEnv('queue.bull.redis.timeoutThreshold');
|
||||||
|
|
||||||
|
let lastAttemptTs = 0;
|
||||||
|
let cumulativeTimeout = 0;
|
||||||
|
|
||||||
|
return () => {
|
||||||
|
const nowTs = Date.now();
|
||||||
|
|
||||||
|
if (nowTs - lastAttemptTs > RESET_LENGTH) {
|
||||||
|
cumulativeTimeout = 0;
|
||||||
|
lastAttemptTs = nowTs;
|
||||||
|
} else {
|
||||||
|
cumulativeTimeout += nowTs - lastAttemptTs;
|
||||||
|
lastAttemptTs = nowTs;
|
||||||
|
if (cumulativeTimeout > MAX_TIMEOUT) {
|
||||||
|
this.logger.error(`[Redis] Unable to connect after max timeout of ${MAX_TIMEOUT} ms`);
|
||||||
|
this.logger.error('Exiting process...');
|
||||||
|
process.exit(1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return RETRY_INTERVAL;
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
private clusterNodes() {
|
||||||
|
return config
|
||||||
|
.getEnv('queue.bull.redis.clusterNodes')
|
||||||
|
.split(',')
|
||||||
|
.filter((pair) => pair.trim().length > 0)
|
||||||
|
.map((pair) => {
|
||||||
|
const [host, port] = pair.split(':');
|
||||||
|
return { host, port: parseInt(port) };
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -14,6 +14,13 @@ import { Push } from '@/push';
|
|||||||
import { ActiveWorkflowManager } from '@/ActiveWorkflowManager';
|
import { ActiveWorkflowManager } from '@/ActiveWorkflowManager';
|
||||||
import { mockInstance } from '../../shared/mocking';
|
import { mockInstance } from '../../shared/mocking';
|
||||||
import type { WorkflowActivateMode } from 'n8n-workflow';
|
import type { WorkflowActivateMode } from 'n8n-workflow';
|
||||||
|
import { RedisClientService } from '@/services/redis/redis-client.service';
|
||||||
|
import type Redis from 'ioredis';
|
||||||
|
import { mock } from 'jest-mock-extended';
|
||||||
|
|
||||||
|
const redisClientService = mockInstance(RedisClientService);
|
||||||
|
const mockRedisClient = mock<Redis>();
|
||||||
|
redisClientService.createClient.mockReturnValue(mockRedisClient);
|
||||||
|
|
||||||
const os = Container.get(OrchestrationService);
|
const os = Container.get(OrchestrationService);
|
||||||
const handler = Container.get(OrchestrationHandlerMainService);
|
const handler = Container.get(OrchestrationHandlerMainService);
|
||||||
@@ -43,20 +50,6 @@ describe('Orchestration Service', () => {
|
|||||||
const eventBus = mockInstance(MessageEventBus);
|
const eventBus = mockInstance(MessageEventBus);
|
||||||
|
|
||||||
beforeAll(async () => {
|
beforeAll(async () => {
|
||||||
jest.mock('ioredis', () => {
|
|
||||||
const Redis = require('ioredis-mock');
|
|
||||||
if (typeof Redis === 'object') {
|
|
||||||
// the first mock is an ioredis shim because ioredis-mock depends on it
|
|
||||||
// https://github.com/stipsan/ioredis-mock/blob/master/src/index.js#L101-L111
|
|
||||||
return {
|
|
||||||
Command: { _transformer: { argument: {}, reply: {} } },
|
|
||||||
};
|
|
||||||
}
|
|
||||||
// second mock for our code
|
|
||||||
return function (...args: any) {
|
|
||||||
return new Redis(args);
|
|
||||||
};
|
|
||||||
});
|
|
||||||
jest.mock('@/services/redis/RedisServicePubSubPublisher', () => {
|
jest.mock('@/services/redis/RedisServicePubSubPublisher', () => {
|
||||||
return jest.fn().mockImplementation(() => {
|
return jest.fn().mockImplementation(() => {
|
||||||
return {
|
return {
|
||||||
|
|||||||
@@ -4,6 +4,21 @@ import config from '@/config';
|
|||||||
import { RedisService } from '@/services/redis.service';
|
import { RedisService } from '@/services/redis.service';
|
||||||
import { mockInstance } from '../../shared/mocking';
|
import { mockInstance } from '../../shared/mocking';
|
||||||
|
|
||||||
|
jest.mock('ioredis', () => {
|
||||||
|
const Redis = require('ioredis-mock');
|
||||||
|
if (typeof Redis === 'object') {
|
||||||
|
// the first mock is an ioredis shim because ioredis-mock depends on it
|
||||||
|
// https://github.com/stipsan/ioredis-mock/blob/master/src/index.js#L101-L111
|
||||||
|
return {
|
||||||
|
Command: { _transformer: { argument: {}, reply: {} } },
|
||||||
|
};
|
||||||
|
}
|
||||||
|
// second mock for our code
|
||||||
|
return function (...args: any) {
|
||||||
|
return new Redis(args);
|
||||||
|
};
|
||||||
|
});
|
||||||
|
|
||||||
mockInstance(Logger);
|
mockInstance(Logger);
|
||||||
const redisService = Container.get(RedisService);
|
const redisService = Container.get(RedisService);
|
||||||
|
|
||||||
@@ -15,20 +30,6 @@ const PUBSUB_CHANNEL = 'testchannel';
|
|||||||
|
|
||||||
describe('RedisService', () => {
|
describe('RedisService', () => {
|
||||||
beforeAll(async () => {
|
beforeAll(async () => {
|
||||||
jest.mock('ioredis', () => {
|
|
||||||
const Redis = require('ioredis-mock');
|
|
||||||
if (typeof Redis === 'object') {
|
|
||||||
// the first mock is an ioredis shim because ioredis-mock depends on it
|
|
||||||
// https://github.com/stipsan/ioredis-mock/blob/master/src/index.js#L101-L111
|
|
||||||
return {
|
|
||||||
Command: { _transformer: { argument: {}, reply: {} } },
|
|
||||||
};
|
|
||||||
}
|
|
||||||
// second mock for our code
|
|
||||||
return function (...args: any) {
|
|
||||||
return new Redis(args);
|
|
||||||
};
|
|
||||||
});
|
|
||||||
setDefaultConfig();
|
setDefaultConfig();
|
||||||
});
|
});
|
||||||
|
|
||||||
|
|||||||
Reference in New Issue
Block a user