From 79513242d689cb85ca629841268416cc19626806 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 20 Aug 2024 16:01:33 -0700 Subject: [PATCH 01/30] initial commit for shared_process_pool --- .../morpheus/utils/shared_process_pool.py | 134 ++++++++++++++++++ 1 file changed, 134 insertions(+) create mode 100644 python/morpheus/morpheus/utils/shared_process_pool.py diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py new file mode 100644 index 0000000000..2b35ee1825 --- /dev/null +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -0,0 +1,134 @@ +import logging +import multiprocessing as mp +import math +import os +import queue +from concurrent.futures import Future +from threading import Lock, Semaphore +import time + +logger = logging.getLogger(__name__) + + +class SharedProcessPool: + + _instance = None + _lock = Lock() + _total_workers = 0 + + def __new__(cls): + """ + Initialize as a singleton. + """ + logger.debug("Creating a new instance of SharedProcessPool...") + with cls._lock: + if cls._instance is None: + cls._instance = super().__new__(cls) + max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.4)) + cls._instance._initialize(max_workers) + logger.debug("SharedProcessPool has been initialized with %s workers.", max_workers) + + else: + logger.debug("SharedProcessPool instance already exists.") + + return cls._instance + + def _initialize(self, total_max_workers): + """ + Initialize a concurrent.futures.ProcessPoolExecutor instance. + """ + + self._total_max_workers = total_max_workers + self._context = mp.get_context("fork") + self._total_usage = 0.0 # the percentage of processes that's currently in use + self._stage_usage = {} # maintain the percentage of processes used by each stage + self._task_queues = {} # maintain a separate task queue for each stage + self._stage_semaphores = {} # maintain a semaphore for each stage + self._processes = [] + + for i in range(total_max_workers): + process = self._context.Process(target=self._worker) + process.start() + self._processes.append(process) + logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) + + def _worker(self): + logger.debug("Worker process %s has been started.", os.getpid()) + + while True: + # iterate over every semaphore + for stage_name, task_queue in self._task_queues.items(): + semaphore = self._stage_semaphores[stage_name] + + if not semaphore.acquire(blocking=False): + # Stage has reached the limitation of processes + continue + + try: + task = task_queue.get_nowait() + except queue.Empty(): + semaphore.release() + continue + + + if task is None: # Stop signal + semaphore.release() + return + + process_fn, args, future = task + try: + result = process_fn(*args) + future.set_result(result) + except Exception as e: + future.set_exception(e) + + semaphore.release() + + time.sleep(0.1) # avoid busy-waiting + + def submit_task(self, stage_name, process_fn, *args): + """ + Submit a task to the corresponding task queue of the stage. + """ + future = Future() + task = (process_fn, args, {}, future) + self._task_queues[stage_name].put(task) + + return future + + def set_usage(self, stage_name, percentage): + """ + Set the maximum percentage of processes that can be used by each stage. + """ + if not 0 < percentage <= 1: + raise ValueError("Percentage must be between 0 and 1.") + + new_total_usage = self._total_usage - self._stage_usage.get(stage_name, 0.0) + percentage + + if new_total_usage > 1.0: + raise ValueError("Total percentage cannot exceed 1.0.") + + self._stage_usage[stage_name] = percentage + self._total_usage = new_total_usage + + allowed_processes_num = max(1, int(self._total_max_workers * percentage)) + self._stage_semaphores[stage_name] = Semaphore(allowed_processes_num) + + if stage_name not in self._task_queues: + self._task_queues[stage_name] = self._context.Queue() + + def shutdown(self): + + for stage_name, task_queue in self._task_queues.items(): + for _ in range(self._total_max_workers): + task_queue.put(None) + logger.debug("Task queue for stage %s has been cleared.", stage_name) + + for i, p in enumerate(self._processes): + p.join() + logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + + logger.debug("Process pool has been terminated.") + + def __del__(self): + self.shutdown() From f8037afa3590f97f8801ae044e5cf543401a5d6d Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 20 Aug 2024 16:25:50 -0700 Subject: [PATCH 02/30] fix CI --- .../morpheus/utils/shared_process_pool.py | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 2b35ee1825..ac85f6cf68 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -1,11 +1,27 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024, NVIDIA CORPORATION & AFFILIATES. All rights reserved. +# SPDX-License-Identifier: Apache-2.0 +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + import logging -import multiprocessing as mp import math +import multiprocessing as mp import os import queue -from concurrent.futures import Future -from threading import Lock, Semaphore import time +from concurrent.futures import Future +from threading import Lock +from threading import Semaphore logger = logging.getLogger(__name__) @@ -33,6 +49,16 @@ def __new__(cls): return cls._instance + def __init__(self): + # Declare the attributes here to avoid W0201: attribute-defined-outside-init + self._total_max_workers = None + self._context = None + self._total_usage = 0.0 + self._stage_usage = {} + self._task_queues = {} + self._stage_semaphores = {} + self._processes = [] + def _initialize(self, total_max_workers): """ Initialize a concurrent.futures.ProcessPoolExecutor instance. @@ -70,8 +96,7 @@ def _worker(self): semaphore.release() continue - - if task is None: # Stop signal + if task is None: # Stop signal semaphore.release() return From 1443d11f5ae829c2470d5eed1fac3eef6ddee35f Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 22 Aug 2024 10:52:19 -0700 Subject: [PATCH 03/30] Add performance verification tests to shared_process_pool --- .../morpheus/utils/shared_process_pool.py | 72 ++++++++++++------- tests/utils/test_shared_process_pool.py | 59 +++++++++++++++ 2 files changed, 104 insertions(+), 27 deletions(-) create mode 100644 tests/utils/test_shared_process_pool.py diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index ac85f6cf68..8bfc9e58e2 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -19,18 +19,39 @@ import os import queue import time -from concurrent.futures import Future from threading import Lock -from threading import Semaphore +# logging.basicConfig(level=logging.DEBUG, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') logger = logging.getLogger(__name__) +class SerializableFuture: + + def __init__(self, manager): + self._result = manager.Value("i", None) + self._exception = manager.Value("i", None) + self._done = manager.Event() + + def set_result(self, result): + self._result.value = result + self._done.set() + + def set_exception(self, exception): + self._exception.value = exception + self._done.set() + + def result(self): + self._done.wait() + if self._exception.value is not None: + raise self._exception.value + return self._result.value + + class SharedProcessPool: _instance = None _lock = Lock() - _total_workers = 0 + _shutdown = False def __new__(cls): """ @@ -40,7 +61,7 @@ def __new__(cls): with cls._lock: if cls._instance is None: cls._instance = super().__new__(cls) - max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.4)) + max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.5)) cls._instance._initialize(max_workers) logger.debug("SharedProcessPool has been initialized with %s workers.", max_workers) @@ -49,42 +70,33 @@ def __new__(cls): return cls._instance - def __init__(self): - # Declare the attributes here to avoid W0201: attribute-defined-outside-init - self._total_max_workers = None - self._context = None - self._total_usage = 0.0 - self._stage_usage = {} - self._task_queues = {} - self._stage_semaphores = {} - self._processes = [] - def _initialize(self, total_max_workers): """ Initialize a concurrent.futures.ProcessPoolExecutor instance. """ - self._total_max_workers = total_max_workers self._context = mp.get_context("fork") + self._manager = self._context.Manager() self._total_usage = 0.0 # the percentage of processes that's currently in use self._stage_usage = {} # maintain the percentage of processes used by each stage - self._task_queues = {} # maintain a separate task queue for each stage - self._stage_semaphores = {} # maintain a semaphore for each stage + self._task_queues = self._manager.dict() # maintain a separate task queue for each stage + self._stage_semaphores = self._manager.dict() # maintain a semaphore for each stage self._processes = [] for i in range(total_max_workers): - process = self._context.Process(target=self._worker) + process = self._context.Process(target=self._worker, args=(self._task_queues, self._stage_semaphores)) process.start() self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) - def _worker(self): + @staticmethod + def _worker(task_queues, stage_semaphores): logger.debug("Worker process %s has been started.", os.getpid()) while True: # iterate over every semaphore - for stage_name, task_queue in self._task_queues.items(): - semaphore = self._stage_semaphores[stage_name] + for stage_name, task_queue in task_queues.items(): + semaphore = stage_semaphores[stage_name] if not semaphore.acquire(blocking=False): # Stage has reached the limitation of processes @@ -92,7 +104,7 @@ def _worker(self): try: task = task_queue.get_nowait() - except queue.Empty(): + except queue.Empty: semaphore.release() continue @@ -115,8 +127,8 @@ def submit_task(self, stage_name, process_fn, *args): """ Submit a task to the corresponding task queue of the stage. """ - future = Future() - task = (process_fn, args, {}, future) + future = SerializableFuture(self._context.Manager()) + task = (process_fn, args, future) self._task_queues[stage_name].put(task) return future @@ -137,10 +149,13 @@ def set_usage(self, stage_name, percentage): self._total_usage = new_total_usage allowed_processes_num = max(1, int(self._total_max_workers * percentage)) - self._stage_semaphores[stage_name] = Semaphore(allowed_processes_num) + self._stage_semaphores[stage_name] = self._manager.Semaphore(allowed_processes_num) if stage_name not in self._task_queues: - self._task_queues[stage_name] = self._context.Queue() + self._task_queues[stage_name] = self._manager.Queue() + + logger.debug("stage_usage: %s", self._stage_usage) + logger.debug("stage semaphores: %s", allowed_processes_num) def shutdown(self): @@ -153,7 +168,10 @@ def shutdown(self): p.join() logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + self._manager.shutdown() + self._shutdown = True logger.debug("Process pool has been terminated.") def __del__(self): - self.shutdown() + if not self._shutdown: + self.shutdown() diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py new file mode 100644 index 0000000000..3ebbc7f815 --- /dev/null +++ b/tests/utils/test_shared_process_pool.py @@ -0,0 +1,59 @@ +import logging +import atexit +import numpy as np +import multiprocessing as mp +import time +from morpheus.utils.shared_process_pool import SharedProcessPool + +logging.basicConfig(level=logging.INFO, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') +logger = logging.getLogger(__name__) + + +def _matrix_multiplication_task(size): + A = np.random.rand(size, size) + B = np.random.rand(size, size) + + mul = np.dot(A, B) + result = (mul, time.time()) + return result + + +def _test_worker(pool, stage_name, task_size, num_tasks): + future_list = [] + for i in range(num_tasks): + future_list.append(pool.submit_task(stage_name, _matrix_multiplication_task, task_size)) + logging.info("Task %s/%s has been submitted to stage %s.", i + 1, num_tasks, stage_name) + + for future in future_list: + future.result() + logging.info("task number %s has been completed in stage: %s", future_list.index(future), stage_name) + + logging.info("All tasks in stage %s have been completed in %.2f seconds.", + stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) + + +def test_shared_process_pool(): + pool = SharedProcessPool() + atexit.register(pool.shutdown) + + pool.set_usage("test_stage_1", 0.1) + pool.set_usage("test_stage_2", 0.3) + pool.set_usage("test_stage_3", 0.6) + + tasks = [("test_stage_1", 8000, 30), ("test_stage_2", 8000, 30), ("test_stage_3", 8000, 30)] + + processes = [] + for task in tasks: + stage_name, task_size, num_tasks = task + p = mp.Process(target=_test_worker, args=(pool, stage_name, task_size, num_tasks)) + processes.append(p) + + for p in processes: + p.start() + + for p in processes: + p.join() + + +if __name__ == "__main__": + test_shared_process_pool() From 86cbed8f1ce3a0ad0044e7d72eec688a4c700b30 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 22 Aug 2024 15:04:18 -0700 Subject: [PATCH 04/30] Add tests and fix shutdown of shared_process_pool --- .../morpheus/utils/shared_process_pool.py | 60 +++++++++++-------- tests/utils/test_shared_process_pool.py | 34 +++++++++-- 2 files changed, 64 insertions(+), 30 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 8bfc9e58e2..3b9350898d 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -21,7 +21,9 @@ import time from threading import Lock -# logging.basicConfig(level=logging.DEBUG, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') +# pylint: disable=W0201 + +logging.basicConfig(level=logging.DEBUG, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') logger = logging.getLogger(__name__) @@ -77,23 +79,32 @@ def _initialize(self, total_max_workers): self._total_max_workers = total_max_workers self._context = mp.get_context("fork") self._manager = self._context.Manager() - self._total_usage = 0.0 # the percentage of processes that's currently in use - self._stage_usage = {} # maintain the percentage of processes used by each stage - self._task_queues = self._manager.dict() # maintain a separate task queue for each stage - self._stage_semaphores = self._manager.dict() # maintain a semaphore for each stage + self._total_usage = 0.0 + self._stage_usage = {} + self._task_queues = self._manager.dict() + self._stage_semaphores = self._manager.dict() self._processes = [] + # TODO: Test the performance of reading the shared variable in each worker loop and try some alternatives + self._shutdown_in_progress = self._manager.Value("b", False) + for i in range(total_max_workers): - process = self._context.Process(target=self._worker, args=(self._task_queues, self._stage_semaphores)) + process = self._context.Process(target=self._worker, + args=(self._task_queues, self._stage_semaphores, + self._shutdown_in_progress)) process.start() self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) @staticmethod - def _worker(task_queues, stage_semaphores): + def _worker(task_queues, stage_semaphores, shutdown_in_progress): logger.debug("Worker process %s has been started.", os.getpid()) while True: + if shutdown_in_progress.value: + logger.debug("Worker process %s has been terminated.", os.getpid()) + return + # iterate over every semaphore for stage_name, task_queue in task_queues.items(): semaphore = stage_semaphores[stage_name] @@ -108,9 +119,9 @@ def _worker(task_queues, stage_semaphores): semaphore.release() continue - if task is None: # Stop signal - semaphore.release() - return + # if task is None: # Stop signal + # semaphore.release() + # return process_fn, args, future = task try: @@ -121,7 +132,7 @@ def _worker(task_queues, stage_semaphores): semaphore.release() - time.sleep(0.1) # avoid busy-waiting + time.sleep(0.1) # Avoid busy-waiting def submit_task(self, stage_name, process_fn, *args): """ @@ -158,19 +169,20 @@ def set_usage(self, stage_name, percentage): logger.debug("stage semaphores: %s", allowed_processes_num) def shutdown(self): - - for stage_name, task_queue in self._task_queues.items(): - for _ in range(self._total_max_workers): - task_queue.put(None) - logger.debug("Task queue for stage %s has been cleared.", stage_name) - - for i, p in enumerate(self._processes): - p.join() - logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) - - self._manager.shutdown() - self._shutdown = True - logger.debug("Process pool has been terminated.") + if not self._shutdown: + self._shutdown_in_progress.value = True + # for stage_name, task_queue in self._task_queues.items(): + # for _ in range(self._total_max_workers): + # task_queue.put(None) + # logger.debug("Task queue for stage %s has been cleared.", stage_name) + + for i, p in enumerate(self._processes): + p.join() + logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + + self._manager.shutdown() + self._shutdown = True + logger.debug("Process pool has been terminated.") def __del__(self): if not self._shutdown: diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 3ebbc7f815..5f553e7355 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -1,8 +1,24 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024, NVIDIA CORPORATION & AFFILIATES. All rights reserved. +# SPDX-License-Identifier: Apache-2.0 +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + import logging -import atexit -import numpy as np import multiprocessing as mp import time + +import numpy as np + from morpheus.utils.shared_process_pool import SharedProcessPool logging.basicConfig(level=logging.INFO, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') @@ -10,10 +26,10 @@ def _matrix_multiplication_task(size): - A = np.random.rand(size, size) - B = np.random.rand(size, size) + matrix_a = np.random.rand(size, size) + matrix_b = np.random.rand(size, size) - mul = np.dot(A, B) + mul = np.dot(matrix_a, matrix_b) result = (mul, time.time()) return result @@ -32,9 +48,15 @@ def _test_worker(pool, stage_name, task_size, num_tasks): stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) +def test_singleton(): + pool_1 = SharedProcessPool() + pool_2 = SharedProcessPool() + + assert pool_1 is pool_2 + + def test_shared_process_pool(): pool = SharedProcessPool() - atexit.register(pool.shutdown) pool.set_usage("test_stage_1", 0.1) pool.set_usage("test_stage_2", 0.3) From 21fd8e8aff0e2e638fb0c691c95c2773760a1cf1 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Fri, 23 Aug 2024 13:28:40 -0700 Subject: [PATCH 05/30] Initial impl of multi_processing_stage --- .../stages/general/multi_processing_stage.py | 108 ++++++++++++++++++ .../morpheus/utils/shared_process_pool.py | 4 + 2 files changed, 112 insertions(+) create mode 100644 python/morpheus/morpheus/stages/general/multi_processing_stage.py diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py new file mode 100644 index 0000000000..a464031411 --- /dev/null +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -0,0 +1,108 @@ +from abc import abstractmethod +import typing + +from morpheus.pipeline.stage_schema import StageSchema +from morpheus.utils.shared_process_pool import SharedProcessPool +from morpheus.config import Config +from morpheus.pipeline.single_port_stage import SinglePortStage +import mrc +import mrc.core.operators as ops + +InputT = typing.TypeVar('InputT') +OutputT = typing.TypeVar('OutputT') + + +class MultiProcessingBaseStage(SinglePortStage, typing.Generic[InputT, OutputT]): + + def __init__(self, *, c: Config, process_pool_usage: float): + super().__init__(c=c) + + self._process_pool_usage = process_pool_usage + + @property + def name(self) -> str: + return "multi-processing-base-stage" + + def accepted_types(self) -> typing.Tuple: + return (InputT, ) + + def compute_schema(self, schema: StageSchema): + return super().compute_schema(schema) + + @abstractmethod + def _on_data(self, data: InputT) -> OutputT: + pass + + +class MultiProcessingStage(MultiProcessingBaseStage[InputT, OutputT]): + + def __init__(self, + *, + c: Config, + process_pool_usage: float, + process_fn: typing.Callable[[InputT], OutputT], + max_in_flight_messages: int = None): + super().__init__(c=c, process_pool_usage=process_pool_usage) + + self._process_fn = process_fn + self._shared_process_pool = SharedProcessPool() + self._shared_process_pool.set_usage(self.name, self._process_pool_usage) + if max_in_flight_messages is None: + # set the multiplier to 1.5 to keep the workers busy + self._max_in_flight_messages = self._shared_process_pool.total_max_workers * 1.5 + else: + self._max_in_flight_messages = max_in_flight_messages + + @property + def name(self) -> str: + return "multi-processing-stage" + + def _on_data(self, data: InputT) -> OutputT: + + future = self._shared_process_pool.submit_task(self.name, self._process_fn, data) + result = future.result() + + return result + + @staticmethod + def create(*, c: Config, process_fn: typing.Callable[[InputT], OutputT], process_pool_usage: float): + + return MultiProcessingStage[InputT, OutputT](c=c, process_pool_usage=process_pool_usage, process_fn=process_fn) + + def _build_single(self, builder: mrc.Builder, input_node: mrc.SegmentObject) -> mrc.SegmentObject: + node = builder.make_node(self.name, ops.map(self._on_data)) + node.lanuch_options.pe_count = self._max_in_flight_messages + + builder.make_edge(input_node, node) + + return node + + +# pipe = LinearPipeline(config) + +# # ...add other stages... + +# # You can derive from the base class if you need to use self inside the process function +# class MyCustomMultiProcessStage(MultiProcessStage[ControlMessage, ControlMessage]): + +# def __init__(self, *, c: Config, process_pool_usage: float, add_column_name: str): +# super().__init__(self, c=c, process_pool_usage=process_pool_usage) + +# self._add_column_name = add_column_name + +# def _on_data(self, data: ControlMessage) -> ControlMessage: + +# with data.payload().mutable_dataframe() as df: +# df[self._add_column_name] = "hello" + +# return data + +# # Add an instance of the custom stage +# pipe.add_stage(MyCustomMultiProcessStage(c=config, process_pool_usage, add_column_name="NewCol") + +# # If you just want to supply a function pointer +# def print_process_id(message): +# print(os.pid()) +# return message + +# pipe.add_stage(MultiProcessingStage.create(c=config, process_fn=print_process_id)) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 3b9350898d..d43820a1bc 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -96,6 +96,10 @@ def _initialize(self, total_max_workers): self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) + @property + def total_max_workers(self): + return self._total_max_workers + @staticmethod def _worker(task_queues, stage_semaphores, shutdown_in_progress): logger.debug("Worker process %s has been started.", os.getpid()) From 46865ae6b6a13a016dec3f565af1dec4d26509a3 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Fri, 23 Aug 2024 17:13:57 -0700 Subject: [PATCH 06/30] initial test for test_multi_processing_stage --- .../stages/general/multi_processing_stage.py | 48 ++++++++------ tests/test_multi_processing_stage.py | 64 +++++++++++++++++++ 2 files changed, 92 insertions(+), 20 deletions(-) create mode 100644 tests/test_multi_processing_stage.py diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index a464031411..2001be4413 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -1,4 +1,4 @@ -from abc import abstractmethod +from abc import ABC, abstractmethod import typing from morpheus.pipeline.stage_schema import StageSchema @@ -12,12 +12,22 @@ OutputT = typing.TypeVar('OutputT') -class MultiProcessingBaseStage(SinglePortStage, typing.Generic[InputT, OutputT]): +class MultiProcessingBaseStage(SinglePortStage, ABC, typing.Generic[InputT, OutputT]): - def __init__(self, *, c: Config, process_pool_usage: float): + def __init__(self, *, c: Config, process_pool_usage: float, max_in_flight_messages: int = None): super().__init__(c=c) self._process_pool_usage = process_pool_usage + self._shared_process_pool = SharedProcessPool() + self._shared_process_pool.set_usage(self.name, self._process_pool_usage) + + if max_in_flight_messages is None: + # set the multiplier to 1.5 to keep the workers busy + self._max_in_flight_messages = int(self._shared_process_pool.total_max_workers * 1.5) + else: + self._max_in_flight_messages = max_in_flight_messages + + # self._max_in_flight_messages = 1 @property def name(self) -> str: @@ -27,12 +37,25 @@ def accepted_types(self) -> typing.Tuple: return (InputT, ) def compute_schema(self, schema: StageSchema): - return super().compute_schema(schema) + for (port_idx, port_schema) in enumerate(schema.input_schemas): + schema.output_schemas[port_idx].set_type(port_schema.get_type()) @abstractmethod def _on_data(self, data: InputT) -> OutputT: pass + def supports_cpp_node(self): + return False + + def _build_single(self, builder: mrc.Builder, input_node: mrc.SegmentObject) -> mrc.SegmentObject: + node = builder.make_node(self.name, ops.map(self._on_data)) + node.launch_options.pe_count = self._max_in_flight_messages + + builder.make_edge(input_node, node) + + return node + + class MultiProcessingStage(MultiProcessingBaseStage[InputT, OutputT]): @@ -42,16 +65,9 @@ def __init__(self, process_pool_usage: float, process_fn: typing.Callable[[InputT], OutputT], max_in_flight_messages: int = None): - super().__init__(c=c, process_pool_usage=process_pool_usage) + super().__init__(c=c, process_pool_usage=process_pool_usage, max_in_flight_messages=max_in_flight_messages) self._process_fn = process_fn - self._shared_process_pool = SharedProcessPool() - self._shared_process_pool.set_usage(self.name, self._process_pool_usage) - if max_in_flight_messages is None: - # set the multiplier to 1.5 to keep the workers busy - self._max_in_flight_messages = self._shared_process_pool.total_max_workers * 1.5 - else: - self._max_in_flight_messages = max_in_flight_messages @property def name(self) -> str: @@ -69,14 +85,6 @@ def create(*, c: Config, process_fn: typing.Callable[[InputT], OutputT], process return MultiProcessingStage[InputT, OutputT](c=c, process_pool_usage=process_pool_usage, process_fn=process_fn) - def _build_single(self, builder: mrc.Builder, input_node: mrc.SegmentObject) -> mrc.SegmentObject: - node = builder.make_node(self.name, ops.map(self._on_data)) - node.lanuch_options.pe_count = self._max_in_flight_messages - - builder.make_edge(input_node, node) - - return node - # pipe = LinearPipeline(config) diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py new file mode 100644 index 0000000000..14574a9d73 --- /dev/null +++ b/tests/test_multi_processing_stage.py @@ -0,0 +1,64 @@ +from typing import Tuple +import cudf +import pytest +import os + +from _utils.dataset_manager import DatasetManager +from morpheus.config import Config +from morpheus.messages import ControlMessage +from morpheus.pipeline import LinearPipeline +from morpheus.stages.general.multi_processing_stage import MultiProcessingBaseStage +from morpheus.stages.general.multi_processing_stage import MultiProcessingStage +from morpheus.stages.input.in_memory_source_stage import InMemorySourceStage +from morpheus.stages.preprocess.deserialize_stage import DeserializeStage + + +def test_constructor(config: Config): + stage = MultiProcessingStage.create(c=config, process_fn=lambda x: x, process_pool_usage=0.5) + assert stage.name == "multi-processing-stage" + + +class DerivedMultiProcessingStage(MultiProcessingBaseStage[ControlMessage, ControlMessage]): + + def __init__(self, + *, + c: Config, + process_pool_usage: float, + add_column_name: str, + max_in_flight_messages: int = None): + super().__init__(c=c, process_pool_usage=process_pool_usage, max_in_flight_messages=max_in_flight_messages) + + self._add_column_name = add_column_name + + @property + def name(self) -> str: + return "derived-multi-processing-stage" + + def accepted_types(self) -> Tuple: + return (ControlMessage, ) + + def _on_data(self, data: ControlMessage) -> ControlMessage: + with data.payload().mutable_dataframe() as df: + df[self._add_column_name] = "Hello" + + return data + +@pytest.mark.use_python +def test_stage_pipe(config: Config, dataset_pandas: DatasetManager): + + config.num_threads = os.cpu_count() + input_df = dataset_pandas["filter_probs.csv"] + + pipe = LinearPipeline(config) + pipe.set_source(InMemorySourceStage(config, [cudf.DataFrame(input_df)])) + pipe.add_stage(DeserializeStage(config, ensure_sliceable_index=True, message_type=ControlMessage)) + pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.5, add_column_name="new_column")) + + pipe.run() + + +# if __name__ == "__main__": +# config = Config() +# dataset_pandas = DatasetManager() +# # test_constructor(config) +# test_stage_pipe(config, dataset_pandas) From 406dbeb0bb234fb6bcaeb3a6b655e76eff05fe60 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Mon, 26 Aug 2024 17:06:33 -0700 Subject: [PATCH 07/30] fix unit test for multi_processing_stage --- .../stages/general/multi_processing_stage.py | 58 +++++++------------ .../morpheus/utils/shared_process_pool.py | 2 +- tests/test_multi_processing_stage.py | 39 ++++++++++--- 3 files changed, 53 insertions(+), 46 deletions(-) diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index 2001be4413..2456e811e6 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -1,13 +1,30 @@ -from abc import ABC, abstractmethod +# SPDX-FileCopyrightText: Copyright (c) 2024, NVIDIA CORPORATION & AFFILIATES. All rights reserved. +# SPDX-License-Identifier: Apache-2.0 +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + import typing +from abc import ABC +from abc import abstractmethod -from morpheus.pipeline.stage_schema import StageSchema -from morpheus.utils.shared_process_pool import SharedProcessPool -from morpheus.config import Config -from morpheus.pipeline.single_port_stage import SinglePortStage import mrc import mrc.core.operators as ops +from morpheus.config import Config +from morpheus.pipeline.single_port_stage import SinglePortStage +from morpheus.pipeline.stage_schema import StageSchema +from morpheus.utils.shared_process_pool import SharedProcessPool + InputT = typing.TypeVar('InputT') OutputT = typing.TypeVar('OutputT') @@ -56,7 +73,6 @@ def _build_single(self, builder: mrc.Builder, input_node: mrc.SegmentObject) -> return node - class MultiProcessingStage(MultiProcessingBaseStage[InputT, OutputT]): def __init__(self, @@ -84,33 +100,3 @@ def _on_data(self, data: InputT) -> OutputT: def create(*, c: Config, process_fn: typing.Callable[[InputT], OutputT], process_pool_usage: float): return MultiProcessingStage[InputT, OutputT](c=c, process_pool_usage=process_pool_usage, process_fn=process_fn) - - -# pipe = LinearPipeline(config) - -# # ...add other stages... - -# # You can derive from the base class if you need to use self inside the process function -# class MyCustomMultiProcessStage(MultiProcessStage[ControlMessage, ControlMessage]): - -# def __init__(self, *, c: Config, process_pool_usage: float, add_column_name: str): -# super().__init__(self, c=c, process_pool_usage=process_pool_usage) - -# self._add_column_name = add_column_name - -# def _on_data(self, data: ControlMessage) -> ControlMessage: - -# with data.payload().mutable_dataframe() as df: -# df[self._add_column_name] = "hello" - -# return data - -# # Add an instance of the custom stage -# pipe.add_stage(MyCustomMultiProcessStage(c=config, process_pool_usage, add_column_name="NewCol") - -# # If you just want to supply a function pointer -# def print_process_id(message): -# print(os.pid()) -# return message - -# pipe.add_stage(MultiProcessingStage.create(c=config, process_fn=print_process_id)) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index d43820a1bc..9345b868bb 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -99,7 +99,7 @@ def _initialize(self, total_max_workers): @property def total_max_workers(self): return self._total_max_workers - + @staticmethod def _worker(task_queues, stage_semaphores, shutdown_in_progress): logger.debug("Worker process %s has been started.", os.getpid()) diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index 14574a9d73..e728a34111 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -1,8 +1,26 @@ +# SPDX-FileCopyrightText: Copyright (c) 2024, NVIDIA CORPORATION & AFFILIATES. All rights reserved. +# SPDX-License-Identifier: Apache-2.0 +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import os from typing import Tuple -import cudf + import pytest -import os +import cudf + +from _utils import assert_results from _utils.dataset_manager import DatasetManager from morpheus.config import Config from morpheus.messages import ControlMessage @@ -10,6 +28,8 @@ from morpheus.stages.general.multi_processing_stage import MultiProcessingBaseStage from morpheus.stages.general.multi_processing_stage import MultiProcessingStage from morpheus.stages.input.in_memory_source_stage import InMemorySourceStage +from morpheus.stages.output.compare_dataframe_stage import CompareDataFrameStage +from morpheus.stages.postprocess.serialize_stage import SerializeStage from morpheus.stages.preprocess.deserialize_stage import DeserializeStage @@ -43,22 +63,23 @@ def _on_data(self, data: ControlMessage) -> ControlMessage: return data + @pytest.mark.use_python def test_stage_pipe(config: Config, dataset_pandas: DatasetManager): config.num_threads = os.cpu_count() input_df = dataset_pandas["filter_probs.csv"] + add_column_name = "new_column" + expected_df = input_df.copy() + expected_df[add_column_name] = "Hello" pipe = LinearPipeline(config) pipe.set_source(InMemorySourceStage(config, [cudf.DataFrame(input_df)])) pipe.add_stage(DeserializeStage(config, ensure_sliceable_index=True, message_type=ControlMessage)) - pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.5, add_column_name="new_column")) + pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.5, add_column_name=add_column_name)) + pipe.add_stage(SerializeStage(config)) + comp_stage = pipe.add_stage(CompareDataFrameStage(config, expected_df)) pipe.run() - -# if __name__ == "__main__": -# config = Config() -# dataset_pandas = DatasetManager() -# # test_constructor(config) -# test_stage_pipe(config, dataset_pandas) + assert_results(comp_stage.get_results()) From f7b7280cea4ab7ee033b37c0b48bc58b88e48da3 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 29 Aug 2024 16:33:38 -0700 Subject: [PATCH 08/30] add unit tests for shared_process_pool --- .../morpheus/utils/shared_process_pool.py | 25 ++- tests/utils/test_shared_process_pool.py | 160 +++++++++++++++++- 2 files changed, 166 insertions(+), 19 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 9345b868bb..093573defb 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -48,6 +48,9 @@ def result(self): raise self._exception.value return self._result.value + def done(self): + return self._done.is_set() + class SharedProcessPool: @@ -85,7 +88,6 @@ def _initialize(self, total_max_workers): self._stage_semaphores = self._manager.dict() self._processes = [] - # TODO: Test the performance of reading the shared variable in each worker loop and try some alternatives self._shutdown_in_progress = self._manager.Value("b", False) for i in range(total_max_workers): @@ -123,13 +125,14 @@ def _worker(task_queues, stage_semaphores, shutdown_in_progress): semaphore.release() continue - # if task is None: # Stop signal - # semaphore.release() - # return + if task is None: + logger.warning("Worker process %s received a None task.", os.getpid()) + semaphore.release() + continue - process_fn, args, future = task + process_fn, args, kwargs, future = task try: - result = process_fn(*args) + result = process_fn(*args, **kwargs) future.set_result(result) except Exception as e: future.set_exception(e) @@ -138,12 +141,12 @@ def _worker(task_queues, stage_semaphores, shutdown_in_progress): time.sleep(0.1) # Avoid busy-waiting - def submit_task(self, stage_name, process_fn, *args): + def submit_task(self, stage_name, process_fn, *args, **kwargs): """ Submit a task to the corresponding task queue of the stage. """ future = SerializableFuture(self._context.Manager()) - task = (process_fn, args, future) + task = (process_fn, args, kwargs, future) self._task_queues[stage_name].put(task) return future @@ -152,7 +155,7 @@ def set_usage(self, stage_name, percentage): """ Set the maximum percentage of processes that can be used by each stage. """ - if not 0 < percentage <= 1: + if not 0 <= percentage <= 1: raise ValueError("Percentage must be between 0 and 1.") new_total_usage = self._total_usage - self._stage_usage.get(stage_name, 0.0) + percentage @@ -175,10 +178,6 @@ def set_usage(self, stage_name, percentage): def shutdown(self): if not self._shutdown: self._shutdown_in_progress.value = True - # for stage_name, task_queue in self._task_queues.items(): - # for _ in range(self._total_max_workers): - # task_queue.put(None) - # logger.debug("Task queue for stage %s has been cleared.", stage_name) for i, p in enumerate(self._processes): p.join() diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 5f553e7355..222f271444 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -13,18 +13,29 @@ # See the License for the specific language governing permissions and # limitations under the License. +import atexit import logging import multiprocessing as mp +import threading import time import numpy as np +import pytest from morpheus.utils.shared_process_pool import SharedProcessPool -logging.basicConfig(level=logging.INFO, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') +# logging.basicConfig(level=logging.INFO, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') logger = logging.getLogger(__name__) +@pytest.fixture(name="shared_process_pool") +def shared_process_pool_fixture(): + pool = SharedProcessPool() + atexit.register(pool.shutdown) # make sure to shutdown the pool before the test exits + + return pool + + def _matrix_multiplication_task(size): matrix_a = np.random.rand(size, size) matrix_b = np.random.rand(size, size) @@ -34,6 +45,22 @@ def _matrix_multiplication_task(size): return result +def _simple_add_task(a, b): + return a + b + + +def _process_func_with_exception(): + raise ValueError("Exception is raised in the process.") + + +def _unserializable_function(): + return threading.Lock() + + +def _arbitrary_function(*args, **kwargs): + return args, kwargs + + def _test_worker(pool, stage_name, task_size, num_tasks): future_list = [] for i in range(num_tasks): @@ -47,6 +74,8 @@ def _test_worker(pool, stage_name, task_size, num_tasks): logging.info("All tasks in stage %s have been completed in %.2f seconds.", stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) + assert len(future_list) == num_tasks + def test_singleton(): pool_1 = SharedProcessPool() @@ -55,14 +84,81 @@ def test_singleton(): assert pool_1 is pool_2 -def test_shared_process_pool(): - pool = SharedProcessPool() +def test_single_task(shared_process_pool): + pool = shared_process_pool + + pool.set_usage("test_stage", 0.5) + + a = 10 + b = 20 + + future = pool.submit_task("test_stage", _simple_add_task, a, b) + assert future.result() == a + b + + future = pool.submit_task("test_stage", _simple_add_task, a=a, b=b) + assert future.result() == a + b + + future = pool.submit_task("test_stage", _simple_add_task, a, b=b) + assert future.result() == a + b + + +def test_multiple_tasks(shared_process_pool): + pool = shared_process_pool + + pool.set_usage("test_stage", 0.5) + + num_tasks = 100 + futures = [] + for _ in range(num_tasks): + futures.append(pool.submit_task("test_stage", _simple_add_task, 10, 20)) + + for future in futures: + assert future.result() == 30 + + +def test_error_process_function(shared_process_pool): + pool = shared_process_pool + + pool.set_usage("test_stage", 0.5) + + with pytest.raises(ValueError): + future = pool.submit_task("test_stage", _process_func_with_exception) + future.result() + + +def test_unserializable_function(shared_process_pool): + pool = shared_process_pool + + pool.set_usage("test_stage", 0.5) + + with pytest.raises(TypeError): + future = pool.submit_task("test_stage", _unserializable_function) + future.result() + + +def test_unserializable_arg(shared_process_pool): + pool = shared_process_pool + + pool.set_usage("test_stage", 0.5) + + with pytest.raises(TypeError): + future = pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) + future.result() + + +def test_multiple_stages(shared_process_pool): + pool = shared_process_pool + + pool.set_usage("test_stage", 0.0) # Remove usage of test_stage in previous tests pool.set_usage("test_stage_1", 0.1) pool.set_usage("test_stage_2", 0.3) pool.set_usage("test_stage_3", 0.6) - tasks = [("test_stage_1", 8000, 30), ("test_stage_2", 8000, 30), ("test_stage_3", 8000, 30)] + task_size = 3000 + task_num = 30 + tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), + ("test_stage_3", task_size, task_num)] processes = [] for task in tasks: @@ -77,5 +173,57 @@ def test_shared_process_pool(): p.join() -if __name__ == "__main__": - test_shared_process_pool() +def test_invalid_stage_usage(shared_process_pool): + pool = shared_process_pool + + # Remove usage of test_stage in previous tests + pool.set_usage("test_stage", 0.0) + pool.set_usage("test_stage_1", 0.0) + pool.set_usage("test_stage_2", 0.0) + pool.set_usage("test_stage_3", 0.0) + + with pytest.raises(ValueError): + pool.set_usage("test_stage", 1.1) + + with pytest.raises(ValueError): + pool.set_usage("test_stage", -0.1) + + pool.set_usage("test_stage_1", 0.5) + pool.set_usage("test_stage_2", 0.4) + + pool.set_usage("test_stage_1", 0.6) # ok to update the usage of an existing stage + + with pytest.raises(ValueError): + pool.set_usage("test_stage_1", 0.7) # not ok to exceed the total usage limit after updating + + with pytest.raises(ValueError): + pool.set_usage("test_stage_3", 0.1) + + +def test_task_completion_before_shutdown(shared_process_pool): + pool = shared_process_pool + + # Remove usage of test_stage in previous tests + pool.set_usage("test_stage", 0.0) + pool.set_usage("test_stage_1", 0.0) + pool.set_usage("test_stage_2", 0.0) + pool.set_usage("test_stage_3", 0.0) + + pool.set_usage("test_stage_1", 0.1) + pool.set_usage("test_stage_2", 0.3) + pool.set_usage("test_stage_3", 0.6) + + task_size = 3000 + task_num = 30 + futures = [] + for _ in range(task_num): + futures.append(pool.submit_task("test_stage_1", _matrix_multiplication_task, task_size)) + futures.append(pool.submit_task("test_stage_2", _matrix_multiplication_task, task_size)) + futures.append(pool.submit_task("test_stage_3", _matrix_multiplication_task, task_size)) + + pool.shutdown() + + # all tasks should be completed before shutdown + assert len(futures) == 3 * task_num + for future in futures: + assert future.done() From 40234e2aa716ab7e1a220cbe76f893b221a69243 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Fri, 30 Aug 2024 01:31:51 -0700 Subject: [PATCH 09/30] Fix SharedProcessPool unit test --- .../stages/general/multi_processing_stage.py | 6 ++-- .../morpheus/utils/shared_process_pool.py | 4 +-- tests/test_multi_processing_stage.py | 1 + tests/utils/test_shared_process_pool.py | 33 ++++++++----------- 4 files changed, 18 insertions(+), 26 deletions(-) diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index 2456e811e6..adc8669d87 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -57,13 +57,13 @@ def compute_schema(self, schema: StageSchema): for (port_idx, port_schema) in enumerate(schema.input_schemas): schema.output_schemas[port_idx].set_type(port_schema.get_type()) + def supports_cpp_node(self): + return False + @abstractmethod def _on_data(self, data: InputT) -> OutputT: pass - def supports_cpp_node(self): - return False - def _build_single(self, builder: mrc.Builder, input_node: mrc.SegmentObject) -> mrc.SegmentObject: node = builder.make_node(self.name, ops.map(self._on_data)) node.launch_options.pe_count = self._max_in_flight_messages diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 093573defb..a389f94e38 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -21,9 +21,6 @@ import time from threading import Lock -# pylint: disable=W0201 - -logging.basicConfig(level=logging.DEBUG, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') logger = logging.getLogger(__name__) @@ -52,6 +49,7 @@ def done(self): return self._done.is_set() +# pylint: disable=W0201 class SharedProcessPool: _instance = None diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index e728a34111..a8dd3835f4 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -68,6 +68,7 @@ def _on_data(self, data: ControlMessage) -> ControlMessage: def test_stage_pipe(config: Config, dataset_pandas: DatasetManager): config.num_threads = os.cpu_count() + input_df = dataset_pandas["filter_probs.csv"] add_column_name = "new_column" expected_df = input_df.copy() diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 222f271444..dfbfcb01ac 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -24,14 +24,15 @@ from morpheus.utils.shared_process_pool import SharedProcessPool -# logging.basicConfig(level=logging.INFO, format='%(asctime)s - %(name)s - %(levelname)s - %(message)s') logger = logging.getLogger(__name__) @pytest.fixture(name="shared_process_pool") def shared_process_pool_fixture(): pool = SharedProcessPool() - atexit.register(pool.shutdown) # make sure to shutdown the pool before the test exits + _reset_shared_process_pool(pool) + + atexit.register(pool.shutdown) # make sure to shutdown the pool before the test exits return pool @@ -61,6 +62,16 @@ def _arbitrary_function(*args, **kwargs): return args, kwargs +# SharedProcessPool is a singleton, need to reset before each test +def _reset_shared_process_pool(pool: SharedProcessPool): + for stage in pool._stage_usage: + pool._stage_usage[stage] = 0.0 + pool._stage_semaphores[stage] = pool._manager.Semaphore(1) + pool._task_queues[stage] = pool._manager.Queue() + + pool._total_usage = 0.0 + + def _test_worker(pool, stage_name, task_size, num_tasks): future_list = [] for i in range(num_tasks): @@ -149,12 +160,6 @@ def test_unserializable_arg(shared_process_pool): def test_multiple_stages(shared_process_pool): pool = shared_process_pool - pool.set_usage("test_stage", 0.0) # Remove usage of test_stage in previous tests - - pool.set_usage("test_stage_1", 0.1) - pool.set_usage("test_stage_2", 0.3) - pool.set_usage("test_stage_3", 0.6) - task_size = 3000 task_num = 30 tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), @@ -176,12 +181,6 @@ def test_multiple_stages(shared_process_pool): def test_invalid_stage_usage(shared_process_pool): pool = shared_process_pool - # Remove usage of test_stage in previous tests - pool.set_usage("test_stage", 0.0) - pool.set_usage("test_stage_1", 0.0) - pool.set_usage("test_stage_2", 0.0) - pool.set_usage("test_stage_3", 0.0) - with pytest.raises(ValueError): pool.set_usage("test_stage", 1.1) @@ -203,12 +202,6 @@ def test_invalid_stage_usage(shared_process_pool): def test_task_completion_before_shutdown(shared_process_pool): pool = shared_process_pool - # Remove usage of test_stage in previous tests - pool.set_usage("test_stage", 0.0) - pool.set_usage("test_stage_1", 0.0) - pool.set_usage("test_stage_2", 0.0) - pool.set_usage("test_stage_3", 0.0) - pool.set_usage("test_stage_1", 0.1) pool.set_usage("test_stage_2", 0.3) pool.set_usage("test_stage_3", 0.6) From d92ef9cb1c3072f1ab86857ff93ec5fbe73aaa6c Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Wed, 4 Sep 2024 19:25:53 -0700 Subject: [PATCH 10/30] fix multi_processing_stage unit test --- .../stages/general/multi_processing_stage.py | 16 ++++- tests/test_multi_processing_stage.py | 60 +++++++++++++++++-- 2 files changed, 68 insertions(+), 8 deletions(-) diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index adc8669d87..4fbff7c4c7 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -44,8 +44,6 @@ def __init__(self, *, c: Config, process_pool_usage: float, max_in_flight_messag else: self._max_in_flight_messages = max_in_flight_messages - # self._max_in_flight_messages = 1 - @property def name(self) -> str: return "multi-processing-base-stage" @@ -89,6 +87,11 @@ def __init__(self, def name(self) -> str: return "multi-processing-stage" + def accepted_types(self) -> typing.Tuple: + input_type = typing.get_args(self.__orig_class__)[0] # pylint: disable=no-member + + return (input_type, ) + def _on_data(self, data: InputT) -> OutputT: future = self._shared_process_pool.submit_task(self.name, self._process_fn, data) @@ -99,4 +102,11 @@ def _on_data(self, data: InputT) -> OutputT: @staticmethod def create(*, c: Config, process_fn: typing.Callable[[InputT], OutputT], process_pool_usage: float): - return MultiProcessingStage[InputT, OutputT](c=c, process_pool_usage=process_pool_usage, process_fn=process_fn) + type_hints = typing.get_type_hints(process_fn) + + input_type = next(iter(type_hints.values())) + output_type = type_hints["return"] + + return MultiProcessingStage[input_type, output_type](c=c, + process_pool_usage=process_pool_usage, + process_fn=process_fn) diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index a8dd3835f4..442c172abb 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -14,8 +14,11 @@ # limitations under the License. import os +from functools import partial +from typing import Generator from typing import Tuple +import pandas as pd import pytest import cudf @@ -24,17 +27,27 @@ from _utils.dataset_manager import DatasetManager from morpheus.config import Config from morpheus.messages import ControlMessage +from morpheus.messages import MessageMeta from morpheus.pipeline import LinearPipeline from morpheus.stages.general.multi_processing_stage import MultiProcessingBaseStage from morpheus.stages.general.multi_processing_stage import MultiProcessingStage +from morpheus.stages.input.in_memory_data_generation_stage import InMemoryDataGenStage from morpheus.stages.input.in_memory_source_stage import InMemorySourceStage +from morpheus.stages.output.in_memory_sink_stage import InMemorySinkStage from morpheus.stages.output.compare_dataframe_stage import CompareDataFrameStage from morpheus.stages.postprocess.serialize_stage import SerializeStage from morpheus.stages.preprocess.deserialize_stage import DeserializeStage +def process_fn(pdf: pd.DataFrame) -> pd.DataFrame: + + pdf["new_column"] = "Hello" + return pdf + + def test_constructor(config: Config): - stage = MultiProcessingStage.create(c=config, process_fn=lambda x: x, process_pool_usage=0.5) + + stage = MultiProcessingStage.create(c=config, process_fn=process_fn, process_pool_usage=0.5) assert stage.name == "multi-processing-stage" @@ -58,17 +71,20 @@ def accepted_types(self) -> Tuple: return (ControlMessage, ) def _on_data(self, data: ControlMessage) -> ControlMessage: - with data.payload().mutable_dataframe() as df: - df[self._add_column_name] = "Hello" + + pdf = data.payload().copy_dataframe().to_pandas() + + future = self._shared_process_pool.submit_task(self.name, process_fn, pdf) + data.payload(MessageMeta(cudf.DataFrame.from_pandas(future.result()))) return data @pytest.mark.use_python -def test_stage_pipe(config: Config, dataset_pandas: DatasetManager): +def test_derived_stage_pipe(config: Config, dataset_pandas: DatasetManager): config.num_threads = os.cpu_count() - + input_df = dataset_pandas["filter_probs.csv"] add_column_name = "new_column" expected_df = input_df.copy() @@ -84,3 +100,37 @@ def test_stage_pipe(config: Config, dataset_pandas: DatasetManager): pipe.run() assert_results(comp_stage.get_results()) + + +def pandas_dataframe_generator(dataset_pandas: DatasetManager, count: int) -> Generator[pd.DataFrame, None, None]: + + df = dataset_pandas["filter_probs.csv"] + + for _ in range(count): + yield df + + +@pytest.mark.use_python +def test_base_stage_with_function_pipe(config: Config, dataset_pandas: DatasetManager): + + config.num_threads = os.cpu_count() + + input_df = dataset_pandas["filter_probs.csv"] + + expected_df = input_df.copy() + expected_df["new_column"] = "Hello" + + df_count = 10 + df_generator = partial(pandas_dataframe_generator, dataset_pandas, df_count) + + pipe = LinearPipeline(config) + pipe.set_source(InMemoryDataGenStage(config, df_generator, output_data_type=pd.DataFrame)) + pipe.add_stage(MultiProcessingStage[pd.DataFrame, pd.DataFrame].create(c=config, + process_fn=process_fn, + process_pool_usage=0.5)) + sink_stage = pipe.add_stage(InMemorySinkStage(config)) + + pipe.run() + + for df in sink_stage.get_messages(): + assert df.equals(expected_df) From 26f8b0f00f1f2b61aa7fa559f944405aae22df1a Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Mon, 9 Sep 2024 15:46:22 -0700 Subject: [PATCH 11/30] add dataclass --- .../morpheus/morpheus/utils/shared_process_pool.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index a389f94e38..32b7d30cec 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -13,6 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. +from dataclasses import dataclass import logging import math import multiprocessing as mp @@ -20,6 +21,7 @@ import queue import time from threading import Lock +import uuid logger = logging.getLogger(__name__) @@ -48,6 +50,17 @@ def result(self): def done(self): return self._done.is_set() +@dataclass +class Task: + id: uuid + process_fn: callable + args: tuple + kwargs: dict + +@dataclass +class TaskResult: + result: any + exception: Exception # pylint: disable=W0201 class SharedProcessPool: From 1c3666c13f693ddcb38a0cd34b5402be21790a96 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Mon, 9 Sep 2024 17:13:45 -0700 Subject: [PATCH 12/30] add result_collection_loop --- .../morpheus/utils/shared_process_pool.py | 61 +++++++++++++++---- 1 file changed, 50 insertions(+), 11 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 32b7d30cec..86077ff7af 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -13,15 +13,18 @@ # See the License for the specific language governing permissions and # limitations under the License. -from dataclasses import dataclass +from concurrent.futures import Future import logging import math import multiprocessing as mp import os import queue import time -from threading import Lock +import threading import uuid +import typing +from dataclasses import dataclass +from threading import Lock logger = logging.getLogger(__name__) @@ -50,18 +53,22 @@ def result(self): def done(self): return self._done.is_set() + @dataclass class Task: - id: uuid + id: uuid.UUID process_fn: callable args: tuple kwargs: dict + @dataclass class TaskResult: + id: uuid.UUID result: any exception: Exception + # pylint: disable=W0201 class SharedProcessPool: @@ -98,23 +105,30 @@ def _initialize(self, total_max_workers): self._task_queues = self._manager.dict() self._stage_semaphores = self._manager.dict() self._processes = [] + self._tasks: typing.Dict[uuid.UUID, Future] = {} + self._completion_queue = self._manager.Queue() self._shutdown_in_progress = self._manager.Value("b", False) for i in range(total_max_workers): process = self._context.Process(target=self._worker, - args=(self._task_queues, self._stage_semaphores, + args=(self._task_queues, + self._stage_semaphores, + self._completion_queue, self._shutdown_in_progress)) process.start() self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) + self._task_result_collection_thread = threading.Thread(target=self._task_result_collection_loop, + args=(self._completion_queue, self._tasks)) + @property def total_max_workers(self): return self._total_max_workers @staticmethod - def _worker(task_queues, stage_semaphores, shutdown_in_progress): + def _worker(task_queues, stage_semaphores, completion_queue, shutdown_in_progress): logger.debug("Worker process %s has been started.", os.getpid()) while True: @@ -141,23 +155,48 @@ def _worker(task_queues, stage_semaphores, shutdown_in_progress): semaphore.release() continue - process_fn, args, kwargs, future = task + process_fn = task.process_fn + args = task.args + kwargs = task.kwargs + + task_result = TaskResult(task.id, None, None) try: result = process_fn(*args, **kwargs) - future.set_result(result) + task_result.result = result except Exception as e: - future.set_exception(e) + task_result.exception = e + + completion_queue.put(task_result) semaphore.release() time.sleep(0.1) # Avoid busy-waiting - def submit_task(self, stage_name, process_fn, *args, **kwargs): + @staticmethod + def _task_result_collection_loop(completion_queue, tasks): + while True: + try: + task_result = completion_queue.get_nowait() + + task_id = task_result.id + future = tasks.pop(task_id) + + if task_result.exception is not None: + future.set_exception(task_result.exception) + else: + future.set_result(task_result.result) + + except queue.Empty: + time.sleep(0.1) + + def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Future: """ Submit a task to the corresponding task queue of the stage. """ - future = SerializableFuture(self._context.Manager()) - task = (process_fn, args, kwargs, future) + # future = SerializableFuture(self._context.Manager()) + task = Task(uuid.uuid4(), process_fn, args, kwargs) + future = Future() + self._tasks[task.id] = future self._task_queues[stage_name].put(task) return future From 5fca5733e025c95d6830755669aaba731a9a5871 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 10 Sep 2024 09:43:43 -0700 Subject: [PATCH 13/30] add threading, need debug --- .../morpheus/utils/shared_process_pool.py | 127 +++++++++++------- tests/utils/test_shared_process_pool.py | 55 ++++---- 2 files changed, 101 insertions(+), 81 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 86077ff7af..66d3b8bc58 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -13,51 +13,37 @@ # See the License for the specific language governing permissions and # limitations under the License. -from concurrent.futures import Future import logging import math import multiprocessing as mp import os +import pickle import queue -import time import threading -import uuid +import time import typing +import uuid +from concurrent.futures import Future from dataclasses import dataclass +from enum import Enum from threading import Lock logger = logging.getLogger(__name__) +logger.setLevel(logging.DEBUG) -class SerializableFuture: - - def __init__(self, manager): - self._result = manager.Value("i", None) - self._exception = manager.Value("i", None) - self._done = manager.Event() - - def set_result(self, result): - self._result.value = result - self._done.set() - - def set_exception(self, exception): - self._exception.value = exception - self._done.set() - def result(self): - self._done.wait() - if self._exception.value is not None: - raise self._exception.value - return self._result.value - - def done(self): - return self._done.is_set() +class PoolStatus(Enum): + INITIALIZING = 0 + RUNNING = 1 + STOPPING = 2 + SHUTDOWN = 3 @dataclass class Task: id: uuid.UUID - process_fn: callable + process_fn: typing.Callable args: tuple kwargs: dict @@ -65,8 +51,8 @@ class Task: @dataclass class TaskResult: id: uuid.UUID - result: any - exception: Exception + result: typing.Any + exception: typing.Optional[Exception] = None # pylint: disable=W0201 @@ -82,14 +68,20 @@ def __new__(cls): """ logger.debug("Creating a new instance of SharedProcessPool...") with cls._lock: + + max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.5)) + if cls._instance is None: cls._instance = super().__new__(cls) - max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.5)) cls._instance._initialize(max_workers) logger.debug("SharedProcessPool has been initialized with %s workers.", max_workers) else: - logger.debug("SharedProcessPool instance already exists.") + if cls._instance.status is not PoolStatus.RUNNING: + cls._instance._initialize(max_workers) + logger.debug("SharedProcessPool has been initialized with %s workers.", max_workers) + else: + logger.debug("SharedProcessPool instance already exists and currently running.") return cls._instance @@ -97,42 +89,56 @@ def _initialize(self, total_max_workers): """ Initialize a concurrent.futures.ProcessPoolExecutor instance. """ + self._status = PoolStatus.INITIALIZING + self._total_max_workers = total_max_workers - self._context = mp.get_context("fork") - self._manager = self._context.Manager() + self._processes = [] self._total_usage = 0.0 self._stage_usage = {} + + self._context = mp.get_context("fork") + self._manager = self._context.Manager() + self._task_queues = self._manager.dict() + self._task_futures: typing.Dict[uuid.UUID, Future] = {} self._stage_semaphores = self._manager.dict() - self._processes = [] - self._tasks: typing.Dict[uuid.UUID, Future] = {} + self._completion_queue = self._manager.Queue() - self._shutdown_in_progress = self._manager.Value("b", False) + self._shutdown_flag = self._manager.Value("b", False) + self._shutdown_event = threading.Event() for i in range(total_max_workers): process = self._context.Process(target=self._worker, args=(self._task_queues, self._stage_semaphores, self._completion_queue, - self._shutdown_in_progress)) + self._shutdown_flag)) process.start() self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) self._task_result_collection_thread = threading.Thread(target=self._task_result_collection_loop, - args=(self._completion_queue, self._tasks)) + args=(self._completion_queue, + self._task_futures, + self._shutdown_event)) + self._task_result_collection_thread.start() + self._status = PoolStatus.RUNNING @property def total_max_workers(self): return self._total_max_workers + @property + def status(self) -> PoolStatus: + return self._status + @staticmethod - def _worker(task_queues, stage_semaphores, completion_queue, shutdown_in_progress): + def _worker(task_queues, stage_semaphores, completion_queue, shutdown_flag): logger.debug("Worker process %s has been started.", os.getpid()) while True: - if shutdown_in_progress.value: + if shutdown_flag.value: logger.debug("Worker process %s has been terminated.", os.getpid()) return @@ -166,20 +172,29 @@ def _worker(task_queues, stage_semaphores, completion_queue, shutdown_in_progres except Exception as e: task_result.exception = e - completion_queue.put(task_result) + try: + completion_queue.put(task_result) + # the result must be serializable + except (pickle.PicklingError, TypeError) as e: + task_result.exception = e + task_result.result = None + completion_queue.put(task_result) semaphore.release() time.sleep(0.1) # Avoid busy-waiting @staticmethod - def _task_result_collection_loop(completion_queue, tasks): + def _task_result_collection_loop(completion_queue, task_futures, shutdown_event): while True: + if shutdown_event.is_set() and completion_queue.empty(): + logger.debug("Task result collection process has been terminated.") + return try: task_result = completion_queue.get_nowait() task_id = task_result.id - future = tasks.pop(task_id) + future = task_futures.pop(task_id) if task_result.exception is not None: future.set_exception(task_result.exception) @@ -193,13 +208,13 @@ def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Future: """ Submit a task to the corresponding task queue of the stage. """ - # future = SerializableFuture(self._context.Manager()) - task = Task(uuid.uuid4(), process_fn, args, kwargs) - future = Future() - self._tasks[task.id] = future - self._task_queues[stage_name].put(task) + with self._lock: + task = Task(uuid.uuid4(), process_fn, args, kwargs) + future = Future() + self._task_futures[task.id] = future + self._task_queues[stage_name].put(task) - return future + return future def set_usage(self, stage_name, percentage): """ @@ -226,17 +241,27 @@ def set_usage(self, stage_name, percentage): logger.debug("stage semaphores: %s", allowed_processes_num) def shutdown(self): - if not self._shutdown: - self._shutdown_in_progress.value = True + if self._status != PoolStatus.SHUTDOWN: + + self._status = PoolStatus.STOPPING + + while self._task_futures: + time.sleep(0.1) + + self._shutdown_flag.value = True for i, p in enumerate(self._processes): p.join() logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + self._shutdown_event.set() + self._task_result_collection_thread.join() + self._manager.shutdown() self._shutdown = True + self._status = PoolStatus.SHUTDOWN logger.debug("Process pool has been terminated.") def __del__(self): - if not self._shutdown: + if self._status != PoolStatus.SHUTDOWN: self.shutdown() diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index dfbfcb01ac..9db1b453cd 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -import atexit import logging import multiprocessing as mp import threading @@ -26,15 +25,17 @@ logger = logging.getLogger(__name__) +# set logger level to debug +logger.setLevel(logging.DEBUG) + @pytest.fixture(name="shared_process_pool") def shared_process_pool_fixture(): pool = SharedProcessPool() - _reset_shared_process_pool(pool) - atexit.register(pool.shutdown) # make sure to shutdown the pool before the test exits + yield pool - return pool + pool.shutdown() def _matrix_multiplication_task(size): @@ -62,28 +63,18 @@ def _arbitrary_function(*args, **kwargs): return args, kwargs -# SharedProcessPool is a singleton, need to reset before each test -def _reset_shared_process_pool(pool: SharedProcessPool): - for stage in pool._stage_usage: - pool._stage_usage[stage] = 0.0 - pool._stage_semaphores[stage] = pool._manager.Semaphore(1) - pool._task_queues[stage] = pool._manager.Queue() - - pool._total_usage = 0.0 - - -def _test_worker(pool, stage_name, task_size, num_tasks): +def _task_submit_worker(pool, stage_name, task_size, num_tasks): future_list = [] for i in range(num_tasks): future_list.append(pool.submit_task(stage_name, _matrix_multiplication_task, task_size)) - logging.info("Task %s/%s has been submitted to stage %s.", i + 1, num_tasks, stage_name) + logging.debug("Task %s/%s has been submitted to stage %s.", i + 1, num_tasks, stage_name) for future in future_list: future.result() - logging.info("task number %s has been completed in stage: %s", future_list.index(future), stage_name) + logging.debug("task number %s has been completed in stage: %s", future_list.index(future), stage_name) - logging.info("All tasks in stage %s have been completed in %.2f seconds.", - stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) + logging.debug("All tasks in stage %s have been completed in %.2f seconds.", + stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) assert len(future_list) == num_tasks @@ -142,8 +133,8 @@ def test_unserializable_function(shared_process_pool): pool.set_usage("test_stage", 0.5) + future = pool.submit_task("test_stage", _unserializable_function) with pytest.raises(TypeError): - future = pool.submit_task("test_stage", _unserializable_function) future.result() @@ -153,22 +144,26 @@ def test_unserializable_arg(shared_process_pool): pool.set_usage("test_stage", 0.5) with pytest.raises(TypeError): - future = pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) - future.result() + pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) def test_multiple_stages(shared_process_pool): pool = shared_process_pool - task_size = 3000 - task_num = 30 - tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), - ("test_stage_3", task_size, task_num)] + pool.set_usage("test_stage_1", 0.1) + pool.set_usage("test_stage_2", 0.3) + pool.set_usage("test_stage_3", 0.6) + + task_size = 3 + task_num = 3 + # tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), + # ("test_stage_3", task_size, task_num)] + tasks = [("test_stage_1", task_size, task_num)] processes = [] for task in tasks: stage_name, task_size, num_tasks = task - p = mp.Process(target=_test_worker, args=(pool, stage_name, task_size, num_tasks)) + p = mp.Process(target=_task_submit_worker, args=(pool, stage_name, task_size, num_tasks)) processes.append(p) for p in processes: @@ -204,10 +199,10 @@ def test_task_completion_before_shutdown(shared_process_pool): pool.set_usage("test_stage_1", 0.1) pool.set_usage("test_stage_2", 0.3) - pool.set_usage("test_stage_3", 0.6) + pool.set_usage("test_stage_3", 0.5) - task_size = 3000 - task_num = 30 + task_size = 3 + task_num = 3 futures = [] for _ in range(task_num): futures.append(pool.submit_task("test_stage_1", _matrix_multiplication_task, task_size)) From 8108fd4549ae9cacd6a89d1515fb8a7ce191881a Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Mon, 16 Sep 2024 15:57:16 -0700 Subject: [PATCH 14/30] Revise shared_process_pool design --- .../morpheus/utils/shared_process_pool.py | 112 +++++++----------- tests/utils/test_shared_process_pool.py | 88 +++++++------- 2 files changed, 87 insertions(+), 113 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 66d3b8bc58..16e5dde1f1 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -17,14 +17,8 @@ import math import multiprocessing as mp import os -import pickle import queue -import threading import time -import typing -import uuid -from concurrent.futures import Future -from dataclasses import dataclass from enum import Enum from threading import Lock @@ -40,19 +34,45 @@ class PoolStatus(Enum): SHUTDOWN = 3 -@dataclass -class Task: - id: uuid.UUID - process_fn: typing.Callable - args: tuple - kwargs: dict +class SimpleFuture: + def __init__(self, manager): + self._result = manager.Value("i", None) + self._exception = manager.Value("i", None) + self._done = manager.Event() + def set_result(self, result): + self._result.value = result + self._done.set() -@dataclass -class TaskResult: - id: uuid.UUID - result: typing.Any - exception: typing.Optional[Exception] = None + def set_exception(self, exception): + self._exception.value = exception + self._done.set() + + def result(self): + self._done.wait() + if self._exception.value is not None: + raise self._exception.value + return self._result.value + + +class Task(SimpleFuture): + def __init__(self, manager, process_fn, args, kwargs): + super().__init__(manager) + self._process_fn = process_fn + self._args = args + self._kwargs = kwargs + + @property + def process_fn(self): + return self._process_fn + + @property + def args(self): + return self._args + + @property + def kwargs(self): + return self._kwargs # pylint: disable=W0201 @@ -100,29 +120,19 @@ def _initialize(self, total_max_workers): self._manager = self._context.Manager() self._task_queues = self._manager.dict() - self._task_futures: typing.Dict[uuid.UUID, Future] = {} self._stage_semaphores = self._manager.dict() - self._completion_queue = self._manager.Queue() - self._shutdown_flag = self._manager.Value("b", False) - self._shutdown_event = threading.Event() for i in range(total_max_workers): process = self._context.Process(target=self._worker, args=(self._task_queues, self._stage_semaphores, - self._completion_queue, self._shutdown_flag)) process.start() self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) - self._task_result_collection_thread = threading.Thread(target=self._task_result_collection_loop, - args=(self._completion_queue, - self._task_futures, - self._shutdown_event)) - self._task_result_collection_thread.start() self._status = PoolStatus.RUNNING @property @@ -134,7 +144,7 @@ def status(self) -> PoolStatus: return self._status @staticmethod - def _worker(task_queues, stage_semaphores, completion_queue, shutdown_flag): + def _worker(task_queues, stage_semaphores, shutdown_flag): logger.debug("Worker process %s has been started.", os.getpid()) while True: @@ -165,56 +175,26 @@ def _worker(task_queues, stage_semaphores, completion_queue, shutdown_flag): args = task.args kwargs = task.kwargs - task_result = TaskResult(task.id, None, None) try: result = process_fn(*args, **kwargs) - task_result.result = result + task.set_result(result) except Exception as e: - task_result.exception = e - - try: - completion_queue.put(task_result) - # the result must be serializable - except (pickle.PicklingError, TypeError) as e: - task_result.exception = e - task_result.result = None - completion_queue.put(task_result) + task.set_exception(e) semaphore.release() time.sleep(0.1) # Avoid busy-waiting - @staticmethod - def _task_result_collection_loop(completion_queue, task_futures, shutdown_event): - while True: - if shutdown_event.is_set() and completion_queue.empty(): - logger.debug("Task result collection process has been terminated.") - return - try: - task_result = completion_queue.get_nowait() - - task_id = task_result.id - future = task_futures.pop(task_id) - if task_result.exception is not None: - future.set_exception(task_result.exception) - else: - future.set_result(task_result.result) - - except queue.Empty: - time.sleep(0.1) - - def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Future: + def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Task: """ Submit a task to the corresponding task queue of the stage. """ with self._lock: - task = Task(uuid.uuid4(), process_fn, args, kwargs) - future = Future() - self._task_futures[task.id] = future + task = Task(self._manager, process_fn, args, kwargs) self._task_queues[stage_name].put(task) - return future + return task def set_usage(self, stage_name, percentage): """ @@ -245,18 +225,12 @@ def shutdown(self): self._status = PoolStatus.STOPPING - while self._task_futures: - time.sleep(0.1) - self._shutdown_flag.value = True for i, p in enumerate(self._processes): p.join() logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) - self._shutdown_event.set() - self._task_result_collection_thread.join() - self._manager.shutdown() self._shutdown = True self._status = PoolStatus.SHUTDOWN diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 9db1b453cd..84d78e3b13 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -63,20 +63,20 @@ def _arbitrary_function(*args, **kwargs): return args, kwargs -def _task_submit_worker(pool, stage_name, task_size, num_tasks): - future_list = [] - for i in range(num_tasks): - future_list.append(pool.submit_task(stage_name, _matrix_multiplication_task, task_size)) - logging.debug("Task %s/%s has been submitted to stage %s.", i + 1, num_tasks, stage_name) +# def _task_submit_worker(pool, stage_name, task_size, num_tasks): +# future_list = [] +# for i in range(num_tasks): +# future_list.append(pool.submit_task(stage_name, _matrix_multiplication_task, task_size)) +# logging.debug("Task %s/%s has been submitted to stage %s.", i + 1, num_tasks, stage_name) - for future in future_list: - future.result() - logging.debug("task number %s has been completed in stage: %s", future_list.index(future), stage_name) +# for future in future_list: +# future.result() +# logging.debug("task number %s has been completed in stage: %s", future_list.index(future), stage_name) - logging.debug("All tasks in stage %s have been completed in %.2f seconds.", - stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) +# logging.debug("All tasks in stage %s have been completed in %.2f seconds.", +# stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) - assert len(future_list) == num_tasks +# assert len(future_list) == num_tasks def test_singleton(): @@ -94,14 +94,14 @@ def test_single_task(shared_process_pool): a = 10 b = 20 - future = pool.submit_task("test_stage", _simple_add_task, a, b) - assert future.result() == a + b + task = pool.submit_task("test_stage", _simple_add_task, a, b) + assert task.result() == a + b - future = pool.submit_task("test_stage", _simple_add_task, a=a, b=b) - assert future.result() == a + b + task = pool.submit_task("test_stage", _simple_add_task, a=a, b=b) + assert task.result() == a + b - future = pool.submit_task("test_stage", _simple_add_task, a, b=b) - assert future.result() == a + b + task = pool.submit_task("test_stage", _simple_add_task, a, b=b) + assert task.result() == a + b def test_multiple_tasks(shared_process_pool): @@ -110,11 +110,11 @@ def test_multiple_tasks(shared_process_pool): pool.set_usage("test_stage", 0.5) num_tasks = 100 - futures = [] + tasks = [] for _ in range(num_tasks): - futures.append(pool.submit_task("test_stage", _simple_add_task, 10, 20)) + tasks.append(pool.submit_task("test_stage", _simple_add_task, 10, 20)) - for future in futures: + for future in tasks: assert future.result() == 30 @@ -124,8 +124,8 @@ def test_error_process_function(shared_process_pool): pool.set_usage("test_stage", 0.5) with pytest.raises(ValueError): - future = pool.submit_task("test_stage", _process_func_with_exception) - future.result() + task = pool.submit_task("test_stage", _process_func_with_exception) + task.result() def test_unserializable_function(shared_process_pool): @@ -133,9 +133,9 @@ def test_unserializable_function(shared_process_pool): pool.set_usage("test_stage", 0.5) - future = pool.submit_task("test_stage", _unserializable_function) + task = pool.submit_task("test_stage", _unserializable_function) with pytest.raises(TypeError): - future.result() + task.result() def test_unserializable_arg(shared_process_pool): @@ -147,30 +147,30 @@ def test_unserializable_arg(shared_process_pool): pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) -def test_multiple_stages(shared_process_pool): - pool = shared_process_pool +# def test_multiple_stages(shared_process_pool): +# pool = shared_process_pool - pool.set_usage("test_stage_1", 0.1) - pool.set_usage("test_stage_2", 0.3) - pool.set_usage("test_stage_3", 0.6) +# pool.set_usage("test_stage_1", 0.1) +# pool.set_usage("test_stage_2", 0.3) +# pool.set_usage("test_stage_3", 0.6) - task_size = 3 - task_num = 3 - # tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), - # ("test_stage_3", task_size, task_num)] - tasks = [("test_stage_1", task_size, task_num)] +# task_size = 3 +# task_num = 3 +# # tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), +# # ("test_stage_3", task_size, task_num)] +# tasks = [("test_stage_1", task_size, task_num)] - processes = [] - for task in tasks: - stage_name, task_size, num_tasks = task - p = mp.Process(target=_task_submit_worker, args=(pool, stage_name, task_size, num_tasks)) - processes.append(p) +# processes = [] +# for task in tasks: +# stage_name, task_size, num_tasks = task +# p = mp.Process(target=_task_submit_worker, args=(pool, stage_name, task_size, num_tasks)) +# processes.append(p) - for p in processes: - p.start() +# for p in processes: +# p.start() - for p in processes: - p.join() +# for p in processes: +# p.join() def test_invalid_stage_usage(shared_process_pool): @@ -214,4 +214,4 @@ def test_task_completion_before_shutdown(shared_process_pool): # all tasks should be completed before shutdown assert len(futures) == 3 * task_num for future in futures: - assert future.done() + assert future._done.is_set() From 31e2247ff1820f0cc1244303187bbbc7e22c63fc Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Mon, 16 Sep 2024 16:55:32 -0700 Subject: [PATCH 15/30] Add pool.status, need debug on stop elegantly --- .../morpheus/utils/shared_process_pool.py | 76 +++++++++++-------- tests/utils/test_shared_process_pool.py | 12 +-- 2 files changed, 53 insertions(+), 35 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 16e5dde1f1..e81eb0b50e 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -35,6 +35,7 @@ class PoolStatus(Enum): class SimpleFuture: + def __init__(self, manager): self._result = manager.Value("i", None) self._exception = manager.Value("i", None) @@ -56,6 +57,7 @@ def result(self): class Task(SimpleFuture): + def __init__(self, manager, process_fn, args, kwargs): super().__init__(manager) self._process_fn = process_fn @@ -80,7 +82,6 @@ class SharedProcessPool: _instance = None _lock = Lock() - _shutdown = False def __new__(cls): """ @@ -109,7 +110,6 @@ def _initialize(self, total_max_workers): """ Initialize a concurrent.futures.ProcessPoolExecutor instance. """ - self._status = PoolStatus.INITIALIZING self._total_max_workers = total_max_workers self._processes = [] @@ -119,21 +119,19 @@ def _initialize(self, total_max_workers): self._context = mp.get_context("fork") self._manager = self._context.Manager() + self._status = self._manager.Value(PoolStatus, PoolStatus.INITIALIZING) + self._task_queues = self._manager.dict() self._stage_semaphores = self._manager.dict() - self._shutdown_flag = self._manager.Value("b", False) - for i in range(total_max_workers): process = self._context.Process(target=self._worker, - args=(self._task_queues, - self._stage_semaphores, - self._shutdown_flag)) + args=(self._task_queues, self._stage_semaphores, self._status)) process.start() self._processes.append(process) logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) - self._status = PoolStatus.RUNNING + self._status.value = PoolStatus.RUNNING @property def total_max_workers(self): @@ -141,14 +139,14 @@ def total_max_workers(self): @property def status(self) -> PoolStatus: - return self._status + return self._status # type: ignore @staticmethod - def _worker(task_queues, stage_semaphores, shutdown_flag): + def _worker(task_queues, stage_semaphores, pool_status): logger.debug("Worker process %s has been started.", os.getpid()) while True: - if shutdown_flag.value: + if pool_status.value == PoolStatus.STOPPING: logger.debug("Worker process %s has been terminated.", os.getpid()) return @@ -166,11 +164,6 @@ def _worker(task_queues, stage_semaphores, shutdown_flag): semaphore.release() continue - if task is None: - logger.warning("Worker process %s received a None task.", os.getpid()) - semaphore.release() - continue - process_fn = task.process_fn args = task.args kwargs = task.kwargs @@ -185,11 +178,13 @@ def _worker(task_queues, stage_semaphores, shutdown_flag): time.sleep(0.1) # Avoid busy-waiting - def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Task: """ Submit a task to the corresponding task queue of the stage. """ + if self._status.value != PoolStatus.RUNNING: + raise ValueError("Cannot submit a task to a SharedProcessPool that is not running.") + with self._lock: task = Task(self._manager, process_fn, args, kwargs) self._task_queues[stage_name].put(task) @@ -220,22 +215,43 @@ def set_usage(self, stage_name, percentage): logger.debug("stage_usage: %s", self._stage_usage) logger.debug("stage semaphores: %s", allowed_processes_num) - def shutdown(self): - if self._status != PoolStatus.SHUTDOWN: + def stop(self): + if self._status.value != PoolStatus.RUNNING: + raise ValueError("Cannot stop a SharedProcessPool that is not running.") + + self._status.value = PoolStatus.STOPPING + for i, p in enumerate(self._processes): + p.join() + logger.debug("Process %s/%s has been joined.", i + 1, self._total_max_workers) + + logger.debug("All tasks have been completed. SharedProcessPool has been stopped.") + self._status.value = PoolStatus.SHUTDOWN + + def kill(self): + if self._status.value != PoolStatus.RUNNING: + raise ValueError("Cannot kill a SharedProcessPool that is not running.") + + for i, p in enumerate(self._processes): + p.terminate() + logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) - self._status = PoolStatus.STOPPING + logger.debug("SharedProcessPool has been killed.") + self._status.value = PoolStatus.SHUTDOWN - self._shutdown_flag.value = True + def join(self, timeout=0): + start_time = time.time() + while self._status.value != PoolStatus.SHUTDOWN: + if time.time() - start_time > timeout: + raise TimeoutError("Join operation has timed out.") + time.sleep(0.1) + logger.debug("SharedProcessPool has been shutdown.") - for i, p in enumerate(self._processes): - p.join() - logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + def reset(self): + if self._status.value != PoolStatus.SHUTDOWN: + raise ValueError("Cannot reset a SharedProcessPool that is not shutdown.") - self._manager.shutdown() - self._shutdown = True - self._status = PoolStatus.SHUTDOWN - logger.debug("Process pool has been terminated.") + self._initialize(self._total_max_workers) def __del__(self): - if self._status != PoolStatus.SHUTDOWN: - self.shutdown() + if self._status.value != PoolStatus.SHUTDOWN: + self.stop() diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 84d78e3b13..a8629f33d3 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -14,7 +14,6 @@ # limitations under the License. import logging -import multiprocessing as mp import threading import time @@ -35,7 +34,9 @@ def shared_process_pool_fixture(): yield pool - pool.shutdown() + print("Pool process in fixture before stop: " + str(pool.status)) + pool.stop() + pool.reset() def _matrix_multiplication_task(size): @@ -79,7 +80,7 @@ def _arbitrary_function(*args, **kwargs): # assert len(future_list) == num_tasks -def test_singleton(): +def test_singleton(shared_process_pool): pool_1 = SharedProcessPool() pool_2 = SharedProcessPool() @@ -88,6 +89,7 @@ def test_singleton(): def test_single_task(shared_process_pool): pool = shared_process_pool + print("Pool status: " + str(pool.status)) pool.set_usage("test_stage", 0.5) @@ -209,9 +211,9 @@ def test_task_completion_before_shutdown(shared_process_pool): futures.append(pool.submit_task("test_stage_2", _matrix_multiplication_task, task_size)) futures.append(pool.submit_task("test_stage_3", _matrix_multiplication_task, task_size)) - pool.shutdown() + pool.stop() - # all tasks should be completed before shutdown + # all tasks should be completed before stopping the pool assert len(futures) == 3 * task_num for future in futures: assert future._done.is_set() From 6a7cf6ea3e604d96464c7c2193399f4f88d3bccd Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 17 Sep 2024 11:48:06 -0700 Subject: [PATCH 16/30] ut passed --- .../morpheus/utils/shared_process_pool.py | 15 ++++--- tests/utils/test_shared_process_pool.py | 44 +------------------ 2 files changed, 11 insertions(+), 48 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index e81eb0b50e..4911777f19 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -195,6 +195,9 @@ def set_usage(self, stage_name, percentage): """ Set the maximum percentage of processes that can be used by each stage. """ + if self._status.value != PoolStatus.RUNNING: + raise ValueError("Cannot set usage to a SharedProcessPool that is not running.") + if not 0 <= percentage <= 1: raise ValueError("Percentage must be between 0 and 1.") @@ -216,8 +219,9 @@ def set_usage(self, stage_name, percentage): logger.debug("stage semaphores: %s", allowed_processes_num) def stop(self): - if self._status.value != PoolStatus.RUNNING: - raise ValueError("Cannot stop a SharedProcessPool that is not running.") + if self._status.value not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): + logger.info("Cannot stop a SharedProcessPool that is not running.") + return self._status.value = PoolStatus.STOPPING for i, p in enumerate(self._processes): @@ -228,8 +232,9 @@ def stop(self): self._status.value = PoolStatus.SHUTDOWN def kill(self): - if self._status.value != PoolStatus.RUNNING: - raise ValueError("Cannot kill a SharedProcessPool that is not running.") + if self._status.value not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): + logger.info("Cannot kill a SharedProcessPool that is not running.") + return for i, p in enumerate(self._processes): p.terminate() @@ -248,7 +253,7 @@ def join(self, timeout=0): def reset(self): if self._status.value != PoolStatus.SHUTDOWN: - raise ValueError("Cannot reset a SharedProcessPool that is not shutdown.") + raise ValueError("Cannot reset a SharedProcessPool that is not already shutdown.") self._initialize(self._total_max_workers) diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index a8629f33d3..eab67216ef 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -64,23 +64,7 @@ def _arbitrary_function(*args, **kwargs): return args, kwargs -# def _task_submit_worker(pool, stage_name, task_size, num_tasks): -# future_list = [] -# for i in range(num_tasks): -# future_list.append(pool.submit_task(stage_name, _matrix_multiplication_task, task_size)) -# logging.debug("Task %s/%s has been submitted to stage %s.", i + 1, num_tasks, stage_name) - -# for future in future_list: -# future.result() -# logging.debug("task number %s has been completed in stage: %s", future_list.index(future), stage_name) - -# logging.debug("All tasks in stage %s have been completed in %.2f seconds.", -# stage_name, (future_list[-1].result()[1] - future_list[0].result()[1])) - -# assert len(future_list) == num_tasks - - -def test_singleton(shared_process_pool): +def test_singleton(): pool_1 = SharedProcessPool() pool_2 = SharedProcessPool() @@ -149,32 +133,6 @@ def test_unserializable_arg(shared_process_pool): pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) -# def test_multiple_stages(shared_process_pool): -# pool = shared_process_pool - -# pool.set_usage("test_stage_1", 0.1) -# pool.set_usage("test_stage_2", 0.3) -# pool.set_usage("test_stage_3", 0.6) - -# task_size = 3 -# task_num = 3 -# # tasks = [("test_stage_1", task_size, task_num), ("test_stage_2", task_size, task_num), -# # ("test_stage_3", task_size, task_num)] -# tasks = [("test_stage_1", task_size, task_num)] - -# processes = [] -# for task in tasks: -# stage_name, task_size, num_tasks = task -# p = mp.Process(target=_task_submit_worker, args=(pool, stage_name, task_size, num_tasks)) -# processes.append(p) - -# for p in processes: -# p.start() - -# for p in processes: -# p.join() - - def test_invalid_stage_usage(shared_process_pool): pool = shared_process_pool From fb2e877cc25803801791c8e81eda9ec099e89b46 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:17:23 -0700 Subject: [PATCH 17/30] Improved and passed unit tests --- .../morpheus/utils/shared_process_pool.py | 276 ++++++++++++++---- tests/utils/test_shared_process_pool.py | 245 +++++++++++----- 2 files changed, 391 insertions(+), 130 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 4911777f19..f78ebfb991 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -55,6 +55,9 @@ def result(self): raise self._exception.value return self._result.value + def done(self): + return self._done.is_set() + class Task(SimpleFuture): @@ -77,6 +80,18 @@ def kwargs(self): return self._kwargs +class CancellationToken: + + def __init__(self, manager): + self._cancel_event = manager.Event() + + def cancel(self): + self._cancel_event.set() + + def is_cancelled(self): + return self._cancel_event.is_set() + + # pylint: disable=W0201 class SharedProcessPool: @@ -85,53 +100,63 @@ class SharedProcessPool: def __new__(cls): """ - Initialize as a singleton. - """ - logger.debug("Creating a new instance of SharedProcessPool...") - with cls._lock: + Singleton pattern for SharedProcessPool. + + Returns + ------- + cls._instance : SharedProcessPool + The SharedProcessPool instance. - max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.5)) + Raises + ------ + RuntimeError + If SharedProcessPool() is called when the instance already exists but not running + """ + with cls._lock: if cls._instance is None: + logger.info("SharedProcessPool.__new__: Creating a new instance...") cls._instance = super().__new__(cls) - cls._instance._initialize(max_workers) - logger.debug("SharedProcessPool has been initialized with %s workers.", max_workers) + cls._instance._initialize() + logger.info("SharedProcessPool has been initialized.") else: if cls._instance.status is not PoolStatus.RUNNING: - cls._instance._initialize(max_workers) - logger.debug("SharedProcessPool has been initialized with %s workers.", max_workers) + raise RuntimeError( + "SharedProcessPool instance already exists but it is not running. Please use start() or reset() to launch the pool." + ) else: - logger.debug("SharedProcessPool instance already exists and currently running.") + logger.debug("SharedProcessPool.__new__: instance already exists and is currently running.") return cls._instance - def _initialize(self, total_max_workers): - """ - Initialize a concurrent.futures.ProcessPoolExecutor instance. - """ + def _initialize(self): + self._status = PoolStatus.INITIALIZING - self._total_max_workers = total_max_workers + self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.5)) self._processes = [] - self._total_usage = 0.0 - self._stage_usage = {} self._context = mp.get_context("fork") self._manager = self._context.Manager() - - self._status = self._manager.Value(PoolStatus, PoolStatus.INITIALIZING) - self._task_queues = self._manager.dict() self._stage_semaphores = self._manager.dict() + self._total_usage = 0.0 + self._stage_usage = {} - for i in range(total_max_workers): + self._cancellation_token = CancellationToken(self._manager) + self._launch_workers() + + self._status = PoolStatus.RUNNING + + def _launch_workers(self): + for i in range(self.total_max_workers): process = self._context.Process(target=self._worker, - args=(self._task_queues, self._stage_semaphores, self._status)) + args=(self._cancellation_token, self._task_queues, self._stage_semaphores)) process.start() self._processes.append(process) - logger.debug("Process %s/%s has been started.", i + 1, total_max_workers) - - self._status.value = PoolStatus.RUNNING + logger.debug("SharedProcessPool._lanch_workers(): Process %s/%s has been started.", + i + 1, + self.total_max_workers) @property def total_max_workers(self): @@ -139,18 +164,18 @@ def total_max_workers(self): @property def status(self) -> PoolStatus: - return self._status # type: ignore + return self._status @staticmethod - def _worker(task_queues, stage_semaphores, pool_status): - logger.debug("Worker process %s has been started.", os.getpid()) + def _worker(cancellation_token, task_queues, stage_semaphores): + logger.debug("SharedProcessPool._worker: Worker process %s has been started.", os.getpid()) while True: - if pool_status.value == PoolStatus.STOPPING: - logger.debug("Worker process %s has been terminated.", os.getpid()) + if cancellation_token.is_cancelled(): + logger.debug("SharedProcessPool._worker: Worker process %s has terminated the worker loop.", + os.getpid()) return - # iterate over every semaphore for stage_name, task_queue in task_queues.items(): semaphore = stage_semaphores[stage_name] @@ -180,23 +205,62 @@ def _worker(task_queues, stage_semaphores, pool_status): def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Task: """ - Submit a task to the corresponding task queue of the stage. + Submit a task to the SharedProcessPool. + + Parameters + ---------- + stage_name : str + The unique name of the stage. + process_fn : Callable + The function to be executed in the process pool. + args : Any + Arbitrary arguments for the process_fn. + kwargs : Any + Arbitrary keyword arguments for the process_fn. + + Returns + ------- + Task + The task object that includes the result of the process_fn. + + Raises + ------ + RuntimeError + If the SharedProcessPool is not running. + ValueError + If the stage_name has not been set in the SharedProcessPool. """ - if self._status.value != PoolStatus.RUNNING: - raise ValueError("Cannot submit a task to a SharedProcessPool that is not running.") + if self._status != PoolStatus.RUNNING: + raise RuntimeError("Cannot submit a task to a SharedProcessPool that is not running.") - with self._lock: - task = Task(self._manager, process_fn, args, kwargs) - self._task_queues[stage_name].put(task) + if stage_name not in self._stage_usage: + raise ValueError(f"Stage {stage_name} has not been set in SharedProcessPool.") - return task + task = Task(self._manager, process_fn, args, kwargs) + self._task_queues[stage_name].put(task) + + return task def set_usage(self, stage_name, percentage): """ - Set the maximum percentage of processes that can be used by each stage. + Set the usage of the SharedProcessPool for a specific stage. + + Parameters + ---------- + stage_name : str + The unique name of the stage. + percentage : float + The percentage of the total workers that will be allocated to the stage, should be between 0 and 1. + + Raises + ------ + RuntimeError + If the SharedProcessPool is not running. + ValueError + If the percentage is not between 0 and 1 or the total usage is greater than 1. """ - if self._status.value != PoolStatus.RUNNING: - raise ValueError("Cannot set usage to a SharedProcessPool that is not running.") + if self._status != PoolStatus.RUNNING: + raise RuntimeError("Cannot set usage to a SharedProcessPool that is not running.") if not 0 <= percentage <= 1: raise ValueError("Percentage must be between 0 and 1.") @@ -215,48 +279,134 @@ def set_usage(self, stage_name, percentage): if stage_name not in self._task_queues: self._task_queues[stage_name] = self._manager.Queue() - logger.debug("stage_usage: %s", self._stage_usage) - logger.debug("stage semaphores: %s", allowed_processes_num) + logger.debug("SharedProcessPool.set_usage(): stage_usage: %s", self._stage_usage) + logger.debug("SharedProcessPool.set_usage(): stage semaphores: %s", allowed_processes_num) + + def start(self): + """ + Start the SharedProcessPool that is currently stopped and keep the settings before last shutdown. + + Raises + ------ + RuntimeError + If the SharedProcessPool is not shutdown. + """ + if self._status != PoolStatus.SHUTDOWN: + raise RuntimeError("Cannot start a SharedProcessPool that is not shutdown.") + + self._launch_workers() + self._status = PoolStatus.RUNNING + + def reset(self): + """ + Clear all the previous settings and restart the SharedProcessPool. + + Raises + ------ + RuntimeError + If the SharedProcessPool is not already shutdown. + """ + if self._status != PoolStatus.SHUTDOWN: + raise RuntimeError( + "SharedProcessPool.reset(): Cannot reset a SharedProcessPool that is not already shutdown.") + + self._initialize() def stop(self): - if self._status.value not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): - logger.info("Cannot stop a SharedProcessPool that is not running.") + """ + Complete existing tasks and stop the SharedProcessPool. + """ + if self._status not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): + logger.info("SharedProcessPool.stop(): Cannot stop a SharedProcessPool that is not running.") return - self._status.value = PoolStatus.STOPPING + # no new tasks will be accepted from this point + self._status = PoolStatus.STOPPING + + # wait for all task queues to be empty + task_queue_count = len(self._task_queues) + empty_task_queues = set() + while len(empty_task_queues) < task_queue_count: + for stage_name, task_queue in self._task_queues.items(): + if task_queue.empty(): + empty_task_queues.add(stage_name) + + self._cancellation_token.cancel() + for i, p in enumerate(self._processes): p.join() logger.debug("Process %s/%s has been joined.", i + 1, self._total_max_workers) - logger.debug("All tasks have been completed. SharedProcessPool has been stopped.") - self._status.value = PoolStatus.SHUTDOWN + logger.debug("SharedProcessPool.stop(): All tasks have been completed. SharedProcessPool has been stopped.") + self._status = PoolStatus.SHUTDOWN - def kill(self): - if self._status.value not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): - logger.info("Cannot kill a SharedProcessPool that is not running.") + def terminate(self): + """ + Terminate all processes and shutdown the SharedProcessPool immediately. + """ + if self._status not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): + logger.info("SharedProcessPool.terminate():Cannot terminate a SharedProcessPool that is not running.") return for i, p in enumerate(self._processes): p.terminate() logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) - logger.debug("SharedProcessPool has been killed.") - self._status.value = PoolStatus.SHUTDOWN + logger.debug("SharedProcessPool.terminate(): SharedProcessPool has been terminated.") + self._status = PoolStatus.SHUTDOWN + + def wait_until_ready(self, timeout=None): + """ + Wait until the SharedProcessPool is running and ready to accept tasks. + + Parameters + ---------- + timeout : _type_, optional + timeout in seconds to wait for the SharedProcessPool to be ready, by default None. + If None, it will wait indefinitely. + + Raises + ------ + RuntimeError + If the SharedProcessPool is not initializing or running. + TimeoutError + If has been waiting more than the timeout. + """ + if self.status not in (PoolStatus.INITIALIZING, PoolStatus.RUNNING): + raise RuntimeError("Cannot wait for a SharedProcessPool that is not initializing.") - def join(self, timeout=0): start_time = time.time() - while self._status.value != PoolStatus.SHUTDOWN: - if time.time() - start_time > timeout: - raise TimeoutError("Join operation has timed out.") + while self.status != PoolStatus.RUNNING: + if timeout is not None and timeout > 0 and time.time() - start_time > timeout: + raise TimeoutError("SharedProcessPool wait_until_ready has timed out.") time.sleep(0.1) - logger.debug("SharedProcessPool has been shutdown.") - def reset(self): - if self._status.value != PoolStatus.SHUTDOWN: - raise ValueError("Cannot reset a SharedProcessPool that is not already shutdown.") + logger.debug("SharedProcessPool.wait_until_ready(): SharedProcessPool is ready.") + + def join(self, timeout=None): + """ + Wait until the SharedProcessPool is terminated. + + Parameters + ---------- + timeout : _type_, optional + timeout in seconds to wait for the SharedProcessPool to be terminated, by default None. + If None, it will wait indefinitely. + + Raises + ------ + TimeoutError + If has been waiting more than the timeout. + """ + start_time = time.time() + + while self._status != PoolStatus.SHUTDOWN: + if timeout is not None and timeout > 0 and time.time() - start_time > timeout: + raise TimeoutError("SharedProcessPool join has timed out.") + time.sleep(0.1) - self._initialize(self._total_max_workers) + logger.debug("SharedProcessPool.join(): SharedProcessPool has been joined.") def __del__(self): - if self._status.value != PoolStatus.SHUTDOWN: + if self._status != PoolStatus.SHUTDOWN: self.stop() diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index eab67216ef..9370381b4b 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -16,47 +16,43 @@ import logging import threading import time - -import numpy as np +from decimal import Decimal +from fractions import Fraction import pytest +from morpheus.utils.shared_process_pool import PoolStatus from morpheus.utils.shared_process_pool import SharedProcessPool logger = logging.getLogger(__name__) -# set logger level to debug -logger.setLevel(logging.DEBUG) - +# This unit test does NOT work well with the `-s` option of pytest. Run pytest without `-s` flag. @pytest.fixture(name="shared_process_pool") def shared_process_pool_fixture(): - pool = SharedProcessPool() + pool = SharedProcessPool() yield pool - print("Pool process in fixture before stop: " + str(pool.status)) + # Stop and reset the pool after each test pool.stop() + pool.join() pool.reset() -def _matrix_multiplication_task(size): - matrix_a = np.random.rand(size, size) - matrix_b = np.random.rand(size, size) - - mul = np.dot(matrix_a, matrix_b) - result = (mul, time.time()) - return result +def _add_task(x, y): + return x + y -def _simple_add_task(a, b): - return a + b +def _time_consuming_task(sleep_time): + time.sleep(sleep_time) + return sleep_time -def _process_func_with_exception(): - raise ValueError("Exception is raised in the process.") +def _function_raises_exception(): + raise RuntimeError("Exception is raised in the process.") -def _unserializable_function(): +def _function_returns_unserializable_result(): return threading.Lock() @@ -64,114 +60,229 @@ def _arbitrary_function(*args, **kwargs): return args, kwargs +def _check_pool_stage_settings(pool: SharedProcessPool, stage_name: str, usage: float): + + assert pool._stage_usage.get(stage_name) == usage + assert stage_name in pool._stage_semaphores + assert stage_name in pool._task_queues + + def test_singleton(): + pool_1 = SharedProcessPool() pool_2 = SharedProcessPool() assert pool_1 is pool_2 -def test_single_task(shared_process_pool): +def test_pool_status(shared_process_pool): + pool = shared_process_pool - print("Pool status: " + str(pool.status)) + pool.wait_until_ready() + assert pool.status == PoolStatus.RUNNING pool.set_usage("test_stage", 0.5) - a = 10 - b = 20 - - task = pool.submit_task("test_stage", _simple_add_task, a, b) - assert task.result() == a + b + pool.stop() + pool.join() + assert pool.status == PoolStatus.SHUTDOWN - task = pool.submit_task("test_stage", _simple_add_task, a=a, b=b) - assert task.result() == a + b + # With pool.start(), the pool should have the same status as before stopping + pool.start() + pool.wait_until_ready() + assert pool.status == PoolStatus.RUNNING + assert pool._total_usage == 0.5 + _check_pool_stage_settings(pool, "test_stage", 0.5) - task = pool.submit_task("test_stage", _simple_add_task, a, b=b) - assert task.result() == a + b + pool.terminate() + pool.join() + assert pool.status == PoolStatus.SHUTDOWN + # With pool.reset(), the pool should reset all the status + pool.reset() + pool.wait_until_ready() + assert pool.status == PoolStatus.RUNNING + assert pool._total_usage == 0.0 + assert not pool._stage_usage + assert not pool._stage_semaphores + assert not pool._task_queues + + +@pytest.mark.parametrize( + "a, b, expected", + [ + (1, 2, 3), # Integers + (complex(1, 2), complex(3, 4), complex(4, 6)), # Complex numbers + (Decimal('1.1'), Decimal('2.2'), Decimal('3.3')), # Decimal numbers + (Fraction(1, 2), Fraction(1, 3), Fraction(5, 6)), # Fractions + ("Hello, ", "world!", "Hello, world!"), # Strings + ([1, 2, 3], [4, 5, 6], [1, 2, 3, 4, 5, 6]), # Lists + ((1, 2, 3), (4, 5, 6), (1, 2, 3, 4, 5, 6)), # Tuples + ]) +def test_submit_single_task(shared_process_pool, a, b, expected): -def test_multiple_tasks(shared_process_pool): pool = shared_process_pool - + pool.wait_until_ready() pool.set_usage("test_stage", 0.5) - num_tasks = 100 - tasks = [] - for _ in range(num_tasks): - tasks.append(pool.submit_task("test_stage", _simple_add_task, 10, 20)) + task = pool.submit_task("test_stage", _add_task, a, b) + assert task.result() == expected - for future in tasks: - assert future.result() == 30 + task = pool.submit_task("test_stage", _add_task, x=a, y=b) + assert task.result() == expected + + task = pool.submit_task("test_stage", _add_task, a, y=b) + assert task.result() == expected + + pool.stop() + pool.join() + + # After the pool is shutdown, it should not accept any new tasks + with pytest.raises(RuntimeError): + pool.submit_task("test_stage", _add_task, 10, 20) -def test_error_process_function(shared_process_pool): +def test_submit_task_with_invalid_stage(shared_process_pool): + pool = shared_process_pool + pool.wait_until_ready() + + with pytest.raises(ValueError): + pool.submit_task("stage_does_not_exist", _add_task, 10, 20) + +def test_submit_task_raises_exception(shared_process_pool): + + pool = shared_process_pool pool.set_usage("test_stage", 0.5) - with pytest.raises(ValueError): - task = pool.submit_task("test_stage", _process_func_with_exception) + task = pool.submit_task("test_stage", _function_raises_exception) + with pytest.raises(RuntimeError): task.result() -def test_unserializable_function(shared_process_pool): - pool = shared_process_pool +def test_submit_task_with_unserializable_result(shared_process_pool): + pool = shared_process_pool pool.set_usage("test_stage", 0.5) - task = pool.submit_task("test_stage", _unserializable_function) + task = pool.submit_task("test_stage", _function_returns_unserializable_result) with pytest.raises(TypeError): task.result() -def test_unserializable_arg(shared_process_pool): - pool = shared_process_pool +def test_submit_task_with_unserializable_arg(shared_process_pool): + pool = shared_process_pool pool.set_usage("test_stage", 0.5) + # Unserializable arguments cannot be submitted to the pool with pytest.raises(TypeError): pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) +@pytest.mark.parametrize( + "a, b, expected", + [ + (1, 2, 3), # Integers + (complex(1, 2), complex(3, 4), complex(4, 6)), # Complex numbers + (Decimal('1.1'), Decimal('2.2'), Decimal('3.3')), # Decimal numbers + (Fraction(1, 2), Fraction(1, 3), Fraction(5, 6)), # Fractions + ("Hello, ", "world!", "Hello, world!"), # Strings + ([1, 2, 3], [4, 5, 6], [1, 2, 3, 4, 5, 6]), # Lists + ((1, 2, 3), (4, 5, 6), (1, 2, 3, 4, 5, 6)), # Tuples + ]) +def test_submit_multiple_tasks(shared_process_pool, a, b, expected): -def test_invalid_stage_usage(shared_process_pool): pool = shared_process_pool + pool.set_usage("test_stage", 0.5) - with pytest.raises(ValueError): - pool.set_usage("test_stage", 1.1) + num_tasks = 100 + tasks = [] + for _ in range(num_tasks): + tasks.append(pool.submit_task("test_stage", _add_task, a, b)) + + for future in tasks: + assert future.result() == expected - with pytest.raises(ValueError): - pool.set_usage("test_stage", -0.1) + +def test_set_usage(shared_process_pool): + + pool = shared_process_pool + pool.wait_until_ready() pool.set_usage("test_stage_1", 0.5) - pool.set_usage("test_stage_2", 0.4) + assert pool._total_usage == 0.5 + _check_pool_stage_settings(pool, "test_stage_1", 0.5) + + pool.set_usage("test_stage_2", 0.3) + assert pool._total_usage == 0.8 + _check_pool_stage_settings(pool, "test_stage_2", 0.3) + + # valid update to the usage of an existing stage + pool.set_usage("test_stage_1", 0.6) + assert pool._total_usage == 0.9 + _check_pool_stage_settings(pool, "test_stage_1", 0.6) - pool.set_usage("test_stage_1", 0.6) # ok to update the usage of an existing stage + # invalid update to the usage of an existing stage, exceeding the total usage limit + with pytest.raises(ValueError): + pool.set_usage("test_stage_1", 0.8) + + # adding a new stage usage, exceeding the total usage limit + with pytest.raises(ValueError): + pool.set_usage("test_stage_3", 0.2) with pytest.raises(ValueError): - pool.set_usage("test_stage_1", 0.7) # not ok to exceed the total usage limit after updating + pool.set_usage("test_stage_1", 1.1) with pytest.raises(ValueError): - pool.set_usage("test_stage_3", 0.1) + pool.set_usage("test_stage_1", -0.1) + # invalid settings should not change the pool status + _check_pool_stage_settings(pool, "test_stage_1", 0.6) + assert pool._total_usage == 0.9 -def test_task_completion_before_shutdown(shared_process_pool): - pool = shared_process_pool +def test_task_completion_with_early_stop(shared_process_pool): + + pool = shared_process_pool pool.set_usage("test_stage_1", 0.1) pool.set_usage("test_stage_2", 0.3) pool.set_usage("test_stage_3", 0.5) - task_size = 3 - task_num = 3 - futures = [] + tasks = [] + + task_num = 5 + sleep_time = 10 for _ in range(task_num): - futures.append(pool.submit_task("test_stage_1", _matrix_multiplication_task, task_size)) - futures.append(pool.submit_task("test_stage_2", _matrix_multiplication_task, task_size)) - futures.append(pool.submit_task("test_stage_3", _matrix_multiplication_task, task_size)) + tasks.append(pool.submit_task("test_stage_1", _time_consuming_task, sleep_time)) + tasks.append(pool.submit_task("test_stage_2", _time_consuming_task, sleep_time)) + tasks.append(pool.submit_task("test_stage_3", _time_consuming_task, sleep_time)) pool.stop() + pool.join() + + # all tasks should be completed before the pool is shutdown + assert len(tasks) == 3 * task_num + for task in tasks: + assert task.done() + + +def test_terminate_running_tasks(shared_process_pool): + + pool = shared_process_pool + pool.set_usage("test_stage_1", 0.1) + pool.set_usage("test_stage_2", 0.3) + pool.set_usage("test_stage_3", 0.5) + + tasks = [] + + task_num = 5 + sleep_time = 100000 + for _ in range(task_num): + tasks.append(pool.submit_task("test_stage_1", _time_consuming_task, sleep_time)) + tasks.append(pool.submit_task("test_stage_2", _time_consuming_task, sleep_time)) + tasks.append(pool.submit_task("test_stage_3", _time_consuming_task, sleep_time)) - # all tasks should be completed before stopping the pool - assert len(futures) == 3 * task_num - for future in futures: - assert future._done.is_set() + # The pool should be shutdown immediately after calling terminate() without waiting for the tasks to complete + pool.terminate() + pool.join() From dcda4842be9e5f03cf6b11c84155f683d1c61c7e Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:26:11 -0700 Subject: [PATCH 18/30] Fix formats --- .../morpheus/morpheus/utils/shared_process_pool.py | 13 ++++++------- tests/test_multi_processing_stage.py | 4 ++-- tests/utils/test_shared_process_pool.py | 3 +++ 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index f78ebfb991..5fca2e2f0a 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -122,11 +122,10 @@ def __new__(cls): else: if cls._instance.status is not PoolStatus.RUNNING: - raise RuntimeError( - "SharedProcessPool instance already exists but it is not running. Please use start() or reset() to launch the pool." - ) - else: - logger.debug("SharedProcessPool.__new__: instance already exists and is currently running.") + raise RuntimeError("SharedProcessPool instance already exists but it is not running.\ + Please use start() or reset() to launch the pool.") + + logger.debug("SharedProcessPool.__new__: instance already exists and is currently running.") return cls._instance @@ -377,7 +376,7 @@ def wait_until_ready(self, timeout=None): start_time = time.time() while self.status != PoolStatus.RUNNING: - if timeout is not None and timeout > 0 and time.time() - start_time > timeout: + if timeout is not None and time.time() - start_time > timeout: raise TimeoutError("SharedProcessPool wait_until_ready has timed out.") time.sleep(0.1) @@ -401,7 +400,7 @@ def join(self, timeout=None): start_time = time.time() while self._status != PoolStatus.SHUTDOWN: - if timeout is not None and timeout > 0 and time.time() - start_time > timeout: + if timeout is not None and time.time() - start_time > timeout: raise TimeoutError("SharedProcessPool join has timed out.") time.sleep(0.1) diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index 442c172abb..34fad2cb39 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -33,8 +33,8 @@ from morpheus.stages.general.multi_processing_stage import MultiProcessingStage from morpheus.stages.input.in_memory_data_generation_stage import InMemoryDataGenStage from morpheus.stages.input.in_memory_source_stage import InMemorySourceStage -from morpheus.stages.output.in_memory_sink_stage import InMemorySinkStage from morpheus.stages.output.compare_dataframe_stage import CompareDataFrameStage +from morpheus.stages.output.in_memory_sink_stage import InMemorySinkStage from morpheus.stages.postprocess.serialize_stage import SerializeStage from morpheus.stages.preprocess.deserialize_stage import DeserializeStage @@ -92,7 +92,7 @@ def test_derived_stage_pipe(config: Config, dataset_pandas: DatasetManager): pipe = LinearPipeline(config) pipe.set_source(InMemorySourceStage(config, [cudf.DataFrame(input_df)])) - pipe.add_stage(DeserializeStage(config, ensure_sliceable_index=True, message_type=ControlMessage)) + pipe.add_stage(DeserializeStage(config, ensure_sliceable_index=True)) pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.5, add_column_name=add_column_name)) pipe.add_stage(SerializeStage(config)) comp_stage = pipe.add_stage(CompareDataFrameStage(config, expected_df)) diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 9370381b4b..1d0865b550 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -18,6 +18,7 @@ import time from decimal import Decimal from fractions import Fraction + import pytest from morpheus.utils.shared_process_pool import PoolStatus @@ -27,6 +28,7 @@ # This unit test does NOT work well with the `-s` option of pytest. Run pytest without `-s` flag. + @pytest.fixture(name="shared_process_pool") def shared_process_pool_fixture(): @@ -180,6 +182,7 @@ def test_submit_task_with_unserializable_arg(shared_process_pool): with pytest.raises(TypeError): pool.submit_task("test_stage", _arbitrary_function, threading.Lock()) + @pytest.mark.parametrize( "a, b, expected", [ From 16d46baa7cd2435d441ae097ea36922e2947b5fd Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Tue, 17 Sep 2024 18:39:14 -0700 Subject: [PATCH 19/30] fix format --- python/morpheus/morpheus/utils/shared_process_pool.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 5fca2e2f0a..3292d2ed9c 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -118,7 +118,7 @@ def __new__(cls): logger.info("SharedProcessPool.__new__: Creating a new instance...") cls._instance = super().__new__(cls) cls._instance._initialize() - logger.info("SharedProcessPool has been initialized.") + logger.info("SharedProcessPool.__new__: SharedProcessPool has been initialized.") else: if cls._instance.status is not PoolStatus.RUNNING: @@ -344,7 +344,7 @@ def terminate(self): Terminate all processes and shutdown the SharedProcessPool immediately. """ if self._status not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): - logger.info("SharedProcessPool.terminate():Cannot terminate a SharedProcessPool that is not running.") + logger.info("SharedProcessPool.terminate(): Cannot terminate a SharedProcessPool that is not running.") return for i, p in enumerate(self._processes): From 41de9dfc8dfb84847b331355227f59a1575207c8 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Wed, 18 Sep 2024 12:29:53 -0700 Subject: [PATCH 20/30] update multi_processing_stage --- .../stages/general/multi_processing_stage.py | 104 +++++++++--- .../morpheus/utils/shared_process_pool.py | 26 ++- tests/test_multi_processing_stage.py | 150 +++++++++++++----- 3 files changed, 204 insertions(+), 76 deletions(-) diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index 4fbff7c4c7..520ae0359b 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -13,8 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. +import functools +import inspect import typing -from abc import ABC from abc import abstractmethod import mrc @@ -29,14 +30,14 @@ OutputT = typing.TypeVar('OutputT') -class MultiProcessingBaseStage(SinglePortStage, ABC, typing.Generic[InputT, OutputT]): +class MultiProcessingBaseStage(SinglePortStage, typing.Generic[InputT, OutputT]): def __init__(self, *, c: Config, process_pool_usage: float, max_in_flight_messages: int = None): super().__init__(c=c) self._process_pool_usage = process_pool_usage self._shared_process_pool = SharedProcessPool() - self._shared_process_pool.set_usage(self.name, self._process_pool_usage) + self._shared_process_pool.wait_until_ready() if max_in_flight_messages is None: # set the multiplier to 1.5 to keep the workers busy @@ -45,15 +46,43 @@ def __init__(self, *, c: Config, process_pool_usage: float, max_in_flight_messag self._max_in_flight_messages = max_in_flight_messages @property + @abstractmethod def name(self) -> str: + """ + Marked as abstract to force the derived stage to provide a unique name. + + Returns: + str: The unique name of the stage. + """ return "multi-processing-base-stage" def accepted_types(self) -> typing.Tuple: - return (InputT, ) + if hasattr(self, "__orig_class__"): + # Derived with abstract types + input_type = typing.get_args(self.__orig_class__)[0] # pylint: disable=no-member + + elif hasattr(self, "__orig_bases__"): + # Derived with concrete types + input_type = typing.get_args(self.__orig_bases__[0])[0] # pylint: disable=no-member + + else: + raise RuntimeError("Could not deduct input type") + + return (input_type, ) def compute_schema(self, schema: StageSchema): - for (port_idx, port_schema) in enumerate(schema.input_schemas): - schema.output_schemas[port_idx].set_type(port_schema.get_type()) + if hasattr(self, "__orig_class__"): + # Derived with abstract types + output_type = typing.get_args(self.__orig_class__)[1] # pylint: disable=no-member + + elif hasattr(self, "__orig_bases__"): + # Derived with concrete types + output_type = typing.get_args(self.__orig_bases__[0])[1] + + else: + raise RuntimeError("Could not deduct output type") + + schema.output_schema.set_type(output_type) def supports_cpp_node(self): return False @@ -71,42 +100,65 @@ def _build_single(self, builder: mrc.Builder, input_node: mrc.SegmentObject) -> return node +def _get_func_signature(func: typing.Callable[[InputT], OutputT]) -> tuple[type, type]: + signature = inspect.signature(func) + + if isinstance(func, functools.partial): + # If the function is a partial, find the type of the first unbound argument + bound_args = func.keywords + input_arg = None + + for param in signature.parameters.values(): + if param.name not in bound_args: + input_arg = param + break + + if input_arg is None: + raise ValueError("Could not find unbound argument in partial function") + input_t = input_arg.annotation + + else: + input_t = next(iter(signature.parameters.values())).annotation + + output_t = signature.return_annotation + + return (input_t, output_t) + + class MultiProcessingStage(MultiProcessingBaseStage[InputT, OutputT]): def __init__(self, *, c: Config, - process_pool_usage: float, + unique_name: str, process_fn: typing.Callable[[InputT], OutputT], + process_pool_usage: float, max_in_flight_messages: int = None): super().__init__(c=c, process_pool_usage=process_pool_usage, max_in_flight_messages=max_in_flight_messages) + self._name = unique_name self._process_fn = process_fn + self._shared_process_pool.set_usage(self.name, self._process_pool_usage) @property def name(self) -> str: - return "multi-processing-stage" - - def accepted_types(self) -> typing.Tuple: - input_type = typing.get_args(self.__orig_class__)[0] # pylint: disable=no-member - - return (input_type, ) + return self._name def _on_data(self, data: InputT) -> OutputT: - - future = self._shared_process_pool.submit_task(self.name, self._process_fn, data) - result = future.result() + task = self._shared_process_pool.submit_task(self.name, self._process_fn, data) + result = task.result() return result @staticmethod - def create(*, c: Config, process_fn: typing.Callable[[InputT], OutputT], process_pool_usage: float): - - type_hints = typing.get_type_hints(process_fn) - - input_type = next(iter(type_hints.values())) - output_type = type_hints["return"] - - return MultiProcessingStage[input_type, output_type](c=c, - process_pool_usage=process_pool_usage, - process_fn=process_fn) + def create(*, + c: Config, + unique_name: str, + process_fn: typing.Callable[[InputT], OutputT], + process_pool_usage: float): + + input_t, output_t = _get_func_signature(process_fn) + return MultiProcessingStage[input_t, output_t](c=c, + unique_name=unique_name, + process_pool_usage=process_pool_usage, + process_fn=process_fn) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 3292d2ed9c..3ca8f6ac6e 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -121,18 +121,14 @@ def __new__(cls): logger.info("SharedProcessPool.__new__: SharedProcessPool has been initialized.") else: - if cls._instance.status is not PoolStatus.RUNNING: - raise RuntimeError("SharedProcessPool instance already exists but it is not running.\ - Please use start() or reset() to launch the pool.") - - logger.debug("SharedProcessPool.__new__: instance already exists and is currently running.") + logger.info("SharedProcessPool.__new__: instance already exists.") return cls._instance def _initialize(self): self._status = PoolStatus.INITIALIZING - self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.5)) + self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.1)) self._processes = [] self._context = mp.get_context("fork") @@ -188,6 +184,12 @@ def _worker(cancellation_token, task_queues, stage_semaphores): semaphore.release() continue + if task is None: + logger.warning("SharedProcessPool._worker: Worker process %s has received a None task.", + os.getpid()) + semaphore.release() + continue + process_fn = task.process_fn args = task.args kwargs = task.kwargs @@ -291,7 +293,8 @@ def start(self): If the SharedProcessPool is not shutdown. """ if self._status != PoolStatus.SHUTDOWN: - raise RuntimeError("Cannot start a SharedProcessPool that is not shutdown.") + logger.warning("SharedProcessPool.start(): Cannot start a SharedProcessPool that is not shutdown.") + return self._launch_workers() self._status = PoolStatus.RUNNING @@ -316,7 +319,7 @@ def stop(self): Complete existing tasks and stop the SharedProcessPool. """ if self._status not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): - logger.info("SharedProcessPool.stop(): Cannot stop a SharedProcessPool that is not running.") + logger.warning("SharedProcessPool.stop(): Cannot stop a SharedProcessPool that is not running.") return # no new tasks will be accepted from this point @@ -343,10 +346,6 @@ def terminate(self): """ Terminate all processes and shutdown the SharedProcessPool immediately. """ - if self._status not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): - logger.info("SharedProcessPool.terminate(): Cannot terminate a SharedProcessPool that is not running.") - return - for i, p in enumerate(self._processes): p.terminate() logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) @@ -407,5 +406,4 @@ def join(self, timeout=None): logger.debug("SharedProcessPool.join(): SharedProcessPool has been joined.") def __del__(self): - if self._status != PoolStatus.SHUTDOWN: - self.stop() + self.terminate() diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index 34fad2cb39..f2132d6050 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -14,12 +14,11 @@ # limitations under the License. import os +import typing from functools import partial from typing import Generator -from typing import Tuple import pandas as pd -import pytest import cudf @@ -29,6 +28,7 @@ from morpheus.messages import ControlMessage from morpheus.messages import MessageMeta from morpheus.pipeline import LinearPipeline +from morpheus.pipeline.stage_decorator import stage from morpheus.stages.general.multi_processing_stage import MultiProcessingBaseStage from morpheus.stages.general.multi_processing_stage import MultiProcessingStage from morpheus.stages.input.in_memory_data_generation_stage import InMemoryDataGenStage @@ -39,16 +39,41 @@ from morpheus.stages.preprocess.deserialize_stage import DeserializeStage -def process_fn(pdf: pd.DataFrame) -> pd.DataFrame: +def _create_df(count: int) -> pd.DataFrame: + return pd.DataFrame({"a": range(count)}, {"b": range(count)}) - pdf["new_column"] = "Hello" - return pdf +def _process_df(df: pd.DataFrame, column: str, value: str) -> pd.DataFrame: + df[column] = value + return df -def test_constructor(config: Config): - stage = MultiProcessingStage.create(c=config, process_fn=process_fn, process_pool_usage=0.5) - assert stage.name == "multi-processing-stage" +def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetManager): + + # Test create() with normal function + stage = MultiProcessingStage.create(c=config, + unique_name="multi-processing-stage-1", + process_fn=_create_df, + process_pool_usage=0.1) + assert stage.name == "multi-processing-stage-1" + input_t, output_t = typing.get_args(stage.__orig_class__) # pylint: disable=no-member + assert input_t == int + assert output_t == pd.DataFrame + + # Test create() with partial function + df = dataset_pandas["csv_sample.csv"] + partial_fn = partial(_process_df, df=df, value="new_value") + + stage = MultiProcessingStage.create(c=config, + unique_name="multi-processing-stage-2", + process_fn=partial_fn, + process_pool_usage=0.1) + + assert stage.name == "multi-processing-stage-2" + input_t, output_t = typing.get_args(stage.__orig_class__) # pylint: disable=no-member + assert stage.accepted_types() == (str, ) + assert input_t == str + assert output_t == pd.DataFrame class DerivedMultiProcessingStage(MultiProcessingBaseStage[ControlMessage, ControlMessage]): @@ -62,30 +87,78 @@ def __init__(self, super().__init__(c=c, process_pool_usage=process_pool_usage, max_in_flight_messages=max_in_flight_messages) self._add_column_name = add_column_name + self._shared_process_pool.set_usage(self.name, self._process_pool_usage) @property def name(self) -> str: return "derived-multi-processing-stage" - def accepted_types(self) -> Tuple: - return (ControlMessage, ) - def _on_data(self, data: ControlMessage) -> ControlMessage: - pdf = data.payload().copy_dataframe().to_pandas() + input_df = data.payload().copy_dataframe() + pdf = input_df.to_pandas() + partial_process_fn = partial(_process_df, column=self._add_column_name, value="Hello") + + task = self._shared_process_pool.submit_task(self.name, partial_process_fn, pdf) - future = self._shared_process_pool.submit_task(self.name, process_fn, pdf) - data.payload(MessageMeta(cudf.DataFrame.from_pandas(future.result()))) + df = cudf.DataFrame.from_pandas(task.result()) + meta = MessageMeta(df) + data.payload(meta) return data -@pytest.mark.use_python +def test_derived_stage_type_deduction(config: Config): + + stage = DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name="new_column") + assert stage.name == "derived-multi-processing-stage" + assert stage.accepted_types() == (ControlMessage, ) + + input_t, output_t = typing.get_args(stage.__orig_bases__[0]) # pylint: disable=no-member + assert input_t == ControlMessage + assert output_t == ControlMessage + + +def pandas_dataframe_generator(dataset_pandas: DatasetManager, count: int) -> Generator[pd.DataFrame, None, None]: + + df = dataset_pandas["csv_sample.csv"] + for _ in range(count): + yield df + + +def test_created_stage_pipe(config: Config, dataset_pandas: DatasetManager): + + config.num_threads = os.cpu_count() + + input_df = dataset_pandas["csv_sample.csv"] + + expected_df = input_df.copy() + expected_df["new_column"] = "Hello" + + df_count = 100 + df_generator = partial(pandas_dataframe_generator, dataset_pandas, df_count) + + partial_fn = partial(_process_df, column="new_column", value="Hello") + + pipe = LinearPipeline(config) + pipe.set_source(InMemoryDataGenStage(config, df_generator, output_data_type=pd.DataFrame)) + pipe.add_stage(MultiProcessingStage[pd.DataFrame, pd.DataFrame].create(c=config, + unique_name="multi-processing-stage-3", + process_fn=partial_fn, + process_pool_usage=0.1)) + sink_stage = pipe.add_stage(InMemorySinkStage(config)) + + pipe.run() + + for df in sink_stage.get_messages(): + assert df.equals(expected_df) + + def test_derived_stage_pipe(config: Config, dataset_pandas: DatasetManager): config.num_threads = os.cpu_count() - input_df = dataset_pandas["filter_probs.csv"] + input_df = dataset_pandas["csv_sample.csv"] add_column_name = "new_column" expected_df = input_df.copy() expected_df[add_column_name] = "Hello" @@ -93,7 +166,7 @@ def test_derived_stage_pipe(config: Config, dataset_pandas: DatasetManager): pipe = LinearPipeline(config) pipe.set_source(InMemorySourceStage(config, [cudf.DataFrame(input_df)])) pipe.add_stage(DeserializeStage(config, ensure_sliceable_index=True)) - pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.5, add_column_name=add_column_name)) + pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name=add_column_name)) pipe.add_stage(SerializeStage(config)) comp_stage = pipe.add_stage(CompareDataFrameStage(config, expected_df)) @@ -102,35 +175,40 @@ def test_derived_stage_pipe(config: Config, dataset_pandas: DatasetManager): assert_results(comp_stage.get_results()) -def pandas_dataframe_generator(dataset_pandas: DatasetManager, count: int) -> Generator[pd.DataFrame, None, None]: - - df = dataset_pandas["filter_probs.csv"] - - for _ in range(count): - yield df - - -@pytest.mark.use_python -def test_base_stage_with_function_pipe(config: Config, dataset_pandas: DatasetManager): - +def test_multiple_stages_pipe(config: Config, dataset_pandas: DatasetManager): config.num_threads = os.cpu_count() - input_df = dataset_pandas["filter_probs.csv"] + input_df = dataset_pandas["csv_sample.csv"] expected_df = input_df.copy() - expected_df["new_column"] = "Hello" + expected_df["new_column_1"] = "new_value_1" + expected_df["new_column_2"] = "new_value_2" - df_count = 10 + df_count = 100 df_generator = partial(pandas_dataframe_generator, dataset_pandas, df_count) + partial_fn = partial(_process_df, column="new_column_2", value="new_value_2") + + @stage + def pdf_to_control_message_stage(pdf: pd.DataFrame) -> ControlMessage: + df = cudf.DataFrame.from_pandas(pdf) + meta = MessageMeta(df) + msg = ControlMessage() + msg.payload(meta) + + return msg + pipe = LinearPipeline(config) pipe.set_source(InMemoryDataGenStage(config, df_generator, output_data_type=pd.DataFrame)) pipe.add_stage(MultiProcessingStage[pd.DataFrame, pd.DataFrame].create(c=config, - process_fn=process_fn, - process_pool_usage=0.5)) - sink_stage = pipe.add_stage(InMemorySinkStage(config)) + unique_name="multi-processing-stage-4", + process_fn=partial_fn, + process_pool_usage=0.1)) + pipe.add_stage(pdf_to_control_message_stage(config)) + pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name="new_column_1")) + pipe.add_stage(SerializeStage(config)) + comp_stage = pipe.add_stage(CompareDataFrameStage(config, expected_df)) pipe.run() - for df in sink_stage.get_messages(): - assert df.equals(expected_df) + assert_results(comp_stage.get_results()) From 5ae1fbdc584a0e05a2de710aa5dac81e96968600 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 19 Sep 2024 16:50:59 -0700 Subject: [PATCH 21/30] fix comments --- .../stages/general/multi_processing_stage.py | 64 +++++-- .../morpheus/utils/shared_process_pool.py | 170 ++++++++++-------- tests/test_multi_processing_stage.py | 19 +- tests/utils/test_shared_process_pool.py | 11 +- 4 files changed, 168 insertions(+), 96 deletions(-) diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index 520ae0359b..f0ca7a8e14 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -35,7 +35,10 @@ class MultiProcessingBaseStage(SinglePortStage, typing.Generic[InputT, OutputT]) def __init__(self, *, c: Config, process_pool_usage: float, max_in_flight_messages: int = None): super().__init__(c=c) + if not 0 <= process_pool_usage <= 1: + raise ValueError("process_pool_usage must be between 0 and 1.") self._process_pool_usage = process_pool_usage + self._shared_process_pool = SharedProcessPool() self._shared_process_pool.wait_until_ready() @@ -45,24 +48,44 @@ def __init__(self, *, c: Config, process_pool_usage: float, max_in_flight_messag else: self._max_in_flight_messages = max_in_flight_messages - @property - @abstractmethod - def name(self) -> str: + def accepted_types(self) -> typing.Tuple: """ - Marked as abstract to force the derived stage to provide a unique name. + There are two approaches to inherit from this class: + - With generic types: MultiProcessingDerivedStage(MultiProcessingBaseStage[InputT, OutputT]) + - With concrete types: MultiProcessingDerivedStage(MultiProcessingBaseStage[int, str]) + + When inheriting with generic types, the derived class can be instantiated like this: + + stage = MultiProcessingDerivedStage[int, str]() + + In this case, typing.Generic stores the stage type in stage.__orig_class__, the concrete types can be accessed + as below: + + input_type = typing.get_args(stage.__orig_class__)[0] # int + output_type = typing.get_args(stage.__orig_class__)[1] # str + + However, when instantiating a stage which inherits with concrete types: + + stage = MultiProcessingDerivedStage() + + The stage instance does not have __orig_class__ attribute (since it is not a generic type). Thus, the concrete + types need be retrieved from its base class (which is a generic type): + + input_type = typing.get_args(stage.__orig_bases__[0])[0] # int + output_type = typing.get_args(stage.__orig_bases__[0])[1] # str + + Raises: + RuntimeError: if the accepted cannot be deducted from either __orig_class__ or __orig_bases__ Returns: - str: The unique name of the stage. + typing.Tuple: accepted input types """ - return "multi-processing-base-stage" - - def accepted_types(self) -> typing.Tuple: if hasattr(self, "__orig_class__"): - # Derived with abstract types + # inherited with generic types input_type = typing.get_args(self.__orig_class__)[0] # pylint: disable=no-member elif hasattr(self, "__orig_bases__"): - # Derived with concrete types + # inherited with concrete types input_type = typing.get_args(self.__orig_bases__[0])[0] # pylint: disable=no-member else: @@ -71,12 +94,21 @@ def accepted_types(self) -> typing.Tuple: return (input_type, ) def compute_schema(self, schema: StageSchema): + """ + See the comment on `accepted_types` for more information on accessing the input and output types. + + Args: + schema (StageSchema): StageSchema + + Raises: + RuntimeError: if the output type cannot be deducted from either __orig_class__ or __orig_bases__ + """ if hasattr(self, "__orig_class__"): - # Derived with abstract types + # inherited with abstract types output_type = typing.get_args(self.__orig_class__)[1] # pylint: disable=no-member elif hasattr(self, "__orig_bases__"): - # Derived with concrete types + # inherited with concrete types output_type = typing.get_args(self.__orig_bases__[0])[1] else: @@ -110,14 +142,18 @@ def _get_func_signature(func: typing.Callable[[InputT], OutputT]) -> tuple[type, for param in signature.parameters.values(): if param.name not in bound_args: + if input_arg is not None: + raise ValueError("Found more than one unbound arguments in partial function") input_arg = param - break if input_arg is None: - raise ValueError("Could not find unbound argument in partial function") + raise ValueError("Cannot find unbound argument in partial function") input_t = input_arg.annotation else: + if len(signature.parameters) != 1: + raise ValueError("Function must have exactly one argument") + input_t = next(iter(signature.parameters.values())).annotation output_t = signature.return_annotation diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 3ca8f6ac6e..a8d586f788 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -18,9 +18,8 @@ import multiprocessing as mp import os import queue -import time +import threading from enum import Enum -from threading import Lock logger = logging.getLogger(__name__) @@ -30,8 +29,8 @@ class PoolStatus(Enum): INITIALIZING = 0 RUNNING = 1 - STOPPING = 2 - SHUTDOWN = 3 + STOPPED = 2 + SHUTDOWN = 4 class SimpleFuture: @@ -96,7 +95,7 @@ def is_cancelled(self): class SharedProcessPool: _instance = None - _lock = Lock() + _lock = threading.Lock() def __new__(cls): """ @@ -115,20 +114,25 @@ def __new__(cls): with cls._lock: if cls._instance is None: - logger.info("SharedProcessPool.__new__: Creating a new instance...") + logger.debug("SharedProcessPool.__new__: Creating a new instance...") cls._instance = super().__new__(cls) cls._instance._initialize() - logger.info("SharedProcessPool.__new__: SharedProcessPool has been initialized.") + logger.debug("SharedProcessPool.__new__: SharedProcessPool has been initialized.") else: - logger.info("SharedProcessPool.__new__: instance already exists.") + logger.debug("SharedProcessPool.__new__: instance already exists.") return cls._instance def _initialize(self): self._status = PoolStatus.INITIALIZING - self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * 0.1)) + cpu_usage = os.environ.get("SHARED_PROCESS_POOL_CPU_USAGE", None) + if cpu_usage is not None: + cpu_usage = float(cpu_usage) + else: + cpu_usage = 0.1 + self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * cpu_usage)) self._processes = [] self._context = mp.get_context("fork") @@ -139,7 +143,12 @@ def _initialize(self): self._stage_usage = {} self._cancellation_token = CancellationToken(self._manager) - self._launch_workers() + self._launch_condition = threading.Condition() + self._join_condition = threading.Condition() + + process_launcher = threading.Thread(target=self._launch_workers) + process_launcher.start() + process_launcher.join() self._status = PoolStatus.RUNNING @@ -152,6 +161,8 @@ def _launch_workers(self): logger.debug("SharedProcessPool._lanch_workers(): Process %s/%s has been started.", i + 1, self.total_max_workers) + with self._launch_condition: + self._launch_condition.notify_all() @property def total_max_workers(self): @@ -179,7 +190,7 @@ def _worker(cancellation_token, task_queues, stage_semaphores): continue try: - task = task_queue.get_nowait() + task = task_queue.get(timeout=0.1) except queue.Empty: semaphore.release() continue @@ -200,9 +211,21 @@ def _worker(cancellation_token, task_queues, stage_semaphores): except Exception as e: task.set_exception(e) + task_queue.task_done() + semaphore.release() - time.sleep(0.1) # Avoid busy-waiting + def _join_process_pool(self): + for task_queue in self._task_queues.values(): + task_queue.join() + + self._cancellation_token.cancel() + for i, p in enumerate(self._processes): + p.join() + logger.debug("Process %s/%s has been joined.", i + 1, len(self._processes)) + + with self._join_condition: + self._join_condition.notify_all() def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Task: """ @@ -299,6 +322,37 @@ def start(self): self._launch_workers() self._status = PoolStatus.RUNNING + def wait_until_ready(self, timeout=None): + """ + Wait until the SharedProcessPool is running and ready to accept tasks. + + Parameters + ---------- + timeout : _type_, optional + timeout in seconds to wait for the SharedProcessPool to be ready, by default None. + If None, it will wait indefinitely. + + Raises + ------ + RuntimeError + If the SharedProcessPool is not initializing or running. + TimeoutError + If has been waiting more than the timeout. + """ + if self._status == PoolStatus.RUNNING: + logger.debug("SharedProcessPool.wait_until_ready(): SharedProcessPool is already running.") + return + + if self._status == PoolStatus.INITIALIZING: + with self._launch_condition: + launched = self._launch_condition.wait(timeout) + if not launched: + raise TimeoutError("Time out.") + else: + raise RuntimeError("Cannot wait for a pool that is not initializing.") + + logger.debug("SharedProcessPool.wait_until_ready(): SharedProcessPool is ready.") + def reset(self): """ Clear all the previous settings and restart the SharedProcessPool. @@ -309,101 +363,67 @@ def reset(self): If the SharedProcessPool is not already shutdown. """ if self._status != PoolStatus.SHUTDOWN: - raise RuntimeError( - "SharedProcessPool.reset(): Cannot reset a SharedProcessPool that is not already shutdown.") + raise RuntimeError("Cannot reset a SharedProcessPool that is not already shutdown.") self._initialize() def stop(self): """ - Complete existing tasks and stop the SharedProcessPool. + Stop receiving any new tasks. """ if self._status not in (PoolStatus.RUNNING, PoolStatus.INITIALIZING): logger.warning("SharedProcessPool.stop(): Cannot stop a SharedProcessPool that is not running.") return # no new tasks will be accepted from this point - self._status = PoolStatus.STOPPING - - # wait for all task queues to be empty - task_queue_count = len(self._task_queues) - empty_task_queues = set() - while len(empty_task_queues) < task_queue_count: - for stage_name, task_queue in self._task_queues.items(): - if task_queue.empty(): - empty_task_queues.add(stage_name) - - self._cancellation_token.cancel() - - for i, p in enumerate(self._processes): - p.join() - logger.debug("Process %s/%s has been joined.", i + 1, self._total_max_workers) - - logger.debug("SharedProcessPool.stop(): All tasks have been completed. SharedProcessPool has been stopped.") - self._status = PoolStatus.SHUTDOWN - - def terminate(self): - """ - Terminate all processes and shutdown the SharedProcessPool immediately. - """ - for i, p in enumerate(self._processes): - p.terminate() - logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + self._status = PoolStatus.STOPPED - logger.debug("SharedProcessPool.terminate(): SharedProcessPool has been terminated.") - self._status = PoolStatus.SHUTDOWN - - def wait_until_ready(self, timeout=None): + def join(self, timeout=None): """ - Wait until the SharedProcessPool is running and ready to accept tasks. + Wait until the SharedProcessPool is terminated. Parameters ---------- timeout : _type_, optional - timeout in seconds to wait for the SharedProcessPool to be ready, by default None. + timeout in seconds to wait for the SharedProcessPool to be terminated, by default None. If None, it will wait indefinitely. Raises ------ RuntimeError - If the SharedProcessPool is not initializing or running. + If is called on a SharedProcessPool that is not stopped. + TimeoutError If has been waiting more than the timeout. """ - if self.status not in (PoolStatus.INITIALIZING, PoolStatus.RUNNING): - raise RuntimeError("Cannot wait for a SharedProcessPool that is not initializing.") - - start_time = time.time() - while self.status != PoolStatus.RUNNING: - if timeout is not None and time.time() - start_time > timeout: - raise TimeoutError("SharedProcessPool wait_until_ready has timed out.") - time.sleep(0.1) - logger.debug("SharedProcessPool.wait_until_ready(): SharedProcessPool is ready.") + if self._status != PoolStatus.STOPPED: + raise RuntimeError("Cannot join SharedProcessPool that is not stopped.") - def join(self, timeout=None): - """ - Wait until the SharedProcessPool is terminated. + process_joiner = threading.Thread(target=self._join_process_pool) + process_joiner.start() - Parameters - ---------- - timeout : _type_, optional - timeout in seconds to wait for the SharedProcessPool to be terminated, by default None. - If None, it will wait indefinitely. + with self._join_condition: + joined = self._join_condition.wait(timeout) + if not joined: + raise TimeoutError("time out.") - Raises - ------ - TimeoutError - If has been waiting more than the timeout. - """ - start_time = time.time() + process_joiner.join() - while self._status != PoolStatus.SHUTDOWN: - if timeout is not None and time.time() - start_time > timeout: - raise TimeoutError("SharedProcessPool join has timed out.") - time.sleep(0.1) + self._status = PoolStatus.SHUTDOWN logger.debug("SharedProcessPool.join(): SharedProcessPool has been joined.") + def terminate(self): + """ + Terminate all processes and shutdown the SharedProcessPool immediately. + """ + for i, p in enumerate(self._processes): + p.terminate() + logger.debug("Process %s/%s has been terminated.", i + 1, self._total_max_workers) + + logger.debug("SharedProcessPool.terminate(): SharedProcessPool has been terminated.") + self._status = PoolStatus.SHUTDOWN + def __del__(self): self.terminate() diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index f2132d6050..d24028b96d 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -21,6 +21,7 @@ import pandas as pd import cudf +import pytest from _utils import assert_results from _utils.dataset_manager import DatasetManager @@ -60,7 +61,7 @@ def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetMana assert input_t == int assert output_t == pd.DataFrame - # Test create() with partial function + # Test create() with partial function with 1 unbound argument df = dataset_pandas["csv_sample.csv"] partial_fn = partial(_process_df, df=df, value="new_value") @@ -75,6 +76,22 @@ def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetMana assert input_t == str assert output_t == pd.DataFrame + # Invalid case: create() with partial function with 0 unbound argument + invalid_partial_fn = partial(_process_df, df=df, column="new_column", value="new_value") + with pytest.raises(ValueError): + MultiProcessingStage.create(c=config, + unique_name="multi-processing-stage-3", + process_fn=invalid_partial_fn, + process_pool_usage=0.1) + + # Invalid case: create() with function with more than 1 arguments + invalid_partial_fn = partial(_process_df, df=df) + with pytest.raises(ValueError): + MultiProcessingStage.create(c=config, + unique_name="multi-processing-stage-4", + process_fn=invalid_partial_fn, + process_pool_usage=0.1) + class DerivedMultiProcessingStage(MultiProcessingBaseStage[ControlMessage, ControlMessage]): diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 1d0865b550..d7e777311c 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -26,13 +26,15 @@ logger = logging.getLogger(__name__) -# This unit test does NOT work well with the `-s` option of pytest. Run pytest without `-s` flag. +# This test has issues with joining processes when testing with pytest `-s` option. Run pytest without `-s` flag. @pytest.fixture(name="shared_process_pool") def shared_process_pool_fixture(): pool = SharedProcessPool() + pool.start() + pool.wait_until_ready() yield pool # Stop and reset the pool after each test @@ -80,7 +82,7 @@ def test_singleton(): def test_pool_status(shared_process_pool): pool = shared_process_pool - pool.wait_until_ready() + # pool.wait_until_ready() assert pool.status == PoolStatus.RUNNING pool.set_usage("test_stage", 0.5) @@ -97,7 +99,6 @@ def test_pool_status(shared_process_pool): _check_pool_stage_settings(pool, "test_stage", 0.5) pool.terminate() - pool.join() assert pool.status == PoolStatus.SHUTDOWN # With pool.reset(), the pool should reset all the status @@ -137,9 +138,8 @@ def test_submit_single_task(shared_process_pool, a, b, expected): assert task.result() == expected pool.stop() - pool.join() - # After the pool is shutdown, it should not accept any new tasks + # After the pool is stopped, it should not accept any new tasks with pytest.raises(RuntimeError): pool.submit_task("test_stage", _add_task, 10, 20) @@ -288,4 +288,3 @@ def test_terminate_running_tasks(shared_process_pool): # The pool should be shutdown immediately after calling terminate() without waiting for the tasks to complete pool.terminate() - pool.join() From 546efdf2b0aa102792655fb7888859a15c2da90c Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 19 Sep 2024 17:25:13 -0700 Subject: [PATCH 22/30] Fix pool status --- .../morpheus/utils/shared_process_pool.py | 29 ++++++++++--------- tests/utils/test_shared_process_pool.py | 13 +++------ 2 files changed, 20 insertions(+), 22 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index a8d586f788..acb2959580 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -30,7 +30,7 @@ class PoolStatus(Enum): INITIALIZING = 0 RUNNING = 1 STOPPED = 2 - SHUTDOWN = 4 + SHUTDOWN = 3 class SimpleFuture: @@ -146,11 +146,8 @@ def _initialize(self): self._launch_condition = threading.Condition() self._join_condition = threading.Condition() - process_launcher = threading.Thread(target=self._launch_workers) - process_launcher.start() - process_launcher.join() + self.start() - self._status = PoolStatus.RUNNING def _launch_workers(self): for i in range(self.total_max_workers): @@ -163,6 +160,7 @@ def _launch_workers(self): self.total_max_workers) with self._launch_condition: self._launch_condition.notify_all() + self._status = PoolStatus.RUNNING @property def total_max_workers(self): @@ -315,12 +313,13 @@ def start(self): RuntimeError If the SharedProcessPool is not shutdown. """ - if self._status != PoolStatus.SHUTDOWN: - logger.warning("SharedProcessPool.start(): Cannot start a SharedProcessPool that is not shutdown.") + if self._status == PoolStatus.RUNNING: + logger.warning("SharedProcessPool.start(): process pool is already running.") return - self._launch_workers() - self._status = PoolStatus.RUNNING + process_launcher = threading.Thread(target=self._launch_workers) + process_launcher.start() + process_launcher.join() def wait_until_ready(self, timeout=None): """ @@ -349,7 +348,7 @@ def wait_until_ready(self, timeout=None): if not launched: raise TimeoutError("Time out.") else: - raise RuntimeError("Cannot wait for a pool that is not initializing.") + raise RuntimeError("Cannot wait for a pool that is not initializing or running.") logger.debug("SharedProcessPool.wait_until_ready(): SharedProcessPool is ready.") @@ -360,10 +359,10 @@ def reset(self): Raises ------ RuntimeError - If the SharedProcessPool is not already shutdown. + If the SharedProcessPool is not already shut down. """ if self._status != PoolStatus.SHUTDOWN: - raise RuntimeError("Cannot reset a SharedProcessPool that is not already shutdown.") + raise RuntimeError("Cannot reset a SharedProcessPool that is not already shut down.") self._initialize() @@ -375,7 +374,7 @@ def stop(self): logger.warning("SharedProcessPool.stop(): Cannot stop a SharedProcessPool that is not running.") return - # no new tasks will be accepted from this point + # No new tasks will be accepted from this point self._status = PoolStatus.STOPPED def join(self, timeout=None): @@ -398,6 +397,10 @@ def join(self, timeout=None): """ if self._status != PoolStatus.STOPPED: + if self._status == PoolStatus.SHUTDOWN: + logging.warning("SharedProcessPool.join(): process pool is already shut down.") + return + raise RuntimeError("Cannot join SharedProcessPool that is not stopped.") process_joiner = threading.Thread(target=self._join_process_pool) diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index d7e777311c..cfea249adb 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -37,9 +37,8 @@ def shared_process_pool_fixture(): pool.wait_until_ready() yield pool - # Stop and reset the pool after each test - pool.stop() - pool.join() + # Terminate and reset the pool after each test + pool.terminate() pool.reset() @@ -82,7 +81,6 @@ def test_singleton(): def test_pool_status(shared_process_pool): pool = shared_process_pool - # pool.wait_until_ready() assert pool.status == PoolStatus.RUNNING pool.set_usage("test_stage", 0.5) @@ -91,7 +89,7 @@ def test_pool_status(shared_process_pool): pool.join() assert pool.status == PoolStatus.SHUTDOWN - # With pool.start(), the pool should have the same status as before stopping + # After pool.start(), the pool should have the same status as before stopping pool.start() pool.wait_until_ready() assert pool.status == PoolStatus.RUNNING @@ -101,7 +99,7 @@ def test_pool_status(shared_process_pool): pool.terminate() assert pool.status == PoolStatus.SHUTDOWN - # With pool.reset(), the pool should reset all the status + # After pool.reset(), the pool should reset all the status pool.reset() pool.wait_until_ready() assert pool.status == PoolStatus.RUNNING @@ -125,7 +123,6 @@ def test_pool_status(shared_process_pool): def test_submit_single_task(shared_process_pool, a, b, expected): pool = shared_process_pool - pool.wait_until_ready() pool.set_usage("test_stage", 0.5) task = pool.submit_task("test_stage", _add_task, a, b) @@ -147,7 +144,6 @@ def test_submit_single_task(shared_process_pool, a, b, expected): def test_submit_task_with_invalid_stage(shared_process_pool): pool = shared_process_pool - pool.wait_until_ready() with pytest.raises(ValueError): pool.submit_task("stage_does_not_exist", _add_task, 10, 20) @@ -211,7 +207,6 @@ def test_submit_multiple_tasks(shared_process_pool, a, b, expected): def test_set_usage(shared_process_pool): pool = shared_process_pool - pool.wait_until_ready() pool.set_usage("test_stage_1", 0.5) assert pool._total_usage == 0.5 From 506c0cf1960a7b4ce4d30610b0c3252e7ace46bf Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 19 Sep 2024 17:35:40 -0700 Subject: [PATCH 23/30] fix test --- tests/utils/test_shared_process_pool.py | 27 +++++++++++++++++++------ 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index cfea249adb..3ce03d452b 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -20,7 +20,7 @@ from fractions import Fraction import pytest - +import multiprocessing as mp from morpheus.utils.shared_process_pool import PoolStatus from morpheus.utils.shared_process_pool import SharedProcessPool @@ -51,6 +51,10 @@ def _time_consuming_task(sleep_time): return sleep_time +def _blocked_until_signaled_task(q: mp.Queue): + return q.get() + + def _function_raises_exception(): raise RuntimeError("Exception is raised in the process.") @@ -247,16 +251,27 @@ def test_task_completion_with_early_stop(shared_process_pool): pool.set_usage("test_stage_2", 0.3) pool.set_usage("test_stage_3", 0.5) + manager = mp.Manager() + queue = manager.Queue() + tasks = [] - task_num = 5 - sleep_time = 10 + task_num = 10 + for _ in range(task_num): - tasks.append(pool.submit_task("test_stage_1", _time_consuming_task, sleep_time)) - tasks.append(pool.submit_task("test_stage_2", _time_consuming_task, sleep_time)) - tasks.append(pool.submit_task("test_stage_3", _time_consuming_task, sleep_time)) + tasks.append(pool.submit_task("test_stage_1", _blocked_until_signaled_task, queue)) + tasks.append(pool.submit_task("test_stage_2", _blocked_until_signaled_task, queue)) + tasks.append(pool.submit_task("test_stage_3", _blocked_until_signaled_task, queue)) pool.stop() + + # No tasks have been completed since they have not been signaled yet + for task in tasks: + assert not task.done() + + for i in range(len(tasks)): + queue.put(i) + pool.join() # all tasks should be completed before the pool is shutdown From 4c97e74deb22ee12f6c476b5d3df6e50d571b8f3 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Thu, 19 Sep 2024 17:53:06 -0700 Subject: [PATCH 24/30] fix format --- .../stages/general/multi_processing_stage.py | 2 +- tests/test_multi_processing_stage.py | 41 ++++++++++--------- 2 files changed, 22 insertions(+), 21 deletions(-) diff --git a/python/morpheus/morpheus/stages/general/multi_processing_stage.py b/python/morpheus/morpheus/stages/general/multi_processing_stage.py index f0ca7a8e14..f96939841a 100644 --- a/python/morpheus/morpheus/stages/general/multi_processing_stage.py +++ b/python/morpheus/morpheus/stages/general/multi_processing_stage.py @@ -153,7 +153,7 @@ def _get_func_signature(func: typing.Callable[[InputT], OutputT]) -> tuple[type, else: if len(signature.parameters) != 1: raise ValueError("Function must have exactly one argument") - + input_t = next(iter(signature.parameters.values())).annotation output_t = signature.return_annotation diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index d24028b96d..1bdfa06999 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -52,12 +52,12 @@ def _process_df(df: pd.DataFrame, column: str, value: str) -> pd.DataFrame: def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetManager): # Test create() with normal function - stage = MultiProcessingStage.create(c=config, + mp_stage = MultiProcessingStage.create(c=config, unique_name="multi-processing-stage-1", process_fn=_create_df, process_pool_usage=0.1) - assert stage.name == "multi-processing-stage-1" - input_t, output_t = typing.get_args(stage.__orig_class__) # pylint: disable=no-member + assert mp_stage.name == "multi-processing-stage-1" + input_t, output_t = typing.get_args(mp_stage.__orig_class__) # pylint: disable=no-member assert input_t == int assert output_t == pd.DataFrame @@ -65,14 +65,14 @@ def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetMana df = dataset_pandas["csv_sample.csv"] partial_fn = partial(_process_df, df=df, value="new_value") - stage = MultiProcessingStage.create(c=config, + mp_stage = MultiProcessingStage.create(c=config, unique_name="multi-processing-stage-2", process_fn=partial_fn, process_pool_usage=0.1) - assert stage.name == "multi-processing-stage-2" - input_t, output_t = typing.get_args(stage.__orig_class__) # pylint: disable=no-member - assert stage.accepted_types() == (str, ) + assert mp_stage.name == "multi-processing-stage-2" + input_t, output_t = typing.get_args(mp_stage.__orig_class__) # pylint: disable=no-member + assert mp_stage.accepted_types() == (str, ) assert input_t == str assert output_t == pd.DataFrame @@ -127,11 +127,11 @@ def _on_data(self, data: ControlMessage) -> ControlMessage: def test_derived_stage_type_deduction(config: Config): - stage = DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name="new_column") - assert stage.name == "derived-multi-processing-stage" - assert stage.accepted_types() == (ControlMessage, ) + mp_stage = DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name="new_column") + assert mp_stage.name == "derived-multi-processing-stage" + assert mp_stage.accepted_types() == (ControlMessage, ) - input_t, output_t = typing.get_args(stage.__orig_bases__[0]) # pylint: disable=no-member + input_t, output_t = typing.get_args(mp_stage.__orig_bases__[0]) # pylint: disable=no-member assert input_t == ControlMessage assert output_t == ControlMessage @@ -160,7 +160,7 @@ def test_created_stage_pipe(config: Config, dataset_pandas: DatasetManager): pipe = LinearPipeline(config) pipe.set_source(InMemoryDataGenStage(config, df_generator, output_data_type=pd.DataFrame)) pipe.add_stage(MultiProcessingStage[pd.DataFrame, pd.DataFrame].create(c=config, - unique_name="multi-processing-stage-3", + unique_name="multi-processing-stage-5", process_fn=partial_fn, process_pool_usage=0.1)) sink_stage = pipe.add_stage(InMemorySinkStage(config)) @@ -198,13 +198,13 @@ def test_multiple_stages_pipe(config: Config, dataset_pandas: DatasetManager): input_df = dataset_pandas["csv_sample.csv"] expected_df = input_df.copy() - expected_df["new_column_1"] = "new_value_1" - expected_df["new_column_2"] = "new_value_2" + expected_df["new_column_1"] = "new_value" + expected_df["new_column_2"] = "Hello" df_count = 100 df_generator = partial(pandas_dataframe_generator, dataset_pandas, df_count) - partial_fn = partial(_process_df, column="new_column_2", value="new_value_2") + partial_fn = partial(_process_df, column="new_column_1", value="new_value") @stage def pdf_to_control_message_stage(pdf: pd.DataFrame) -> ControlMessage: @@ -217,12 +217,13 @@ def pdf_to_control_message_stage(pdf: pd.DataFrame) -> ControlMessage: pipe = LinearPipeline(config) pipe.set_source(InMemoryDataGenStage(config, df_generator, output_data_type=pd.DataFrame)) - pipe.add_stage(MultiProcessingStage[pd.DataFrame, pd.DataFrame].create(c=config, - unique_name="multi-processing-stage-4", - process_fn=partial_fn, - process_pool_usage=0.1)) + pipe.add_stage( + MultiProcessingStage.create(c=config, + unique_name="multi-processing-stage-6", + process_fn=partial_fn, + process_pool_usage=0.1)) pipe.add_stage(pdf_to_control_message_stage(config)) - pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name="new_column_1")) + pipe.add_stage(DerivedMultiProcessingStage(c=config, process_pool_usage=0.1, add_column_name="new_column_2")) pipe.add_stage(SerializeStage(config)) comp_stage = pipe.add_stage(CompareDataFrameStage(config, expected_df)) From ee8633ec9375f970042f23377ebf90e080c7db98 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Fri, 20 Sep 2024 10:18:42 -0700 Subject: [PATCH 25/30] fix CI --- .../morpheus/utils/shared_process_pool.py | 5 +---- tests/test_multi_processing_stage.py | 14 +++++++------- tests/utils/test_shared_process_pool.py | 17 +++++++++++++++-- 3 files changed, 23 insertions(+), 13 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index acb2959580..8e3addcb40 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -23,8 +23,6 @@ logger = logging.getLogger(__name__) -logger.setLevel(logging.DEBUG) - class PoolStatus(Enum): INITIALIZING = 0 @@ -131,7 +129,7 @@ def _initialize(self): if cpu_usage is not None: cpu_usage = float(cpu_usage) else: - cpu_usage = 0.1 + cpu_usage = 0.5 self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * cpu_usage)) self._processes = [] @@ -148,7 +146,6 @@ def _initialize(self): self.start() - def _launch_workers(self): for i in range(self.total_max_workers): process = self._context.Process(target=self._worker, diff --git a/tests/test_multi_processing_stage.py b/tests/test_multi_processing_stage.py index 1bdfa06999..d83e9b8d3d 100644 --- a/tests/test_multi_processing_stage.py +++ b/tests/test_multi_processing_stage.py @@ -19,9 +19,9 @@ from typing import Generator import pandas as pd +import pytest import cudf -import pytest from _utils import assert_results from _utils.dataset_manager import DatasetManager @@ -53,9 +53,9 @@ def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetMana # Test create() with normal function mp_stage = MultiProcessingStage.create(c=config, - unique_name="multi-processing-stage-1", - process_fn=_create_df, - process_pool_usage=0.1) + unique_name="multi-processing-stage-1", + process_fn=_create_df, + process_pool_usage=0.1) assert mp_stage.name == "multi-processing-stage-1" input_t, output_t = typing.get_args(mp_stage.__orig_class__) # pylint: disable=no-member assert input_t == int @@ -66,9 +66,9 @@ def test_create_stage_type_deduction(config: Config, dataset_pandas: DatasetMana partial_fn = partial(_process_df, df=df, value="new_value") mp_stage = MultiProcessingStage.create(c=config, - unique_name="multi-processing-stage-2", - process_fn=partial_fn, - process_pool_usage=0.1) + unique_name="multi-processing-stage-2", + process_fn=partial_fn, + process_pool_usage=0.1) assert mp_stage.name == "multi-processing-stage-2" input_t, output_t = typing.get_args(mp_stage.__orig_class__) # pylint: disable=no-member diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 3ce03d452b..c89f3015c7 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -14,13 +14,14 @@ # limitations under the License. import logging +import multiprocessing as mp import threading import time from decimal import Decimal from fractions import Fraction import pytest -import multiprocessing as mp + from morpheus.utils.shared_process_pool import PoolStatus from morpheus.utils.shared_process_pool import SharedProcessPool @@ -29,11 +30,23 @@ # This test has issues with joining processes when testing with pytest `-s` option. Run pytest without `-s` flag. +@pytest.fixture(scope="session", autouse=True) +def setup_and_teardown(): + + pool = SharedProcessPool() + # Since SharedProcessPool might be used in other tests, terminate and reset the pool before the test starts + pool.terminate() + pool.reset() + yield + + # Terminate the pool after all tests are done + pool.terminate() + + @pytest.fixture(name="shared_process_pool") def shared_process_pool_fixture(): pool = SharedProcessPool() - pool.start() pool.wait_until_ready() yield pool From 63675d7cbd85a2ae95116268fb3a2d2b4e332cbd Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Fri, 20 Sep 2024 15:13:24 -0700 Subject: [PATCH 26/30] fix comments and CI --- .../morpheus/utils/shared_process_pool.py | 15 ++++++----- tests/utils/test_shared_process_pool.py | 27 ++++++++++--------- 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 8e3addcb40..ad58fdf3f8 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -20,6 +20,7 @@ import queue import threading from enum import Enum +from typing import Callable logger = logging.getLogger(__name__) @@ -33,7 +34,7 @@ class PoolStatus(Enum): class SimpleFuture: - def __init__(self, manager): + def __init__(self, manager: mp.managers.SyncManager): self._result = manager.Value("i", None) self._exception = manager.Value("i", None) self._done = manager.Event() @@ -58,7 +59,7 @@ def done(self): class Task(SimpleFuture): - def __init__(self, manager, process_fn, args, kwargs): + def __init__(self, manager: mp.managers.SyncManager, process_fn: Callable, args, kwargs): super().__init__(manager) self._process_fn = process_fn self._args = args @@ -79,7 +80,7 @@ def kwargs(self): class CancellationToken: - def __init__(self, manager): + def __init__(self, manager: mp.managers.SyncManager): self._cancel_event = manager.Event() def cancel(self): @@ -222,7 +223,7 @@ def _join_process_pool(self): with self._join_condition: self._join_condition.notify_all() - def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Task: + def submit_task(self, stage_name: str, process_fn: Callable, *args, **kwargs) -> Task: """ Submit a task to the SharedProcessPool. @@ -260,7 +261,7 @@ def submit_task(self, stage_name, process_fn, *args, **kwargs) -> Task: return task - def set_usage(self, stage_name, percentage): + def set_usage(self, stage_name: str, percentage: float): """ Set the usage of the SharedProcessPool for a specific stage. @@ -318,7 +319,7 @@ def start(self): process_launcher.start() process_launcher.join() - def wait_until_ready(self, timeout=None): + def wait_until_ready(self, timeout: float | None = None): """ Wait until the SharedProcessPool is running and ready to accept tasks. @@ -374,7 +375,7 @@ def stop(self): # No new tasks will be accepted from this point self._status = PoolStatus.STOPPED - def join(self, timeout=None): + def join(self, timeout: float | None = None): """ Wait until the SharedProcessPool is terminated. diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index c89f3015c7..f3b1ed6362 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -16,7 +16,6 @@ import logging import multiprocessing as mp import threading -import time from decimal import Decimal from fractions import Fraction @@ -27,13 +26,14 @@ logger = logging.getLogger(__name__) -# This test has issues with joining processes when testing with pytest `-s` option. Run pytest without `-s` flag. +# This test has issues with joining processes when testing with pytest `-s` option. Run pytest without `-s` flag @pytest.fixture(scope="session", autouse=True) def setup_and_teardown(): pool = SharedProcessPool() + # Since SharedProcessPool might be used in other tests, terminate and reset the pool before the test starts pool.terminate() pool.reset() @@ -59,11 +59,6 @@ def _add_task(x, y): return x + y -def _time_consuming_task(sleep_time): - time.sleep(sleep_time) - return sleep_time - - def _blocked_until_signaled_task(q: mp.Queue): return q.get() @@ -300,14 +295,20 @@ def test_terminate_running_tasks(shared_process_pool): pool.set_usage("test_stage_2", 0.3) pool.set_usage("test_stage_3", 0.5) + manager = mp.Manager() + queue = manager.Queue() + tasks = [] - task_num = 5 - sleep_time = 100000 + task_num = 10 + for _ in range(task_num): - tasks.append(pool.submit_task("test_stage_1", _time_consuming_task, sleep_time)) - tasks.append(pool.submit_task("test_stage_2", _time_consuming_task, sleep_time)) - tasks.append(pool.submit_task("test_stage_3", _time_consuming_task, sleep_time)) + tasks.append(pool.submit_task("test_stage_1", _blocked_until_signaled_task, queue)) + tasks.append(pool.submit_task("test_stage_2", _blocked_until_signaled_task, queue)) + tasks.append(pool.submit_task("test_stage_3", _blocked_until_signaled_task, queue)) - # The pool should be shutdown immediately after calling terminate() without waiting for the tasks to complete pool.terminate() + + # No tasks have been completed since they have not been signaled yet + for task in tasks: + assert not task.done() From c37aa39ee196c41d2256ba0df40cc6017fb9428b Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Fri, 20 Sep 2024 15:44:04 -0700 Subject: [PATCH 27/30] fix type annotation --- python/morpheus/morpheus/utils/shared_process_pool.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index ad58fdf3f8..618c335fa7 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -34,7 +34,7 @@ class PoolStatus(Enum): class SimpleFuture: - def __init__(self, manager: mp.managers.SyncManager): + def __init__(self, manager): self._result = manager.Value("i", None) self._exception = manager.Value("i", None) self._done = manager.Event() @@ -59,7 +59,7 @@ def done(self): class Task(SimpleFuture): - def __init__(self, manager: mp.managers.SyncManager, process_fn: Callable, args, kwargs): + def __init__(self, manager, process_fn: Callable, args, kwargs): super().__init__(manager) self._process_fn = process_fn self._args = args @@ -80,7 +80,7 @@ def kwargs(self): class CancellationToken: - def __init__(self, manager: mp.managers.SyncManager): + def __init__(self, manager): self._cancel_event = manager.Event() def cancel(self): From 889623059064afe0a219a61e2330c69def7a06dc Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Wed, 25 Sep 2024 15:43:58 -0700 Subject: [PATCH 28/30] fix comments --- .../morpheus/morpheus/utils/shared_process_pool.py | 4 +++- tests/utils/test_shared_process_pool.py | 14 +++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index 618c335fa7..f25045c153 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -126,9 +126,11 @@ def __new__(cls): def _initialize(self): self._status = PoolStatus.INITIALIZING - cpu_usage = os.environ.get("SHARED_PROCESS_POOL_CPU_USAGE", None) + cpu_usage = os.environ.get("MORPHEUS_SHARED_PROCESS_POOL_CPU_USAGE", None) if cpu_usage is not None: cpu_usage = float(cpu_usage) + if not 0 <= cpu_usage <= 1: + raise ValueError("CPU usage must be between 0 and 1.") else: cpu_usage = 0.5 self._total_max_workers = math.floor(max(1, len(os.sched_getaffinity(0)) * cpu_usage)) diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index f3b1ed6362..694e7258ee 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -36,11 +36,13 @@ def setup_and_teardown(): # Since SharedProcessPool might be used in other tests, terminate and reset the pool before the test starts pool.terminate() + pool.join() pool.reset() yield # Terminate the pool after all tests are done pool.terminate() + pool.join() @pytest.fixture(name="shared_process_pool") @@ -50,8 +52,9 @@ def shared_process_pool_fixture(): pool.wait_until_ready() yield pool - # Terminate and reset the pool after each test - pool.terminate() + # Stop and reset the pool after each test + pool.stop() + pool.join() pool.reset() @@ -109,6 +112,7 @@ def test_pool_status(shared_process_pool): _check_pool_stage_settings(pool, "test_stage", 0.5) pool.terminate() + pool.join() assert pool.status == PoolStatus.SHUTDOWN # After pool.reset(), the pool should reset all the status @@ -308,7 +312,7 @@ def test_terminate_running_tasks(shared_process_pool): tasks.append(pool.submit_task("test_stage_3", _blocked_until_signaled_task, queue)) pool.terminate() + pool.join() - # No tasks have been completed since they have not been signaled yet - for task in tasks: - assert not task.done() + # As pool.terminate() is called, at least some of the tasks are not finished + assert any(not task.done() for task in tasks) From 790701e8e54a320de0cd9386b12dc26160d98bb4 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Wed, 25 Sep 2024 15:48:41 -0700 Subject: [PATCH 29/30] fix test --- tests/utils/test_shared_process_pool.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/utils/test_shared_process_pool.py b/tests/utils/test_shared_process_pool.py index 694e7258ee..e1d605f4bb 100644 --- a/tests/utils/test_shared_process_pool.py +++ b/tests/utils/test_shared_process_pool.py @@ -304,13 +304,16 @@ def test_terminate_running_tasks(shared_process_pool): tasks = [] - task_num = 10 + task_num = 50 for _ in range(task_num): tasks.append(pool.submit_task("test_stage_1", _blocked_until_signaled_task, queue)) tasks.append(pool.submit_task("test_stage_2", _blocked_until_signaled_task, queue)) tasks.append(pool.submit_task("test_stage_3", _blocked_until_signaled_task, queue)) + for i in range(len(tasks)): + queue.put(i) + pool.terminate() pool.join() From 5e21087222b597dbdc0220c2bba6c0319e2e7b00 Mon Sep 17 00:00:00 2001 From: Yuchen Zhang <134643420+yczhang-nv@users.noreply.github.com> Date: Wed, 25 Sep 2024 15:59:32 -0700 Subject: [PATCH 30/30] fix format --- python/morpheus/morpheus/utils/shared_process_pool.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/morpheus/morpheus/utils/shared_process_pool.py b/python/morpheus/morpheus/utils/shared_process_pool.py index f25045c153..1abfe06d30 100644 --- a/python/morpheus/morpheus/utils/shared_process_pool.py +++ b/python/morpheus/morpheus/utils/shared_process_pool.py @@ -158,8 +158,10 @@ def _launch_workers(self): logger.debug("SharedProcessPool._lanch_workers(): Process %s/%s has been started.", i + 1, self.total_max_workers) + with self._launch_condition: self._launch_condition.notify_all() + self._status = PoolStatus.RUNNING @property