fix: refine pool statuses handling
[poolifier.git] / src / pools / abstract-pool.ts
index 56be9a1a9ffe0ca22131b40d9308adab87b9894a..b72d8cbcb4c3354294dda16718a6d229e64d04e0 100644 (file)
@@ -10,7 +10,6 @@ import type {
 } from '../utility-types'
 import {
   DEFAULT_TASK_NAME,
-  DEFAULT_WORKER_CHOICE_STRATEGY_OPTIONS,
   EMPTY_FUNCTION,
   average,
   exponentialDelay,
@@ -43,7 +42,6 @@ import type {
   WorkerUsage
 } from './worker'
 import {
-  type MeasurementStatisticsRequirements,
   Measurements,
   WorkerChoiceStrategies,
   type WorkerChoiceStrategy,
@@ -56,7 +54,12 @@ import {
   checkFilePath,
   checkValidTasksQueueOptions,
   checkValidWorkerChoiceStrategy,
-  updateMeasurementStatistics
+  getDefaultTasksQueueOptions,
+  updateEluWorkerUsage,
+  updateRunTimeWorkerUsage,
+  updateTaskStatisticsWorkerUsage,
+  updateWaitTimeWorkerUsage,
+  waitWorkerNodeEvents
 } from './utils'
 
 /**
@@ -77,11 +80,6 @@ export abstract class AbstractPool<
   /** @inheritDoc */
   public emitter?: EventEmitterAsyncResource
 
-  /**
-   * Dynamic pool maximum size property placeholder.
-   */
-  protected readonly max?: number
-
   /**
    * The task execution response promise map:
    * - `key`: The message id of each submitted task.
@@ -132,22 +130,25 @@ export abstract class AbstractPool<
   /**
    * Constructs a new poolifier pool.
    *
-   * @param numberOfWorkers - Number of workers that this pool should manage.
+   * @param minimumNumberOfWorkers - Minimum number of workers that this pool manages.
    * @param filePath - Path to the worker file.
    * @param opts - Options for the pool.
+   * @param maximumNumberOfWorkers - Maximum number of workers that this pool manages.
    */
   public constructor (
-    protected readonly numberOfWorkers: number,
+    protected readonly minimumNumberOfWorkers: number,
     protected readonly filePath: string,
-    protected readonly opts: PoolOptions<Worker>
+    protected readonly opts: PoolOptions<Worker>,
+    protected readonly maximumNumberOfWorkers?: number
   ) {
     if (!this.isMain()) {
       throw new Error(
         'Cannot start a pool from a worker with the same type as the pool'
       )
     }
+    this.checkPoolType()
     checkFilePath(this.filePath)
-    this.checkNumberOfWorkers(this.numberOfWorkers)
+    this.checkMinimumNumberOfWorkers(this.minimumNumberOfWorkers)
     this.checkPoolOptions(this.opts)
 
     this.chooseWorkerNode = this.chooseWorkerNode.bind(this)
@@ -182,20 +183,28 @@ export abstract class AbstractPool<
     this.startTimestamp = performance.now()
   }
 
-  private checkNumberOfWorkers (numberOfWorkers: number): void {
-    if (numberOfWorkers == null) {
+  private checkPoolType (): void {
+    if (this.type === PoolTypes.fixed && this.maximumNumberOfWorkers != null) {
+      throw new Error(
+        'Cannot instantiate a fixed pool with a maximum number of workers specified at initialization'
+      )
+    }
+  }
+
+  private checkMinimumNumberOfWorkers (minimumNumberOfWorkers: number): void {
+    if (minimumNumberOfWorkers == null) {
       throw new Error(
         'Cannot instantiate a pool without specifying the number of workers'
       )
-    } else if (!Number.isSafeInteger(numberOfWorkers)) {
+    } else if (!Number.isSafeInteger(minimumNumberOfWorkers)) {
       throw new TypeError(
         'Cannot instantiate a pool with a non safe integer number of workers'
       )
-    } else if (numberOfWorkers < 0) {
+    } else if (minimumNumberOfWorkers < 0) {
       throw new RangeError(
         'Cannot instantiate a pool with a negative number of workers'
       )
-    } else if (this.type === PoolTypes.fixed && numberOfWorkers === 0) {
+    } else if (this.type === PoolTypes.fixed && minimumNumberOfWorkers === 0) {
       throw new RangeError('Cannot instantiate a fixed pool with zero worker')
     }
   }
@@ -211,9 +220,8 @@ export abstract class AbstractPool<
       this.checkValidWorkerChoiceStrategyOptions(
         opts.workerChoiceStrategyOptions as WorkerChoiceStrategyOptions
       )
-      this.opts.workerChoiceStrategyOptions = {
-        ...DEFAULT_WORKER_CHOICE_STRATEGY_OPTIONS,
-        ...opts.workerChoiceStrategyOptions
+      if (opts.workerChoiceStrategyOptions != null) {
+        this.opts.workerChoiceStrategyOptions = opts.workerChoiceStrategyOptions
       }
       this.opts.restartWorkerOnError = opts.restartWorkerOnError ?? true
       this.opts.enableEvents = opts.enableEvents ?? true
@@ -240,25 +248,10 @@ export abstract class AbstractPool<
         'Invalid worker choice strategy options: must be a plain object'
       )
     }
-    if (
-      workerChoiceStrategyOptions?.retries != null &&
-      !Number.isSafeInteger(workerChoiceStrategyOptions.retries)
-    ) {
-      throw new TypeError(
-        'Invalid worker choice strategy options: retries must be an integer'
-      )
-    }
-    if (
-      workerChoiceStrategyOptions?.retries != null &&
-      workerChoiceStrategyOptions.retries < 0
-    ) {
-      throw new RangeError(
-        `Invalid worker choice strategy options: retries '${workerChoiceStrategyOptions.retries}' must be greater or equal than zero`
-      )
-    }
     if (
       workerChoiceStrategyOptions?.weights != null &&
-      Object.keys(workerChoiceStrategyOptions.weights).length !== this.maxSize
+      Object.keys(workerChoiceStrategyOptions.weights).length !==
+        (this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers)
     ) {
       throw new Error(
         'Invalid worker choice strategy options: must have a weight for each worker node'
@@ -291,11 +284,11 @@ export abstract class AbstractPool<
       started: this.started,
       ready: this.ready,
       strategy: this.opts.workerChoiceStrategy as WorkerChoiceStrategy,
-      minSize: this.minSize,
-      maxSize: this.maxSize,
-      ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+      minSize: this.minimumNumberOfWorkers,
+      maxSize: this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers,
+      ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
         .runTime.aggregate &&
-        this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+        this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
           .waitTime.aggregate && { utilization: round(this.utilization) }),
       workerNodes: this.workerNodes.length,
       idleWorkerNodes: this.workerNodes.reduce(
@@ -305,6 +298,13 @@ export abstract class AbstractPool<
             : accumulator,
         0
       ),
+      ...(this.opts.enableTasksQueue === true && {
+        stealingWorkerNodes: this.workerNodes.reduce(
+          (accumulator, workerNode) =>
+            workerNode.info.stealing ? accumulator + 1 : accumulator,
+          0
+        )
+      }),
       busyWorkerNodes: this.workerNodes.reduce(
         (accumulator, _workerNode, workerNodeKey) =>
           this.isWorkerNodeBusy(workerNodeKey) ? accumulator + 1 : accumulator,
@@ -349,7 +349,7 @@ export abstract class AbstractPool<
           accumulator + workerNode.usage.tasks.failed,
         0
       ),
-      ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+      ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
         .runTime.aggregate && {
         runTime: {
           minimum: round(
@@ -366,7 +366,7 @@ export abstract class AbstractPool<
               )
             )
           ),
-          ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+          ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
             .runTime.average && {
             average: round(
               average(
@@ -378,7 +378,7 @@ export abstract class AbstractPool<
               )
             )
           }),
-          ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+          ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
             .runTime.median && {
             median: round(
               median(
@@ -392,7 +392,7 @@ export abstract class AbstractPool<
           })
         }
       }),
-      ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+      ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
         .waitTime.aggregate && {
         waitTime: {
           minimum: round(
@@ -409,7 +409,7 @@ export abstract class AbstractPool<
               )
             )
           ),
-          ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+          ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
             .waitTime.average && {
             average: round(
               average(
@@ -421,7 +421,7 @@ export abstract class AbstractPool<
               )
             )
           }),
