-
-
Notifications
You must be signed in to change notification settings - Fork 734
Closed
Description
Sometimes we fail to clean up Pandas dataframes. Here are a couple of examples using dask.dataframe and dask.array on a "cluster" in the local environment. In the dask.array case we cleanly remove used data. In the dask.dataframe case the memory of our process remains high even after we delete things.
Clean case with dask.array
In [1]: import psutil
In [2]: proc = psutil.Process()
In [3]: proc.memory_info().rss / 1e6
Out[3]: 39.768064
In [4]: from distributed import Client, LocalCluster
In [5]: cluster = LocalCluster(nanny=False, n_workers=4, threads_per_worker=1)
In [6]: client = Client(cluster)
In [7]: bokeh.command.util - CRITICAL - Cannot start Bokeh server, port 8787 is already in use
In [7]:
In [7]: import dask.array as da
In [8]: x = da.random.random(100000000, chunks=(1000000,)).persist()
In [9]: proc.memory_info().rss / 1e6
Out[9]: 872.411136
In [10]: proc.memory_info().rss / 1e6
Out[10]: 872.411136
In [11]: proc.memory_info().rss / 1e6
Out[11]: 872.411136
In [12]: del x
In [13]: proc.memory_info().rss / 1e6
Out[13]: 72.081408
In [14]: proc.memory_info().rss / 1e6
Out[14]: 72.081408
Leaking case with Pandas dataframes
In [1]: import psutil
In [2]: proc = psutil.Process()
In [3]: proc.memory_info().rss / 1e6
Out[3]: 38.109184
In [4]: proc.memory_info().rss / 1e6
Out[4]: 38.37952
In [5]: from distributed import Client, LocalCluster
In [6]: proc.memory_info().rss / 1e6
Out[6]: 51.851264
In [7]: cluster = LocalCluster(nanny=False, n_workers=4, threads_per_worker=1)
In [8]: client = Client(cluster)bokeh.command.util - CRITICAL - Cannot start Boe
In [8]: client = Client(cluster)
In [9]: proc.memory_info().rss / 1e6
Out[9]: 65.073152
In [10]: import dask.dataframe as dd
In [11]: df = dd.demo.daily_stock('GOOG', '2008', '2010', freq='1s',
...: random_state=1234).persist()
In [12]: proc.memory_info().rss / 1e6
Out[12]: 584.060928
In [13]: proc.memory_info().rss / 1e6
Out[13]: 685.867008
In [14]: proc.memory_info().rss / 1e6
Out[14]: 804.74112
In [15]: proc.memory_info().rss / 1e6
Out[15]: 927.670272
In [16]: proc.memory_info().rss / 1e6
Out[16]: 996.499456
In [17]: proc.memory_info().rss / 1e6
Out[17]: 997.445632
In [18]: proc.memory_info().rss / 1e6
Out[18]: 997.445632
In [19]: del df
In [20]: cluster.scheduler.tasks
Out[20]: {}
In [21]: proc.memory_info().rss / 1e6
Out[21]: 997.711872
cc @pitrou and @jreback any thoughts on how to track this down or avoid it?
Metadata
Metadata
Assignees
Labels
No labels