fix: ensure the task concurrency is respected at queued task
[poolifier.git] / src / pools / abstract-pool.ts
index 8beede87c359e3559e41aa7652a007ee64eacaff..0d3d21832a715191e6efad9723c6a47454ae2215 100644 (file)
@@ -1,5 +1,6 @@
 import { randomUUID } from 'node:crypto'
 import { performance } from 'node:perf_hooks'
+import { existsSync } from 'node:fs'
 import type {
   MessageValue,
   PromiseResponseWrapper,
@@ -9,10 +10,12 @@ import {
   DEFAULT_TASK_NAME,
   DEFAULT_WORKER_CHOICE_STRATEGY_OPTIONS,
   EMPTY_FUNCTION,
+  isAsyncFunction,
   isKillBehavior,
   isPlainObject,
   median,
-  round
+  round,
+  updateMeasurementStatistics
 } from '../utils'
 import { KillBehaviors } from '../worker/worker-options'
 import {
@@ -28,12 +31,12 @@ import {
 import type {
   IWorker,
   IWorkerNode,
-  MessageHandler,
   WorkerInfo,
   WorkerType,
   WorkerUsage
 } from './worker'
 import {
+  type MeasurementStatisticsRequirements,
   Measurements,
   WorkerChoiceStrategies,
   type WorkerChoiceStrategy,
@@ -62,17 +65,15 @@ export abstract class AbstractPool<
   public readonly emitter?: PoolEmitter
 
   /**
-   * The execution response promise map.
+   * The task execution response promise map.
    *
    * - `key`: The message id of each submitted task.
    * - `value`: An object that contains the worker, the execution response promise resolve and reject callbacks.
    *
    * When we receive a message from the worker, we get a map entry with the promise resolve/reject bound to the message id.
    */
-  protected promiseResponseMap: Map<
-  string,
-  PromiseResponseWrapper<Worker, Response>
-  > = new Map<string, PromiseResponseWrapper<Worker, Response>>()
+  protected promiseResponseMap: Map<string, PromiseResponseWrapper<Response>> =
+    new Map<string, PromiseResponseWrapper<Response>>()
 
   /**
    * Worker choice strategy context referencing a worker choice algorithm implementation.
@@ -83,6 +84,10 @@ export abstract class AbstractPool<
   Response
   >
 
+  /**
+   * Whether the pool is starting or not.
+   */
+  private readonly starting: boolean
   /**
    * The start timestamp of the pool.
    */
@@ -110,6 +115,7 @@ export abstract class AbstractPool<
     this.chooseWorkerNode = this.chooseWorkerNode.bind(this)
     this.executeTask = this.executeTask.bind(this)
     this.enqueueTask = this.enqueueTask.bind(this)
+    this.dequeueTask = this.dequeueTask.bind(this)
     this.checkAndEmitEvents = this.checkAndEmitEvents.bind(this)
 
     if (this.opts.enableEvents === true) {
@@ -127,9 +133,9 @@ export abstract class AbstractPool<
 
     this.setupHook()
 
-    while (this.workerNodes.length < this.numberOfWorkers) {
-      this.createAndSetupWorker()
-    }
+    this.starting = true
+    this.startPool()
+    this.starting = false
 
     this.startTimestamp = performance.now()
   }
@@ -137,10 +143,14 @@ export abstract class AbstractPool<
   private checkFilePath (filePath: string): void {
     if (
       filePath == null ||
+      typeof filePath !== 'string' ||
       (typeof filePath === 'string' && filePath.trim().length === 0)
     ) {
       throw new Error('Please specify a file with a worker implementation')
     }
+    if (!existsSync(filePath)) {
+      throw new Error(`Cannot find the worker file '${filePath}'`)
+    }
   }
 
   private checkNumberOfWorkers (numberOfWorkers: number): void {
@@ -163,13 +173,21 @@ export abstract class AbstractPool<
 
   protected checkDynamicPoolSize (min: number, max: number): void {
     if (this.type === PoolTypes.dynamic) {
-      if (min > max) {
+      if (max == null) {
+        throw new Error(
+          'Cannot instantiate a dynamic pool without specifying the maximum pool size'
+        )
+      } else if (!Number.isSafeInteger(max)) {
+        throw new TypeError(
+          'Cannot instantiate a dynamic pool with a non safe integer maximum pool size'
+        )
+      } else if (min > max) {
         throw new RangeError(
           'Cannot instantiate a dynamic pool with a maximum pool size inferior to the minimum pool size'
         )
-      } else if (min === 0 && max === 0) {
+      } else if (max === 0) {
         throw new RangeError(
-          'Cannot instantiate a dynamic pool with a minimum pool size and a maximum pool size equal to zero'
+          'Cannot instantiate a dynamic pool with a pool size equal to zero'
         )
       } else if (min === max) {
         throw new RangeError(
@@ -268,6 +286,18 @@ export abstract class AbstractPool<
     }
   }
 
+  private startPool (): void {
+    while (
+      this.workerNodes.reduce(
+        (accumulator, workerNode) =>
+          !workerNode.info.dynamic ? accumulator + 1 : accumulator,
+        0
+      ) < this.numberOfWorkers
+    ) {
+      this.createAndSetupWorkerNode()
+    }
+  }
+
   /** @inheritDoc */
   public get info (): PoolInfo {
     return {
@@ -405,28 +435,28 @@ export abstract class AbstractPool<
     }
   }
 
-  private get starting (): boolean {
-    return (
-      this.workerNodes.length < this.minSize ||
-      (this.workerNodes.length >= this.minSize &&
-        this.workerNodes.some(workerNode => !workerNode.info.ready))
-    )
-  }
-
+  /**
+   * The pool readiness boolean status.
+   */
   private get ready (): boolean {
     return (
-      this.workerNodes.length >= this.minSize &&
-      this.workerNodes.every(workerNode => workerNode.info.ready)
+      this.workerNodes.reduce(
+        (accumulator, workerNode) =>
+          !workerNode.info.dynamic && workerNode.info.ready
+            ? accumulator + 1
+            : accumulator,
+        0
+      ) >= this.minSize
     )
   }
 
   /**
-   * Gets the approximate pool utilization.
+   * The approximate pool utilization.
    *
    * @returns The pool utilization.
    */
   private get utilization (): number {
-    const poolRunTimeCapacity =
+    const poolTimeCapacity =
       (performance.now() - this.startTimestamp) * this.maxSize
     const totalTasksRunTime = this.workerNodes.reduce(
       (accumulator, workerNode) =>
@@ -438,42 +468,31 @@ export abstract class AbstractPool<
         accumulator + (workerNode.usage.waitTime?.aggregate ?? 0),
       0
     )
-    return (totalTasksRunTime + totalTasksWaitTime) / poolRunTimeCapacity
+    return (totalTasksRunTime + totalTasksWaitTime) / poolTimeCapacity
   }
 
   /**
-   * Pool type.
+   * The pool type.
    *
    * If it is `'dynamic'`, it provides the `max` property.
    */
   protected abstract get type (): PoolType
 
   /**
-   * Gets the worker type.
+   * The worker type.
    */
   protected abstract get worker (): WorkerType
 
   /**
-   * Pool minimum size.
+   * The pool minimum size.
    */
   protected abstract get minSize (): number
 
   /**
-   * Pool maximum size.
+   * The pool maximum size.
    */
   protected abstract get maxSize (): number
 
-  /**
-   * Get the worker given its id.
-   *
-   * @param workerId - The worker id.
-   * @returns The worker if found in the pool worker nodes, `undefined` otherwise.
-   */
-  private getWorkerById (workerId: number): Worker | undefined {
-    return this.workerNodes.find(workerNode => workerNode.info.id === workerId)
-      ?.worker
-  }
-
   /**
    * Checks if the worker id sent in the received message from a worker is valid.
    *
@@ -483,7 +502,7 @@ export abstract class AbstractPool<
   private checkMessageWorkerId (message: MessageValue<Response>): void {
     if (
       message.workerId != null &&
-      this.getWorkerById(message.workerId) == null
+      this.getWorkerNodeKeyByWorkerId(message.workerId) == null
     ) {
       throw new Error(
         `Worker message received from unknown worker '${message.workerId}'`
@@ -503,6 +522,20 @@ export abstract class AbstractPool<
     )
   }
 
+  /**
+   * Gets the worker node key given its worker id.
+   *
+   * @param workerId - The worker id.
+   * @returns The worker node key if the worker id is found in the pool worker nodes, `undefined` otherwise.
+   */
+  private getWorkerNodeKeyByWorkerId (workerId: number): number | undefined {
+    for (const [workerNodeKey, workerNode] of this.workerNodes.entries()) {
+      if (workerNode.info.id === workerId) {
+        return workerNodeKey
+      }
+    }
+  }
+
   /** @inheritDoc */
   public setWorkerChoiceStrategy (
     workerChoiceStrategy: WorkerChoiceStrategy,
@@ -516,9 +549,9 @@ export abstract class AbstractPool<
     if (workerChoiceStrategyOptions != null) {
       this.setWorkerChoiceStrategyOptions(workerChoiceStrategyOptions)
     }
-    for (const workerNode of this.workerNodes) {
+    for (const [workerNodeKey, workerNode] of this.workerNodes.entries()) {
       workerNode.resetUsage()
-      this.setWorkerStatistics(workerNode.worker)
+      this.sendWorkerStatisticsMessageToWorker(workerNodeKey)
     }
   }
 
@@ -595,37 +628,35 @@ export abstract class AbstractPool<
 
   /** @inheritDoc */
   public async execute (data?: Data, name?: string): Promise<Response> {
-    const timestamp = performance.now()
-    const workerNodeKey = this.chooseWorkerNode()
-    const submittedTask: Task<Data> = {
-      name: name ?? DEFAULT_TASK_NAME,
-      // eslint-disable-next-line @typescript-eslint/consistent-type-assertions
-      data: data ?? ({} as Data),
-      timestamp,
-      workerId: this.getWorkerInfo(workerNodeKey).id as number,
-      id: randomUUID()
-    }
-    const res = new Promise<Response>((resolve, reject) => {
-      this.promiseResponseMap.set(submittedTask.id as string, {
+    return await new Promise<Response>((resolve, reject) => {
+      const timestamp = performance.now()
+      const workerNodeKey = this.chooseWorkerNode()
+      const task: Task<Data> = {
+        name: name ?? DEFAULT_TASK_NAME,
+        // eslint-disable-next-line @typescript-eslint/consistent-type-assertions
+        data: data ?? ({} as Data),
+        timestamp,
+        workerId: this.getWorkerInfo(workerNodeKey).id as number,
+        id: randomUUID()
+      }
+      this.promiseResponseMap.set(task.id as string, {
         resolve,
         reject,
-        worker: this.workerNodes[workerNodeKey].worker
+        workerNodeKey
       })
+      if (
+        this.opts.enableTasksQueue === true &&
+        (this.busy ||
+          this.workerNodes[workerNodeKey].usage.tasks.executing >=
+            ((this.opts.tasksQueueOptions as TasksQueueOptions)
+              .concurrency as number))
+      ) {
+        this.enqueueTask(workerNodeKey, task)
+      } else {
+        this.executeTask(workerNodeKey, task)
+      }
+      this.checkAndEmitEvents()
     })
-    if (
-      this.opts.enableTasksQueue === true &&
-      (this.busy ||
-        this.workerNodes[workerNodeKey].usage.tasks.executing >=
-          ((this.opts.tasksQueueOptions as TasksQueueOptions)
-            .concurrency as number))
-    ) {
-      this.enqueueTask(workerNodeKey, submittedTask)
-    } else {
-      this.executeTask(workerNodeKey, submittedTask)
-    }
-    this.checkAndEmitEvents()
-    // eslint-disable-next-line @typescript-eslint/return-await
-    return res
   }
 
   /** @inheritDoc */
@@ -639,18 +670,20 @@ export abstract class AbstractPool<
             resolve()
           })
         })
-        await this.destroyWorker(workerNode.worker)
+        await this.destroyWorkerNode(workerNodeKey)
         await workerExitPromise
       })
     )
   }
 
   /**
-   * Terminates the given worker.
+   * Terminates the worker node given its worker node key.
    *
-   * @param worker - A worker within `workerNodes`.
+   * @param workerNodeKey - The worker node key.
    */
-  protected abstract destroyWorker (worker: Worker): void | Promise<void>
+  protected abstract destroyWorkerNode (
+    workerNodeKey: number
+  ): void | Promise<void>
 
   /**
    * Setup hook to execute code before worker nodes are created in the abstract constructor.
@@ -692,14 +725,13 @@ export abstract class AbstractPool<
    * Hook executed after the worker task execution.
    * Can be overridden.
    *
-   * @param worker - The worker.
+   * @param workerNodeKey - The worker node key.
    * @param message - The received message.
    */
   protected afterTaskExecutionHook (
-    worker: Worker,
+    workerNodeKey: number,
     message: MessageValue<Response>
   ): void {
-    const workerNodeKey = this.getWorkerNodeKey(worker)
     const workerUsage = this.workerNodes[workerNodeKey].usage
     this.updateTaskStatisticsWorkerUsage(workerUsage, message)
     this.updateRunTimeWorkerUsage(workerUsage, message)
@@ -729,38 +761,12 @@ export abstract class AbstractPool<
     workerUsage: WorkerUsage,
     message: MessageValue<Response>
   ): void {
-    if (
-      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().runTime
-        .aggregate
-    ) {
-      const taskRunTime = message.taskPerformance?.runTime ?? 0
-      workerUsage.runTime.aggregate =
-        (workerUsage.runTime.aggregate ?? 0) + taskRunTime
-      workerUsage.runTime.minimum = Math.min(
-        taskRunTime,
-        workerUsage.runTime?.minimum ?? Infinity
-      )
-      workerUsage.runTime.maximum = Math.max(
-        taskRunTime,
-        workerUsage.runTime?.maximum ?? -Infinity
-      )
-      if (
-        this.workerChoiceStrategyContext.getTaskStatisticsRequirements().runTime
-          .average &&
-        workerUsage.tasks.executed !== 0
-      ) {
-        workerUsage.runTime.average =
-          workerUsage.runTime.aggregate / workerUsage.tasks.executed
-      }
-      if (
-        this.workerChoiceStrategyContext.getTaskStatisticsRequirements().runTime
-          .median &&
-        message.taskPerformance?.runTime != null
-      ) {
-        workerUsage.runTime.history.push(message.taskPerformance.runTime)
-        workerUsage.runTime.median = median(workerUsage.runTime.history)
-      }
-    }
+    updateMeasurementStatistics(
+      workerUsage.runTime,
+      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().runTime,
+      message.taskPerformance?.runTime ?? 0,
+      workerUsage.tasks.executed
+    )
   }
 
   private updateWaitTimeWorkerUsage (
@@ -769,53 +775,34 @@ export abstract class AbstractPool<
   ): void {
     const timestamp = performance.now()
     const taskWaitTime = timestamp - (task.timestamp ?? timestamp)
-    if (
-      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().waitTime
-        .aggregate
-    ) {
-      workerUsage.waitTime.aggregate =
-        (workerUsage.waitTime?.aggregate ?? 0) + taskWaitTime
-      workerUsage.waitTime.minimum = Math.min(
-        taskWaitTime,
-        workerUsage.waitTime?.minimum ?? Infinity
-      )
-      workerUsage.waitTime.maximum = Math.max(
-        taskWaitTime,
-        workerUsage.waitTime?.maximum ?? -Infinity
-      )
-      if (
-        this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
-          .waitTime.average &&
-        workerUsage.tasks.executed !== 0
-      ) {
-        workerUsage.waitTime.average =
-          workerUsage.waitTime.aggregate / workerUsage.tasks.executed
-      }
-      if (
-        this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
-          .waitTime.median
-      ) {
-        workerUsage.waitTime.history.push(taskWaitTime)
-        workerUsage.waitTime.median = median(workerUsage.waitTime.history)
-      }
-    }
+    updateMeasurementStatistics(
+      workerUsage.waitTime,
+      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().waitTime,
+      taskWaitTime,
+      workerUsage.tasks.executed
+    )
   }
 
   private updateEluWorkerUsage (
     workerUsage: WorkerUsage,
     message: MessageValue<Response>
   ): void {
-    if (
+    const eluTaskStatisticsRequirements: MeasurementStatisticsRequirements =
       this.workerChoiceStrategyContext.getTaskStatisticsRequirements().elu
-        .aggregate
-    ) {
+    updateMeasurementStatistics(
+      workerUsage.elu.active,
+      eluTaskStatisticsRequirements,
+      message.taskPerformance?.elu?.active ?? 0,
+      workerUsage.tasks.executed
+    )
+    updateMeasurementStatistics(
+      workerUsage.elu.idle,
+      eluTaskStatisticsRequirements,
+      message.taskPerformance?.elu?.idle ?? 0,
+      workerUsage.tasks.executed
+    )
+    if (eluTaskStatisticsRequirements.aggregate) {
       if (message.taskPerformance?.elu != null) {
-        workerUsage.elu.idle.aggregate =
-          (workerUsage.elu.idle?.aggregate ?? 0) +
-          message.taskPerformance.elu.idle
-        workerUsage.elu.active.aggregate =
-          (workerUsage.elu.active?.aggregate ?? 0) +
-          message.taskPerformance.elu.active
         if (workerUsage.elu.utilization != null) {
           workerUsage.elu.utilization =
             (workerUsage.elu.utilization +
@@ -824,43 +811,6 @@ export abstract class AbstractPool<
         } else {
           workerUsage.elu.utilization = message.taskPerformance.elu.utilization
         }
-        workerUsage.elu.idle.minimum = Math.min(
-          message.taskPerformance.elu.idle,
-          workerUsage.elu.idle?.minimum ?? Infinity
-        )
-        workerUsage.elu.idle.maximum = Math.max(
-          message.taskPerformance.elu.idle,
-          workerUsage.elu.idle?.maximum ?? -Infinity
-        )
-        workerUsage.elu.active.minimum = Math.min(
-          message.taskPerformance.elu.active,
-          workerUsage.elu.active?.minimum ?? Infinity
-        )
-        workerUsage.elu.active.maximum = Math.max(
-          message.taskPerformance.elu.active,
-          workerUsage.elu.active?.maximum ?? -Infinity
-        )
-        if (
-          this.workerChoiceStrategyContext.getTaskStatisticsRequirements().elu
-            .average &&
-          workerUsage.tasks.executed !== 0
-        ) {
-          workerUsage.elu.idle.average =
-            workerUsage.elu.idle.aggregate / workerUsage.tasks.executed
-          workerUsage.elu.active.average =
-            workerUsage.elu.active.aggregate / workerUsage.tasks.executed
-        }
-        if (
-          this.workerChoiceStrategyContext.getTaskStatisticsRequirements().elu
-            .median
-        ) {
-          workerUsage.elu.idle.history.push(message.taskPerformance.elu.idle)
-          workerUsage.elu.active.history.push(
-            message.taskPerformance.elu.active
-          )
-          workerUsage.elu.idle.median = median(workerUsage.elu.idle.history)
-          workerUsage.elu.active.median = median(workerUsage.elu.active.history)
-        }
       }
     }
   }
@@ -870,15 +820,15 @@ export abstract class AbstractPool<
    *
    * The default worker choice strategy uses a round robin algorithm to distribute the tasks.
    *
-   * @returns The worker node key
+   * @returns The chosen worker node key
    */
   private chooseWorkerNode (): number {
     if (this.shallCreateDynamicWorker()) {
-      const worker = this.createAndSetupDynamicWorker()
+      const workerNodeKey = this.createAndSetupDynamicWorkerNode()
       if (
         this.workerChoiceStrategyContext.getStrategyPolicy().useDynamicWorker
       ) {
-        return this.getWorkerNodeKey(worker)
+        return workerNodeKey
       }
     }
     return this.workerChoiceStrategyContext.execute()
@@ -894,29 +844,16 @@ export abstract class AbstractPool<
   }
 
   /**
-   * Sends a message to the given worker.
+   * Sends a message to worker given its worker node key.
    *
-   * @param worker - The worker which should receive the message.
+   * @param workerNodeKey - The worker node key.
    * @param message - The message.
    */
   protected abstract sendToWorker (
-    worker: Worker,
+    workerNodeKey: number,
     message: MessageValue<Data>
   ): void
 
-  /**
-   * Registers a listener callback on the given worker.
-   *
-   * @param worker - The worker which should register a listener.
-   * @param listener - The message listener callback.
-   */
-  private registerWorkerMessageListener<Message extends Data | Response>(
-    worker: Worker,
-    listener: (message: MessageValue<Message>) => void
-  ): void {
-    worker.on('message', listener as MessageHandler<Worker>)
-  }
-
   /**
    * Creates a new worker.
    *
@@ -925,46 +862,30 @@ export abstract class AbstractPool<
   protected abstract createWorker (): Worker
 
   /**
-   * Function that can be hooked up when a worker has been newly created and moved to the pool worker nodes.
-   * Can be overridden.
+   * Creates a new, completely set up worker node.
    *
-   * @param worker - The newly created worker.
+   * @returns New, completely set up worker node key.
    */
-  protected afterWorkerSetup (worker: Worker): void {
-    // Listen to worker messages.
-    this.registerWorkerMessageListener(worker, this.workerListener())
-    // Send startup message to worker.
-    this.sendToWorker(worker, {
-      ready: false,
-      workerId: this.getWorkerInfo(this.getWorkerNodeKey(worker)).id as number
-    })
-    // Setup worker task statistics computation.
-    this.setWorkerStatistics(worker)
-  }
-
-  /**
-   * Creates a new worker and sets it up completely in the pool worker nodes.
-   *
-   * @returns New, completely set up worker.
-   */
-  protected createAndSetupWorker (): Worker {
+  protected createAndSetupWorkerNode (): number {
     const worker = this.createWorker()
 
     worker.on('message', this.opts.messageHandler ?? EMPTY_FUNCTION)
     worker.on('error', this.opts.errorHandler ?? EMPTY_FUNCTION)
     worker.on('error', error => {
-      if (this.emitter != null) {
-        this.emitter.emit(PoolEvents.error, error)
-      }
+      const workerNodeKey = this.getWorkerNodeKey(worker)
+      const workerInfo = this.getWorkerInfo(workerNodeKey)
+      workerInfo.ready = false
+      this.workerNodes[workerNodeKey].closeChannel()
+      this.emitter?.emit(PoolEvents.error, error)
       if (this.opts.restartWorkerOnError === true && !this.starting) {
-        if (this.getWorkerInfo(this.getWorkerNodeKey(worker)).dynamic) {
-          this.createAndSetupDynamicWorker()
+        if (workerInfo.dynamic) {
+          this.createAndSetupDynamicWorkerNode()
         } else {
-          this.createAndSetupWorker()
+          this.createAndSetupWorkerNode()
         }
       }
       if (this.opts.enableTasksQueue === true) {
-        this.redistributeQueuedTasks(worker)
+        this.redistributeQueuedTasks(workerNodeKey)
       }
     })
     worker.on('online', this.opts.onlineHandler ?? EMPTY_FUNCTION)
@@ -973,83 +894,167 @@ export abstract class AbstractPool<
       this.removeWorkerNode(worker)
     })
 
-    this.pushWorkerNode(worker)
+    const workerNodeKey = this.addWorkerNode(worker)
 
-    this.afterWorkerSetup(worker)
+    this.afterWorkerNodeSetup(workerNodeKey)
 
-    return worker
+    return workerNodeKey
   }
 
-  private redistributeQueuedTasks (worker: Worker): void {
-    const workerNodeKey = this.getWorkerNodeKey(worker)
+  /**
+   * Creates a new, completely set up dynamic worker node.
+   *
+   * @returns New, completely set up dynamic worker node key.
+   */
+  protected createAndSetupDynamicWorkerNode (): number {
+    const workerNodeKey = this.createAndSetupWorkerNode()
+    this.registerWorkerMessageListener(workerNodeKey, message => {
+      const localWorkerNodeKey = this.getWorkerNodeKeyByWorkerId(
+        message.workerId
+      ) as number
+      const workerUsage = this.workerNodes[localWorkerNodeKey].usage
+      if (
+        isKillBehavior(KillBehaviors.HARD, message.kill) ||
+        (message.kill != null &&
+          ((this.opts.enableTasksQueue === false &&
+            workerUsage.tasks.executing === 0) ||
+            (this.opts.enableTasksQueue === true &&
+              workerUsage.tasks.executing === 0 &&
+              this.tasksQueueSize(localWorkerNodeKey) === 0)))
+      ) {
+        // Kill message received from the worker: no new tasks are submitted to that worker for a while ( > maxInactiveTime)
+        const destroyWorkerNodeBounded = this.destroyWorkerNode.bind(this)
+        if (isAsyncFunction(destroyWorkerNodeBounded)) {
+          (
+            destroyWorkerNodeBounded as (workerNodeKey: number) => Promise<void>
+          )(localWorkerNodeKey).catch(EMPTY_FUNCTION)
+        } else {
+          (destroyWorkerNodeBounded as (workerNodeKey: number) => void)(
+            localWorkerNodeKey
+          )
+        }
+      }
+    })
+    const workerInfo = this.getWorkerInfo(workerNodeKey)
+    workerInfo.dynamic = true
+    if (this.workerChoiceStrategyContext.getStrategyPolicy().useDynamicWorker) {
+      workerInfo.ready = true
+    }
+    this.sendToWorker(workerNodeKey, {
+      checkActive: true,
+      workerId: workerInfo.id as number
+    })
+    return workerNodeKey
+  }
+
+  /**
+   * Registers a listener callback on the worker given its worker node key.
+   *
+   * @param workerNodeKey - The worker node key.
+   * @param listener - The message listener callback.
+   */
+  protected abstract registerWorkerMessageListener<
+    Message extends Data | Response
+  >(
+    workerNodeKey: number,
+    listener: (message: MessageValue<Message>) => void
+  ): void
+
+  /**
+   * Method hooked up after a worker node has been newly created.
+   * Can be overridden.
+   *
+   * @param workerNodeKey - The newly created worker node key.
+   */
+  protected afterWorkerNodeSetup (workerNodeKey: number): void {
+    // Listen to worker messages.
+    this.registerWorkerMessageListener(workerNodeKey, this.workerListener())
+    // Send the startup message to worker.
+    this.sendStartupMessageToWorker(workerNodeKey)
+    // Send the worker statistics message to worker.
+    this.sendWorkerStatisticsMessageToWorker(workerNodeKey)
+  }
+
+  /**
+   * Sends the startup message to worker given its worker node key.
+   *
+   * @param workerNodeKey - The worker node key.
+   */
+  protected abstract sendStartupMessageToWorker (workerNodeKey: number): void
+
+  /**
+   * Sends the worker statistics message to worker given its worker node key.
+   *
+   * @param workerNodeKey - The worker node key.
+   */
+  private sendWorkerStatisticsMessageToWorker (workerNodeKey: number): void {
+    this.sendToWorker(workerNodeKey, {
+      statistics: {
+        runTime:
+          this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+            .runTime.aggregate,
+        elu: this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+          .elu.aggregate
+      },
+      workerId: this.getWorkerInfo(workerNodeKey).id as number
+    })
+  }
+
+  private redistributeQueuedTasks (workerNodeKey: number): void {
     while (this.tasksQueueSize(workerNodeKey) > 0) {
       let targetWorkerNodeKey: number = workerNodeKey
       let minQueuedTasks = Infinity
+      let executeTask = false
       for (const [workerNodeId, workerNode] of this.workerNodes.entries()) {
+        const workerInfo = this.getWorkerInfo(workerNodeId)
         if (
           workerNodeId !== workerNodeKey &&
+          workerInfo.ready &&
           workerNode.usage.tasks.queued === 0
         ) {
+          if (
+            this.workerNodes[workerNodeId].usage.tasks.executing <
+            (this.opts.tasksQueueOptions?.concurrency as number)
+          ) {
+            executeTask = true
+          }
           targetWorkerNodeKey = workerNodeId
           break
         }
         if (
           workerNodeId !== workerNodeKey &&
+          workerInfo.ready &&
           workerNode.usage.tasks.queued < minQueuedTasks
         ) {
           minQueuedTasks = workerNode.usage.tasks.queued
           targetWorkerNodeKey = workerNodeId
         }
       }
-      this.enqueueTask(
-        targetWorkerNodeKey,
-        this.dequeueTask(workerNodeKey) as Task<Data>
-      )
-    }
-  }
-
-  /**
-   * Creates a new dynamic worker and sets it up completely in the pool worker nodes.
-   *
-   * @returns New, completely set up dynamic worker.
-   */
-  protected createAndSetupDynamicWorker (): Worker {
-    const worker = this.createAndSetupWorker()
-    this.registerWorkerMessageListener(worker, message => {
-      const workerNodeKey = this.getWorkerNodeKey(worker)
-      if (
-        isKillBehavior(KillBehaviors.HARD, message.kill) ||
-        (message.kill != null &&
-          ((this.opts.enableTasksQueue === false &&
-            this.workerNodes[workerNodeKey].usage.tasks.executing === 0) ||
-            (this.opts.enableTasksQueue === true &&
-              this.workerNodes[workerNodeKey].usage.tasks.executing === 0 &&
-              this.tasksQueueSize(workerNodeKey) === 0)))
-      ) {
-        // Kill message received from the worker: no new tasks are submitted to that worker for a while ( > maxInactiveTime)
-        void (this.destroyWorker(worker) as Promise<void>)
+      if (executeTask) {
+        this.executeTask(
+          targetWorkerNodeKey,
+          this.dequeueTask(workerNodeKey) as Task<Data>
+        )
+      } else {
+        this.enqueueTask(
+          targetWorkerNodeKey,
+          this.dequeueTask(workerNodeKey) as Task<Data>
+        )
       }
-    })
-    const workerInfo = this.getWorkerInfo(this.getWorkerNodeKey(worker))
-    workerInfo.dynamic = true
-    this.sendToWorker(worker, {
-      checkAlive: true,
-      workerId: workerInfo.id as number
-    })
-    return worker
+    }
   }
 
   /**
-   * This function is the listener registered for each worker message.
+   * This method is the listener registered for each worker message.
    *
    * @returns The listener function to execute when a message is received from a worker.
    */
   protected workerListener (): (message: MessageValue<Response>) => void {
     return message => {
       this.checkMessageWorkerId(message)
-      if (message.ready != null && message.workerId != null) {
-        // Worker ready message received
-        this.handleWorkerReadyMessage(message)
+      if (message.ready != null) {
+        // Worker ready response received
+        this.handleWorkerReadyResponse(message)
       } else if (message.id != null) {
         // Task execution response received
         this.handleTaskExecutionResponse(message)
@@ -1057,10 +1062,10 @@ export abstract class AbstractPool<
     }
   }
 
-  private handleWorkerReadyMessage (message: MessageValue<Response>): void {
-    const worker = this.getWorkerById(message.workerId)
-    this.getWorkerInfo(this.getWorkerNodeKey(worker as Worker)).ready =
-      message.ready as boolean
+  private handleWorkerReadyResponse (message: MessageValue<Response>): void {
+    this.getWorkerInfo(
+      this.getWorkerNodeKeyByWorkerId(message.workerId) as number
+    ).ready = message.ready as boolean
     if (this.emitter != null && this.ready) {
       this.emitter.emit(PoolEvents.ready, this.info)
     }
@@ -1070,16 +1075,14 @@ export abstract class AbstractPool<
     const promiseResponse = this.promiseResponseMap.get(message.id as string)
     if (promiseResponse != null) {
       if (message.taskError != null) {
-        if (this.emitter != null) {
-          this.emitter.emit(PoolEvents.taskError, message.taskError)
-        }
+        this.emitter?.emit(PoolEvents.taskError, message.taskError)
         promiseResponse.reject(message.taskError.message)
       } else {
         promiseResponse.resolve(message.data as Response)
       }
-      this.afterTaskExecutionHook(promiseResponse.worker, message)
+      const workerNodeKey = promiseResponse.workerNodeKey
+      this.afterTaskExecutionHook(workerNodeKey, message)
       this.promiseResponseMap.delete(message.id as string)
-      const workerNodeKey = this.getWorkerNodeKey(promiseResponse.worker)
       if (
         this.opts.enableTasksQueue === true &&
         this.tasksQueueSize(workerNodeKey) > 0
@@ -1105,22 +1108,34 @@ export abstract class AbstractPool<
   }
 
   /**
-   * Gets the worker information.
+   * Gets the worker information given its worker node key.
    *
    * @param workerNodeKey - The worker node key.
+   * @returns The worker information.
    */
-  private getWorkerInfo (workerNodeKey: number): WorkerInfo {
+  protected getWorkerInfo (workerNodeKey: number): WorkerInfo {
     return this.workerNodes[workerNodeKey].info
   }
 
   /**
-   * Pushes the given worker in the pool worker nodes.
+   * Adds the given worker in the pool worker nodes.
    *
    * @param worker - The worker.
-   * @returns The worker nodes length.
+   * @returns The added worker node key.
+   * @throws {@link https://nodejs.org/api/errors.html#class-error} If the added worker node is not found.
    */
-  private pushWorkerNode (worker: Worker): number {
-    return this.workerNodes.push(new WorkerNode(worker, this.worker))
+  private addWorkerNode (worker: Worker): number {
+    const workerNode = new WorkerNode<Worker, Data>(worker, this.worker)
+    // Flag the worker node as ready at pool startup.
+    if (this.starting) {
+      workerNode.info.ready = true
+    }
+    this.workerNodes.push(workerNode)
+    const workerNodeKey = this.getWorkerNodeKey(worker)
+    if (workerNodeKey === -1) {
+      throw new Error('Worker node not found')
+    }
+    return workerNodeKey
   }
 
   /**
@@ -1136,9 +1151,15 @@ export abstract class AbstractPool<
     }
   }
 
+  /**
+   * Executes the given task on the worker given its worker node key.
+   *
+   * @param workerNodeKey - The worker node key.
+   * @param task - The task to execute.
+   */
   private executeTask (workerNodeKey: number, task: Task<Data>): void {
     this.beforeTaskExecutionHook(workerNodeKey, task)
-    this.sendToWorker(this.workerNodes[workerNodeKey].worker, task)
+    this.sendToWorker(workerNodeKey, task)
   }
 
   private enqueueTask (workerNodeKey: number, task: Task<Data>): number {
@@ -1168,17 +1189,4 @@ export abstract class AbstractPool<
       this.flushTasksQueue(workerNodeKey)
     }
   }
-
-  private setWorkerStatistics (worker: Worker): void {
-    this.sendToWorker(worker, {
-      statistics: {
-        runTime:
-          this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
-            .runTime.aggregate,
-        elu: this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
-          .elu.aggregate
-      },
-      workerId: this.getWorkerInfo(this.getWorkerNodeKey(worker)).id as number
-    })
-  }
 }