-          ...(this.workerChoiceStrategyContext.getTaskStatisticsRequirements()
+          ...(this.workerChoiceStrategyContext?.getTaskStatisticsRequirements()
             .waitTime.median && {
             median: round(
               median(
@@ -449,7 +449,7 @@ export abstract class AbstractPool<
             ? accumulator + 1
             : accumulator,
         0
-      ) >= this.minSize
+      ) >= this.minimumNumberOfWorkers
     )
   }
 
@@ -460,7 +460,8 @@ export abstract class AbstractPool<
    */
   private get utilization (): number {
     const poolTimeCapacity =
-      (performance.now() - this.startTimestamp) * this.maxSize
+      (performance.now() - this.startTimestamp) *
+      (this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers)
     const totalTasksRunTime = this.workerNodes.reduce(
       (accumulator, workerNode) =>
         accumulator + (workerNode.usage.runTime?.aggregate ?? 0),
@@ -486,20 +487,6 @@ export abstract class AbstractPool<
    */
   protected abstract get worker (): WorkerType
 
-  /**
-   * The pool minimum size.
-   */
-  protected get minSize (): number {
-    return this.numberOfWorkers
-  }
-
-  /**
-   * The pool maximum size.
-   */
-  protected get maxSize (): number {
-    return this.max ?? this.numberOfWorkers
-  }
-
   /**
    * Checks if the worker id sent in the received message from a worker is valid.
    *
@@ -516,18 +503,6 @@ export abstract class AbstractPool<
     }
   }
 
-  /**
-   * Gets the given worker its worker node key.
-   *
-   * @param worker - The worker.
-   * @returns The worker node key if found in the pool worker nodes, `-1` otherwise.
-   */
-  private getWorkerNodeKeyByWorker (worker: Worker): number {
-    return this.workerNodes.findIndex(
-      workerNode => workerNode.worker === worker
-    )
-  }
-
   /**
    * Gets the worker node key given its worker id.
    *
@@ -564,11 +539,11 @@ export abstract class AbstractPool<
     workerChoiceStrategyOptions: WorkerChoiceStrategyOptions
   ): void {
     this.checkValidWorkerChoiceStrategyOptions(workerChoiceStrategyOptions)
-    this.opts.workerChoiceStrategyOptions = {
-      ...DEFAULT_WORKER_CHOICE_STRATEGY_OPTIONS,
-      ...workerChoiceStrategyOptions
+    if (workerChoiceStrategyOptions != null) {
+      this.opts.workerChoiceStrategyOptions = workerChoiceStrategyOptions
     }
     this.workerChoiceStrategyContext.setOptions(
+      this,
       this.opts.workerChoiceStrategyOptions
     )
   }
@@ -615,12 +590,9 @@ export abstract class AbstractPool<
     tasksQueueOptions: TasksQueueOptions
   ): TasksQueueOptions {
     return {
-      ...{
-        size: Math.pow(this.maxSize, 2),
-        concurrency: 1,
-        taskStealing: true,
-        tasksStealingOnBackPressure: true
-      },
+      ...getDefaultTasksQueueOptions(
+        this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers
+      ),
       ...tasksQueueOptions
     }
   }
@@ -673,7 +645,10 @@ export abstract class AbstractPool<
    * The pool filling boolean status.
    */
   protected get full (): boolean {
-    return this.workerNodes.length >= this.maxSize
+    return (
+      this.workerNodes.length >=
+      (this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers)
+    )
   }
 
   /**
@@ -981,7 +956,7 @@ export abstract class AbstractPool<
         (accumulator, workerNode) =>
           !workerNode.info.dynamic ? accumulator + 1 : accumulator,
         0
-      ) < this.numberOfWorkers
+      ) < this.minimumNumberOfWorkers
     ) {
       this.createAndSetupWorkerNode()
     }
@@ -1014,10 +989,12 @@ export abstract class AbstractPool<
     this.started = false
   }
 
-  protected async sendKillMessageToWorker (
-    workerNodeKey: number
-  ): Promise<void> {
+  private async sendKillMessageToWorker (workerNodeKey: number): Promise<void> {
     await new Promise<void>((resolve, reject) => {
+      if (this.workerNodes?.[workerNodeKey] == null) {
+        resolve()
+        return
+      }
       const killMessageListener = (message: MessageValue<Response>): void => {
         this.checkMessageWorkerId(message)
         if (message.kill === 'success') {
@@ -1043,7 +1020,22 @@ export abstract class AbstractPool<
    *
    * @param workerNodeKey - The worker node key.
    */
-  protected abstract destroyWorkerNode (workerNodeKey: number): Promise<void>
+  protected async destroyWorkerNode (workerNodeKey: number): Promise<void> {
+    this.flagWorkerNodeAsNotReady(workerNodeKey)
+    const flushedTasks = this.flushTasksQueue(workerNodeKey)
+    const workerNode = this.workerNodes[workerNodeKey]
+    await waitWorkerNodeEvents(
+      workerNode,
+      'taskFinished',
+      flushedTasks,
+      this.opts.tasksQueueOptions?.tasksFinishedTimeout ??
+        getDefaultTasksQueueOptions(
+          this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers
+        ).tasksFinishedTimeout
+    )
+    await this.sendKillMessageToWorker(workerNodeKey)
+    await workerNode.terminate()
+  }
 
   /**
    * Setup hook to execute code before worker nodes are created in the abstract constructor.
@@ -1074,7 +1066,11 @@ export abstract class AbstractPool<
     if (this.workerNodes[workerNodeKey]?.usage != null) {
       const workerUsage = this.workerNodes[workerNodeKey].usage
       ++workerUsage.tasks.executing
-      this.updateWaitTimeWorkerUsage(workerUsage, task)
+      updateWaitTimeWorkerUsage(
+        this.workerChoiceStrategyContext,
+        workerUsage,
+        task
+      )
     }
     if (
       this.shallUpdateTaskFunctionWorkerUsage(workerNodeKey) &&
@@ -1086,7 +1082,11 @@ export abstract class AbstractPool<
         workerNodeKey
       ].getTaskFunctionWorkerUsage(task.name as string) as WorkerUsage
       ++taskFunctionWorkerUsage.tasks.executing
-      this.updateWaitTimeWorkerUsage(taskFunctionWorkerUsage, task)
+      updateWaitTimeWorkerUsage(
+        this.workerChoiceStrategyContext,
+        taskFunctionWorkerUsage,
+        task
+      )
     }
   }
 
@@ -1101,11 +1101,21 @@ export abstract class AbstractPool<
     workerNodeKey: number,
     message: MessageValue<Response>
   ): void {
+    let needWorkerChoiceStrategyUpdate = false
     if (this.workerNodes[workerNodeKey]?.usage != null) {
       const workerUsage = this.workerNodes[workerNodeKey].usage
-      this.updateTaskStatisticsWorkerUsage(workerUsage, message)
-      this.updateRunTimeWorkerUsage(workerUsage, message)
-      this.updateEluWorkerUsage(workerUsage, message)
+      updateTaskStatisticsWorkerUsage(workerUsage, message)
+      updateRunTimeWorkerUsage(
+        this.workerChoiceStrategyContext,
+        workerUsage,
+        message
+      )
+      updateEluWorkerUsage(
+        this.workerChoiceStrategyContext,
+        workerUsage,
+        message
+      )
+      needWorkerChoiceStrategyUpdate = true
     }
     if (
       this.shallUpdateTaskFunctionWorkerUsage(workerNodeKey) &&
@@ -1118,9 +1128,21 @@ export abstract class AbstractPool<
       ].getTaskFunctionWorkerUsage(
         message.taskPerformance?.name as string
       ) as WorkerUsage
-      this.updateTaskStatisticsWorkerUsage(taskFunctionWorkerUsage, message)
-      this.updateRunTimeWorkerUsage(taskFunctionWorkerUsage, message)
-      this.updateEluWorkerUsage(taskFunctionWorkerUsage, message)
+      updateTaskStatisticsWorkerUsage(taskFunctionWorkerUsage, message)
+      updateRunTimeWorkerUsage(
+        this.workerChoiceStrategyContext,
+        taskFunctionWorkerUsage,
+        message
+      )
+      updateEluWorkerUsage(
+        this.workerChoiceStrategyContext,
+        taskFunctionWorkerUsage,
+        message
+      )
+      needWorkerChoiceStrategyUpdate = true
+    }
+    if (needWorkerChoiceStrategyUpdate) {
+      this.workerChoiceStrategyContext.update(workerNodeKey)
     }
   }
 
@@ -1139,84 +1161,6 @@ export abstract class AbstractPool<
     )
   }
 
-  private updateTaskStatisticsWorkerUsage (
-    workerUsage: WorkerUsage,
-    message: MessageValue<Response>
-  ): void {
-    const workerTaskStatistics = workerUsage.tasks
-    if (
-      workerTaskStatistics.executing != null &&
-      workerTaskStatistics.executing > 0
-    ) {
-      --workerTaskStatistics.executing
-    }
-    if (message.workerError == null) {
-      ++workerTaskStatistics.executed
-    } else {
-      ++workerTaskStatistics.failed
-    }
-  }
-
-  private updateRunTimeWorkerUsage (
-    workerUsage: WorkerUsage,
-    message: MessageValue<Response>
-  ): void {
-    if (message.workerError != null) {
-      return
-    }
-    updateMeasurementStatistics(
-      workerUsage.runTime,
-      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().runTime,
-      message.taskPerformance?.runTime ?? 0
-    )
-  }
-
-  private updateWaitTimeWorkerUsage (
-    workerUsage: WorkerUsage,
-    task: Task<Data>
-  ): void {
-    const timestamp = performance.now()
-    const taskWaitTime = timestamp - (task.timestamp ?? timestamp)
-    updateMeasurementStatistics(
-      workerUsage.waitTime,
-      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().waitTime,
-      taskWaitTime
-    )
-  }
-
-  private updateEluWorkerUsage (
-    workerUsage: WorkerUsage,
-    message: MessageValue<Response>
-  ): void {
-    if (message.workerError != null) {
-      return
-    }
-    const eluTaskStatisticsRequirements: MeasurementStatisticsRequirements =
-      this.workerChoiceStrategyContext.getTaskStatisticsRequirements().elu
-    updateMeasurementStatistics(
-      workerUsage.elu.active,
-      eluTaskStatisticsRequirements,
-      message.taskPerformance?.elu?.active ?? 0
-    )
-    updateMeasurementStatistics(
-      workerUsage.elu.idle,
-      eluTaskStatisticsRequirements,
-      message.taskPerformance?.elu?.idle ?? 0
-    )
-    if (eluTaskStatisticsRequirements.aggregate) {
-      if (message.taskPerformance?.elu != null) {
-        if (workerUsage.elu.utilization != null) {
-          workerUsage.elu.utilization =
-            (workerUsage.elu.utilization +
-              message.taskPerformance.elu.utilization) /
-            2
-        } else {
-          workerUsage.elu.utilization = message.taskPerformance.elu.utilization
-        }
-      }
-    }
-  }
-
   /**
    * Chooses a worker node for the next task.
    *
@@ -1241,9 +1185,7 @@ export abstract class AbstractPool<
    *
    * @returns Whether to create a dynamic worker or not.
    */
-  private shallCreateDynamicWorker (): boolean {
-    return this.type === PoolTypes.dynamic && !this.full && this.internalBusy()
-  }
+  protected abstract shallCreateDynamicWorker (): boolean
 
   /**
    * Sends a message to worker given its worker node key.
@@ -1278,33 +1220,36 @@ export abstract class AbstractPool<
       this.opts.errorHandler ?? EMPTY_FUNCTION
     )
     workerNode.registerWorkerEventHandler('error', (error: Error) => {
-      const workerNodeKey = this.getWorkerNodeKeyByWorker(workerNode.worker)
-      this.flagWorkerNodeAsNotReady(workerNodeKey)
-      const workerInfo = this.getWorkerInfo(workerNodeKey)
+      workerNode.info.ready = false
       this.emitter?.emit(PoolEvents.error, error)
-      this.workerNodes[workerNodeKey].closeChannel()
       if (
         this.started &&
-        !this.starting &&
         !this.destroying &&
         this.opts.restartWorkerOnError === true
       ) {
-        if (workerInfo.dynamic) {
+        if (workerNode.info.dynamic) {
           this.createAndSetupDynamicWorkerNode()
         } else {
           this.createAndSetupWorkerNode()
         }
       }
-      if (this.started && this.opts.enableTasksQueue === true) {
-        this.redistributeQueuedTasks(workerNodeKey)
+      if (
+        this.started &&
+        !this.destroying &&
+        this.opts.enableTasksQueue === true
+      ) {
+        this.redistributeQueuedTasks(this.workerNodes.indexOf(workerNode))
       }
+      workerNode?.terminate().catch(error => {
+        this.emitter?.emit(PoolEvents.error, error)
+      })
     })
     workerNode.registerWorkerEventHandler(
       'exit',
       this.opts.exitHandler ?? EMPTY_FUNCTION
     )
     workerNode.registerOnceWorkerEventHandler('exit', () => {
-      this.removeWorkerNode(workerNode.worker)
+      this.removeWorkerNode(workerNode)
     })
     const workerNodeKey = this.addWorkerNode(workerNode)
     this.afterWorkerNodeSetup(workerNodeKey)
@@ -1462,6 +1407,10 @@ export abstract class AbstractPool<
     })
   }
 
+  private cannotStealTask (): boolean {
+    return this.workerNodes.length <= 1 || this.info.queuedTasks === 0
+  }
+
   private handleTask (workerNodeKey: number, task: Task<Data>): void {
     if (this.shallExecuteTask(workerNodeKey)) {
       this.executeTask(workerNodeKey, task)
@@ -1471,7 +1420,10 @@ export abstract class AbstractPool<
   }
 
   private redistributeQueuedTasks (workerNodeKey: number): void {
-    if (this.workerNodes.length <= 1) {
+    if (workerNodeKey === -1) {
+      return
+    }
+    if (this.cannotStealTask()) {
       return
     }
     while (this.tasksQueueSize(workerNodeKey) > 0) {
@@ -1565,15 +1517,22 @@ export abstract class AbstractPool<
     eventDetail: WorkerNodeEventDetail,
     previousStolenTask?: Task<Data>
   ): void => {
-    if (this.workerNodes.length <= 1) {
-      return
-    }
     const { workerNodeKey } = eventDetail
     if (workerNodeKey == null) {
       throw new Error(
-        'WorkerNode event detail workerNodeKey attribute must be defined'
+        'WorkerNode event detail workerNodeKey property must be defined'
       )
     }
+    if (
+      this.cannotStealTask() ||
+      (this.info.stealingWorkerNodes as number) >
+        Math.floor(this.workerNodes.length / 2)
+    ) {
+      if (previousStolenTask != null) {
+        this.getWorkerInfo(workerNodeKey).stealing = false
+      }
+      return
+    }
     const workerNodeTasksUsage = this.workerNodes[workerNodeKey].usage.tasks
     if (
       previousStolenTask != null &&
@@ -1581,6 +1540,7 @@ export abstract class AbstractPool<
       (workerNodeTasksUsage.executing > 0 ||
         this.tasksQueueSize(workerNodeKey) > 0)
     ) {
+      this.getWorkerInfo(workerNodeKey).stealing = false
       for (const taskName of this.workerNodes[workerNodeKey].info
         .taskFunctionNames as string[]) {
         this.resetTaskSequentiallyStolenStatisticsTaskFunctionWorkerUsage(
@@ -1591,6 +1551,7 @@ export abstract class AbstractPool<
       this.resetTaskSequentiallyStolenStatisticsWorkerUsage(workerNodeKey)
       return
     }
+    this.getWorkerInfo(workerNodeKey).stealing = true
     const stolenTask = this.workerNodeStealTask(workerNodeKey)
     if (
       this.shallUpdateTaskFunctionWorkerUsage(workerNodeKey) &&
@@ -1637,6 +1598,7 @@ export abstract class AbstractPool<
     const sourceWorkerNode = workerNodes.find(
       (sourceWorkerNode, sourceWorkerNodeKey) =>
         sourceWorkerNode.info.ready &&
+        !sourceWorkerNode.info.stealing &&
         sourceWorkerNodeKey !== workerNodeKey &&
         sourceWorkerNode.usage.tasks.queued > 0
     )
@@ -1655,7 +1617,11 @@ export abstract class AbstractPool<
   private readonly handleBackPressureEvent = (
     eventDetail: WorkerNodeEventDetail
   ): void => {
-    if (this.workerNodes.length <= 1) {
+    if (
+      this.cannotStealTask() ||
+      (this.info.stealingWorkerNodes as number) >
+        Math.floor(this.workerNodes.length / 2)
+    ) {
       return
     }
     const { workerId } = eventDetail
@@ -1675,16 +1641,19 @@ export abstract class AbstractPool<
       if (
         sourceWorkerNode.usage.tasks.queued > 0 &&
         workerNode.info.ready &&
+        !workerNode.info.stealing &&
         workerNode.info.id !== workerId &&
         workerNode.usage.tasks.queued <
           (this.opts.tasksQueueOptions?.size as number) - sizeOffset
       ) {
+        this.getWorkerInfo(workerNodeKey).stealing = true
         const task = sourceWorkerNode.popTask() as Task<Data>
         this.handleTask(workerNodeKey, task)
         this.updateTaskStolenStatisticsWorkerUsage(
           workerNodeKey,
           task.name as string
         )
+        this.getWorkerInfo(workerNodeKey).stealing = false
       }
     }
   }
@@ -1732,6 +1701,7 @@ export abstract class AbstractPool<
     const promiseResponse = this.promiseResponseMap.get(taskId as string)
     if (promiseResponse != null) {
       const { resolve, reject, workerNodeKey, asyncResource } = promiseResponse
+      const workerNode = this.workerNodes[workerNodeKey]
       if (workerError != null) {
         this.emitter?.emit(PoolEvents.taskError, workerError)
         asyncResource != null
@@ -1748,10 +1718,10 @@ export abstract class AbstractPool<
       }
       asyncResource?.emitDestroy()
       this.afterTaskExecutionHook(workerNodeKey, message)
-      this.workerChoiceStrategyContext.update(workerNodeKey)
       this.promiseResponseMap.delete(taskId as string)
-      if (this.opts.enableTasksQueue === true) {
-        const workerNodeTasksUsage = this.workerNodes[workerNodeKey].usage.tasks
+      workerNode?.emit('taskFinished', taskId)
+      if (this.opts.enableTasksQueue === true && !this.destroying) {
+        const workerNodeTasksUsage = workerNode.usage.tasks
         if (
           this.tasksQueueSize(workerNodeKey) > 0 &&
           workerNodeTasksUsage.executing <
@@ -1767,7 +1737,7 @@ export abstract class AbstractPool<
           this.tasksQueueSize(workerNodeKey) === 0 &&
           workerNodeTasksUsage.sequentiallyStolen === 0
         ) {
-          this.workerNodes[workerNodeKey].emit('idleWorkerNode', {
+          workerNode.emit('idleWorkerNode', {
             workerId: workerId as number,
             workerNodeKey
           })
@@ -1788,13 +1758,10 @@ export abstract class AbstractPool<
     }
   }
 
-  private checkAndEmitDynamicWorkerCreationEvents (): void {
-    if (this.type === PoolTypes.dynamic) {
-      if (this.full) {
-        this.emitter?.emit(PoolEvents.full, this.info)
-      }
-    }
-  }
+  /**
+   * Emits dynamic worker creation events.
+   */
+  protected abstract checkAndEmitDynamicWorkerCreationEvents (): void
 
   /**
    * Gets the worker information given its worker node key.
@@ -1819,7 +1786,10 @@ export abstract class AbstractPool<
         env: this.opts.env,
         workerOptions: this.opts.workerOptions,
         tasksQueueBackPressureSize:
-          this.opts.tasksQueueOptions?.size ?? Math.pow(this.maxSize, 2)
+          this.opts.tasksQueueOptions?.size ??
+          getDefaultTasksQueueOptions(
+            this.maximumNumberOfWorkers ?? this.minimumNumberOfWorkers
+          ).size
       }
     )
     // Flag the worker node as ready at pool startup.
@@ -1846,12 +1816,12 @@ export abstract class AbstractPool<
   }
 
   /**
-   * Removes the worker node associated to the give given worker from the pool worker nodes.
+   * Removes the worker node from the pool worker nodes.
    *
-   * @param worker - The worker.
+   * @param workerNode - The worker node.
    */
-  private removeWorkerNode (worker: Worker): void {
-    const workerNodeKey = this.getWorkerNodeKeyByWorker(worker)
+  private removeWorkerNode (workerNode: IWorkerNode<Worker, Data>): void {
+    const workerNodeKey = this.workerNodes.indexOf(workerNode)
     if (workerNodeKey !== -1) {
       this.workerNodes.splice(workerNodeKey, 1)
       this.workerChoiceStrategyContext.remove(workerNodeKey)
@@ -1905,14 +1875,17 @@ export abstract class AbstractPool<
     return this.workerNodes[workerNodeKey].tasksQueueSize()
   }
 
-  protected flushTasksQueue (workerNodeKey: number): void {
+  protected flushTasksQueue (workerNodeKey: number): number {
+    let flushedTasks = 0
     while (this.tasksQueueSize(workerNodeKey) > 0) {
       this.executeTask(
         workerNodeKey,
         this.dequeueTask(workerNodeKey) as Task<Data>
       )
+      ++flushedTasks
     }
     this.workerNodes[workerNodeKey].clearTasksQueue()
+    return flushedTasks
   }
 
   private flushTasksQueues (): void {