From 150ff22e8d703b95188da651c6f624c79dfb43e9 Mon Sep 17 00:00:00 2001 From: Peter Andreas Entschev Date: Tue, 28 Mar 2023 12:59:05 -0700 Subject: [PATCH] Add argument to enable RMM alloaction tracking in benchmarks --- dask_cuda/benchmarks/common.py | 1 + dask_cuda/benchmarks/utils.py | 20 ++++++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/dask_cuda/benchmarks/common.py b/dask_cuda/benchmarks/common.py index c7e0cb833..1335334ab 100644 --- a/dask_cuda/benchmarks/common.py +++ b/dask_cuda/benchmarks/common.py @@ -126,6 +126,7 @@ def run(client: Client, args: Namespace, config: Config): args.rmm_release_threshold, args.rmm_log_directory, args.enable_rmm_statistics, + args.enable_rmm_track_allocations, ) address_to_index, results, message_data = gather_bench_results(client, args, config) p2p_bw = peer_to_peer_bandwidths(message_data, address_to_index) diff --git a/dask_cuda/benchmarks/utils.py b/dask_cuda/benchmarks/utils.py index 8d1cad039..32d882be0 100644 --- a/dask_cuda/benchmarks/utils.py +++ b/dask_cuda/benchmarks/utils.py @@ -131,6 +131,17 @@ def parse_benchmark_args(description="Generic dask-cuda Benchmark", args_list=[] "This enables spilling implementations such as JIT-Unspill to provides more " "information on out-of-memory errors", ) + cluster_args.add_argument( + "--enable-rmm-track-allocations", + action="store_true", + help="When enabled, wraps the memory resource used by each worker with a " + "``rmm.mr.TrackingResourceAdaptor``, which tracks the amount of memory " + "allocated." + "NOTE: This option enables additional diagnostics to be collected and " + "reported by the Dask dashboard. However, there is significant overhead " + "associated with this and it should only be used for debugging and memory " + "profiling.", + ) cluster_args.add_argument( "--enable-tcp-over-ucx", default=None, @@ -339,6 +350,7 @@ def get_cluster_options(args): "CUDA_VISIBLE_DEVICES": args.devs, "interface": args.interface, "device_memory_limit": args.device_memory_limit, + "dashboard_address": 18787, **ucx_options, } if args.no_silence_logs: @@ -370,6 +382,7 @@ def setup_memory_pool( release_threshold=None, log_directory=None, statistics=False, + rmm_track_allocations=False, ): import cupy @@ -399,6 +412,10 @@ def setup_memory_pool( rmm.mr.set_current_device_resource( rmm.mr.StatisticsResourceAdaptor(rmm.mr.get_current_device_resource()) ) + if rmm_track_allocations: + rmm.mr.set_current_device_resource( + rmm.mr.TrackingResourceAdaptor(rmm.mr.get_current_device_resource()) + ) def setup_memory_pools( @@ -411,6 +428,7 @@ def setup_memory_pools( release_threshold, log_directory, statistics, + rmm_track_allocations, ): if not is_gpu: return @@ -423,6 +441,7 @@ def setup_memory_pools( release_threshold=release_threshold, log_directory=log_directory, statistics=statistics, + rmm_track_allocations=rmm_track_allocations, ) # Create an RMM pool on the scheduler due to occasional deserialization # of CUDA objects. May cause issues with InfiniBand otherwise. @@ -435,6 +454,7 @@ def setup_memory_pools( release_threshold=release_threshold, log_directory=log_directory, statistics=statistics, + rmm_track_allocations=rmm_track_allocations, )