├── .gitignore ├── .travis.yml ├── CODE_OF_CONDUCT.md ├── LICENSE ├── README.md ├── dask_memusage.py ├── pyproject.toml └── test_memusage.py /.gitignore: -------------------------------------------------------------------------------- 1 | # Byte-compiled / optimized / DLL files 2 | __pycache__/ 3 | *.py[cod] 4 | *$py.class 5 | 6 | # C extensions 7 | *.so 8 | 9 | # Distribution / packaging 10 | .Python 11 | build/ 12 | develop-eggs/ 13 | dist/ 14 | downloads/ 15 | eggs/ 16 | .eggs/ 17 | lib/ 18 | lib64/ 19 | parts/ 20 | sdist/ 21 | var/ 22 | wheels/ 23 | *.egg-info/ 24 | .installed.cfg 25 | *.egg 26 | MANIFEST 27 | 28 | # PyInstaller 29 | # Usually these files are written by a python script from a template 30 | # before PyInstaller builds the exe, so as to inject date/other infos into it. 31 | *.manifest 32 | *.spec 33 | 34 | # Installer logs 35 | pip-log.txt 36 | pip-delete-this-directory.txt 37 | 38 | # Unit test / coverage reports 39 | htmlcov/ 40 | .tox/ 41 | .coverage 42 | .coverage.* 43 | .cache 44 | nosetests.xml 45 | coverage.xml 46 | *.cover 47 | .hypothesis/ 48 | .pytest_cache/ 49 | 50 | # Translations 51 | *.mo 52 | *.pot 53 | 54 | # Django stuff: 55 | *.log 56 | local_settings.py 57 | db.sqlite3 58 | 59 | # Flask stuff: 60 | instance/ 61 | .webassets-cache 62 | 63 | # Scrapy stuff: 64 | .scrapy 65 | 66 | # Sphinx documentation 67 | docs/_build/ 68 | 69 | # PyBuilder 70 | target/ 71 | 72 | # Jupyter Notebook 73 | .ipynb_checkpoints 74 | 75 | # pyenv 76 | .python-version 77 | 78 | # celery beat schedule file 79 | celerybeat-schedule 80 | 81 | # SageMath parsed files 82 | *.sage.py 83 | 84 | # Environments 85 | .env 86 | .venv 87 | env/ 88 | venv/ 89 | ENV/ 90 | env.bak/ 91 | venv.bak/ 92 | 93 | # Spyder project settings 94 | .spyderproject 95 | .spyproject 96 | 97 | # Rope project settings 98 | .ropeproject 99 | 100 | # mkdocs documentation 101 | /site 102 | 103 | # mypy 104 | .mypy_cache/ 105 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: python 2 | python: 3 | - "3.6" 4 | - "3.7" 5 | - "3.8" 6 | 7 | install: 8 | - pip install flit 9 | - flit install 10 | 11 | script: 12 | - flake8 dask_memusage.py 13 | - pytest 14 | -------------------------------------------------------------------------------- /CODE_OF_CONDUCT.md: -------------------------------------------------------------------------------- 1 | # Contributor Covenant Code of Conduct 2 | 3 | ## Our Pledge 4 | 5 | In the interest of fostering an open and welcoming environment, we as contributors and maintainers pledge to making participation in our project and our community a harassment-free experience for everyone, regardless of age, body size, disability, ethnicity, gender identity and expression, level of experience, nationality, personal appearance, race, religion, or sexual identity and orientation. 6 | 7 | ## Our Standards 8 | 9 | Examples of behavior that contributes to creating a positive environment include: 10 | 11 | * Using welcoming and inclusive language 12 | * Being respectful of differing viewpoints and experiences 13 | * Gracefully accepting constructive criticism 14 | * Focusing on what is best for the community 15 | * Showing empathy towards other community members 16 | 17 | Examples of unacceptable behavior by participants include: 18 | 19 | * The use of sexualized language or imagery and unwelcome sexual attention or advances 20 | * Trolling, insulting/derogatory comments, and personal or political attacks 21 | * Public or private harassment 22 | * Publishing others' private information, such as a physical or electronic address, without explicit permission 23 | * Other conduct which could reasonably be considered inappropriate in a professional setting 24 | 25 | ## Our Responsibilities 26 | 27 | Project maintainers are responsible for clarifying the standards of acceptable behavior and are expected to take appropriate and fair corrective action in response to any instances of unacceptable behavior. 28 | 29 | Project maintainers have the right and responsibility to remove, edit, or reject comments, commits, code, wiki edits, issues, and other contributions that are not aligned to this Code of Conduct, or to ban temporarily or permanently any contributor for other behaviors that they deem inappropriate, threatening, offensive, or harmful. 30 | 31 | ## Scope 32 | 33 | This Code of Conduct applies both within project spaces and in public spaces when an individual is representing the project or its community. Examples of representing a project or community include using an official project e-mail address, posting via an official social media account, or acting as an appointed representative at an online or offline event. Representation of a project may be further defined and clarified by project maintainers. 34 | 35 | ## Enforcement 36 | 37 | Instances of abusive, harassing, or otherwise unacceptable behavior may be reported by contacting the project team at itamar@itamarst.org. The project team will review and investigate all complaints, and will respond in a way that it deems appropriate to the circumstances. The project team is obligated to maintain confidentiality with regard to the reporter of an incident. Further details of specific enforcement policies may be posted separately. 38 | 39 | Project maintainers who do not follow or enforce the Code of Conduct in good faith may face temporary or permanent repercussions as determined by other members of the project's leadership. 40 | 41 | ## Attribution 42 | 43 | This Code of Conduct is adapted from the [Contributor Covenant][homepage], version 1.4, available at [http://contributor-covenant.org/version/1/4][version] 44 | 45 | [homepage]: http://contributor-covenant.org 46 | [version]: http://contributor-covenant.org/version/1/4/ 47 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2018 Itamar Turner-Trauring 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. 22 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # dask-memusage 2 | 3 | If you're using Dask with tasks that use a lot of memory, RAM is your bottleneck for parallelism. 4 | That means you want to know how much memory each task uses: 5 | 6 | 1. So you can set the highest parallelism level (process or threads) for each machine, given available to RAM. 7 | 2. In order to know where to focus memory optimization efforts. 8 | 9 | `dask-memusage` is an MIT-licensed statistical memory profiler for Dask's Distributed scheduler that can help you with both these problems. 10 | 11 | `dask-memusage` polls your processes for memory usage and records the minimum and maximum usage for each task in the Dask execution graph in a CSV: 12 | 13 | ```csv 14 | task_key,min_memory_mb,max_memory_mb 15 | "('from_sequence-map-sum-part-e15703211a549e75b11c63e0054b53e5', 0)",44.84765625,96.98046875 16 | "('from_sequence-map-sum-part-e15703211a549e75b11c63e0054b53e5', 1)",47.015625,97.015625 17 | "('sum-part-e15703211a549e75b11c63e0054b53e5', 0)",0,0 18 | "('sum-part-e15703211a549e75b11c63e0054b53e5', 1)",0,0 19 | sum-aggregate-apply-no_allocate-4c30eb545d4c778f0320d973d9fc8ea6,0,0 20 | apply-no_allocate-4c30eb545d4c778f0320d973d9fc8ea6,47.265625,47.265625 21 | task_key,min_memory_mb,max_memory_mb 22 | "('from_sequence-map-sum-part-e15703211a549e75b11c63e0054b53e5', 0)",44.84765625,96.98046875 23 | "('from_sequence-map-sum-part-e15703211a549e75b11c63e0054b53e5', 1)",47.015625,97.015625 24 | "('sum-part-e15703211a549e75b11c63e0054b53e5', 0)",0,0 25 | "('sum-part-e15703211a549e75b11c63e0054b53e5', 1)",0,0 26 | sum-aggregate-apply-no_allocate-4c30eb545d4c778f0320d973d9fc8ea6,0,0 27 | apply-no_allocate-4c30eb545d4c778f0320d973d9fc8ea6,47.265625,47.265625 28 | ``` 29 | 30 | You may also find the [Fil memory profiler](https://pythonspeed.com/fil) useful in tracking down which specific parts of your code are responsible for peak memory allocations. 31 | 32 | ## Example 33 | 34 | Here's a working standalone program using `dask-memusage`; notice you just need to add two lines of code: 35 | 36 | ```python 37 | from time import sleep 38 | import numpy as np 39 | from dask.bag import from_sequence 40 | from dask import compute 41 | from dask.distributed import Client, LocalCluster 42 | 43 | from dask_memusage import install # <-- IMPORT 44 | 45 | def allocate_50mb(x): 46 | """Allocate 50MB of RAM.""" 47 | sleep(1) 48 | arr = np.ones((50, 1024, 1024), dtype=np.uint8) 49 | sleep(1) 50 | return x * 2 51 | 52 | def no_allocate(y): 53 | """Don't allocate any memory.""" 54 | return y * 2 55 | 56 | def make_bag(): 57 | """Create a bag.""" 58 | return from_sequence( 59 | [1, 2], npartitions=2 60 | ).map(allocate_50mb).sum().apply(no_allocate) 61 | 62 | def main(): 63 | cluster = LocalCluster(n_workers=2, threads_per_worker=1, 64 | memory_limit=None) 65 | install(cluster.scheduler, "memusage.csv") # <-- INSTALL 66 | client = Client(cluster) 67 | compute(make_bag()) 68 | 69 | if __name__ == '__main__': 70 | main() 71 | ``` 72 | 73 | ## Usage 74 | 75 | *Important:* Make sure your workers only have a single thread! Otherwise the results will be wrong. 76 | 77 | ### Installation 78 | 79 | On the machine where you are running the Distributed scheduler, run: 80 | 81 | ```console 82 | $ pip install dask_memusage 83 | ``` 84 | 85 | Or if you're using Conda: 86 | 87 | ```console 88 | $ conda install -c conda-forge dask-memusage 89 | ``` 90 | 91 | ### API usage 92 | 93 | ```python 94 | # Add to your Scheduler object, which is e.g. your LocalCluster's scheduler 95 | # attribute: 96 | from dask_memoryusage import install 97 | install(scheduler, "/tmp/memusage.csv") 98 | ``` 99 | 100 | ### CLI usage 101 | 102 | ```console 103 | $ dask-scheduler --preload dask_memusage --memusage.csv /tmp/memusage.csv 104 | ``` 105 | 106 | ## Limitations 107 | 108 | * Again, make sure you only have one thread per worker process. 109 | * This is statistical profiling, running every 10ms. 110 | Tasks that take less than that won't have accurate information. 111 | 112 | ## Help 113 | 114 | Need help? File a ticket at https://github.com/itamarst/dask-memusage/issues/new 115 | -------------------------------------------------------------------------------- /dask_memusage.py: -------------------------------------------------------------------------------- 1 | """Low-impact, task-level memory profiling for Dask. 2 | 3 | API usage: 4 | 5 | from dask_memoryusage import install 6 | install(scheduler, "/tmp/memusage.csv") 7 | 8 | CLI usage: 9 | 10 | dask-scheduler --preload dask_memusage --memusage.csv /tmp/memusage.csv 11 | 12 | """ 13 | 14 | import os 15 | import csv 16 | from time import sleep 17 | from threading import Lock, Thread 18 | from collections import defaultdict 19 | 20 | from psutil import Process 21 | import click 22 | 23 | from distributed.diagnostics.plugin import SchedulerPlugin 24 | from distributed.client import Client 25 | from distributed.scheduler import Scheduler 26 | 27 | 28 | __all__ = ["install"] 29 | __version__ = "1.1" 30 | 31 | 32 | def _process_memory(): 33 | """Return process memory usage, in MB. 34 | 35 | We include memory used by subprocesses. 36 | """ 37 | proc = Process(os.getpid()) 38 | return sum([ 39 | p.memory_info().rss / (1024 * 1024) 40 | for p in [proc] + list(proc.children(recursive=True)) 41 | ]) 42 | 43 | 44 | class _WorkerMemory(object): 45 | """Track memory usage by each worker.""" 46 | 47 | def __init__(self, scheduler_address): 48 | self._scheduler_address = scheduler_address 49 | self._lock = Lock() 50 | self._worker_memory = defaultdict(list) 51 | 52 | def start(self): 53 | """Start the thread.""" 54 | t = Thread(target=self._fetch_memory, name="WorkerMemory") 55 | t.setDaemon(True) 56 | t.start() 57 | 58 | def _add_memory(self, worker_address, mem): 59 | """Record memory timepoint for a worker.""" 60 | self._worker_memory[worker_address].append(mem) 61 | 62 | def _fetch_memory(self): 63 | """Retrieve worker memory every 10ms.""" 64 | client = Client(self._scheduler_address, timeout=30) 65 | while True: 66 | worker_to_mem = client.run(_process_memory) 67 | with self._lock: 68 | for worker, mem in worker_to_mem.items(): 69 | self._add_memory(worker, mem) 70 | sleep(0.01) 71 | 72 | def memory_for_task(self, worker_address): 73 | """The worker finished its previous task. 74 | 75 | Return its memory usage and then reset it. 76 | """ 77 | with self._lock: 78 | result = self._worker_memory[worker_address] 79 | if not result: 80 | result = [0] 81 | del self._worker_memory[worker_address] 82 | return result 83 | 84 | 85 | class MemoryUsagePlugin(SchedulerPlugin): 86 | """Record max and min memory usage for a task. 87 | 88 | Assumptions: 89 | 90 | * One task per process: each process has a single thread running a single 91 | task at a time. 92 | 93 | Limitations: 94 | 95 | * Statistical profiling at 10ms resolution. 96 | """ 97 | def __init__(self, scheduler, csv_path): 98 | SchedulerPlugin.__init__(self) 99 | f = open(os.path.join(csv_path), "w", buffering=1) 100 | self._csv = csv.writer(f) 101 | self._csv.writerow(["task_key", "min_memory_mb", "max_memory_mb"]) 102 | self._worker_memory = _WorkerMemory(scheduler.address) 103 | self._worker_memory.start() 104 | 105 | def transition(self, key, start, finish, *args, **kwargs): 106 | """Called by the Scheduler every time a task changes status.""" 107 | # If the task finished, record its memory usage: 108 | if start == "processing" and finish in ("memory", "erred"): 109 | worker_address = kwargs["worker"] 110 | memory_usage = self._worker_memory.memory_for_task(worker_address) 111 | max_memory_usage = max(memory_usage) 112 | min_memory_usage = min(memory_usage) 113 | self._csv.writerow([key, min_memory_usage, max_memory_usage]) 114 | 115 | 116 | def install(scheduler: Scheduler, csv_path: str): 117 | """Register the memory usage profiler with a distributed Scheduler. 118 | 119 | :param scheduler: The Distributed Scheduler to register with. 120 | :param csv_path: The filesystem path where the CSV file will be written. 121 | """ 122 | plugin = MemoryUsagePlugin(scheduler, csv_path) 123 | scheduler.add_plugin(plugin) 124 | 125 | 126 | @click.command() 127 | @click.option("--memusage-csv", default="memusage.csv") 128 | def dask_setup(scheduler, memusage_csv): 129 | install(scheduler, memusage_csv) 130 | -------------------------------------------------------------------------------- /pyproject.toml: -------------------------------------------------------------------------------- 1 | [build-system] 2 | requires = ["flit"] 3 | build-backend = "flit.buildapi" 4 | 5 | [tool.flit.metadata] 6 | module = "dask_memusage" 7 | description-file = "README.md" 8 | requires-python = ">=3.6" 9 | requires = [ 10 | "distributed", 11 | "click", 12 | ] 13 | author = "Itamar Turner-Trauring" 14 | author-email = "itamar@itamarst.org" 15 | home-page = "https://github.com/itamarst/dask-memusage" 16 | classifiers = [ 17 | "Programming Language :: Python :: 3", 18 | "License :: OSI Approved :: MIT License", 19 | ] 20 | 21 | [tool.flit.metadata.requires-extra] 22 | test = [ 23 | "pytest", 24 | "flake8", 25 | "numpy", 26 | "dask[bag]" 27 | ] -------------------------------------------------------------------------------- /test_memusage.py: -------------------------------------------------------------------------------- 1 | """Tests for dask_memusage.""" 2 | 3 | import os 4 | from time import sleep 5 | from csv import DictReader 6 | from subprocess import Popen 7 | 8 | import numpy as np 9 | from dask.bag import from_sequence 10 | from dask import compute 11 | from dask.distributed import Client, LocalCluster 12 | 13 | from dask_memusage import install, _WorkerMemory 14 | 15 | 16 | def allocate_50mb(x): 17 | """Allocate 50MB of RAM.""" 18 | sleep(1) 19 | arr = np.ones((50, 1024, 1024), dtype=np.uint8) 20 | sleep(1) 21 | return x * 2 22 | 23 | def no_allocate(y): 24 | """Don't allocate any memory.""" 25 | return y * 2 26 | 27 | 28 | def make_bag(): 29 | """Create a bag.""" 30 | return from_sequence( 31 | [1, 2], npartitions=2 32 | ).map(allocate_50mb).sum().apply(no_allocate) 33 | 34 | 35 | def test_workermemory(): 36 | """_WorkerMemory adds memory per-worker and removes it per-worker. 37 | 38 | If no info is known, we just get 0. 39 | """ 40 | wm = _WorkerMemory("") 41 | assert wm.memory_for_task("123") == [0] 42 | wm._add_memory("123", 7) 43 | wm._add_memory("456", 3) 44 | wm._add_memory("123", 8) 45 | assert wm.memory_for_task("123") == [7, 8] 46 | assert wm.memory_for_task("456") == [3] 47 | assert wm.memory_for_task("123") == [0] 48 | assert wm.memory_for_task("456") == [0] 49 | 50 | 51 | def test_highlevel_python_usage(tmpdir): 52 | """We can add a MemoryUsagePlugin and get memory usage written out.""" 53 | tempfile = str(tmpdir / "out.csv") 54 | cluster = LocalCluster(n_workers=2, threads_per_worker=1, 55 | memory_limit=None) 56 | install(cluster.scheduler, tempfile) 57 | client = Client(cluster) 58 | compute(make_bag()) 59 | check_csv(tempfile) 60 | 61 | 62 | def test_commandline_usage(tmpdir): 63 | """We can add a MemoryUsagePlugin to a dask-scheduler subprocess.""" 64 | tempfile = str(tmpdir / "out.csv") 65 | env = os.environ.copy() 66 | env["PYTHONPATH"] = "." 67 | scheduler = Popen(["dask-scheduler", 68 | "--port", "3333", 69 | "--host", "127.0.0.1", 70 | "--preload", "dask_memusage", 71 | "--memusage-csv", tempfile], 72 | env=env) 73 | worker = Popen(["dask-worker", "tcp://127.0.0.1:3333", 74 | "--nthreads", "1"], 75 | env=env) 76 | try: 77 | client = Client("tcp://127.0.0.1:3333") 78 | compute(make_bag()) 79 | check_csv(tempfile) 80 | finally: 81 | worker.kill() 82 | scheduler.kill() 83 | 84 | 85 | def check_csv(csv_file): 86 | """Make sure CSV contains appropriate information.""" 87 | with open(csv_file) as f: 88 | print(f.read()) 89 | result = [] 90 | found_50mb = 0 91 | found_0mb = 0 92 | with open(csv_file) as f: 93 | csv = DictReader(f) 94 | for row in csv: 95 | allocated = float(row["max_memory_mb"]) - float(row["min_memory_mb"]) 96 | if "allocate_50mb" in row["task_key"]: 97 | assert 70 > allocated > 49 98 | found_50mb += 1 99 | elif "no_allocate" in row["task_key"]: 100 | assert 1 > allocated >= 0 101 | found_0mb += 1 102 | assert found_50mb >= 2 103 | assert found_0mb >= 1 104 | --------------------------------------------------------------------------------