From 21a8152bd4a4b3200010622586bc9d264548a1d2 Mon Sep 17 00:00:00 2001 From: Asli Bese Date: Wed, 20 Nov 2024 12:06:25 -0500 Subject: [PATCH 01/10] initial commit, faq on dask and parallelization --- docs/source/notebooks/FAQ_dask_parallel.ipynb | 19101 ++++++++++++++++ 1 file changed, 19101 insertions(+) create mode 100644 docs/source/notebooks/FAQ_dask_parallel.ipynb diff --git a/docs/source/notebooks/FAQ_dask_parallel.ipynb b/docs/source/notebooks/FAQ_dask_parallel.ipynb new file mode 100644 index 0000000..7778a9f --- /dev/null +++ b/docs/source/notebooks/FAQ_dask_parallel.ipynb @@ -0,0 +1,19101 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "id": "b898ae93-3234-44b1-a88f-5aa0c1d8f6b9", + "metadata": {}, + "source": [ + "# Processing Large Climate Datasets with Dask and Xarray\n", + "### How to align memory chunks with on-disk chunks?\n", + "Datasets on PAVICS are stored on disk in chunks (small blocks of data) which Dask processes one at a time. To efficiently process large datasets and prevent memory overloads, aligning in-memory chunks with the on-disk chunk structure is essential. This allows Dask to load and process data in a way that minimizes memory usage and I/O operations, speeding up computation. \n", + "\n", + "I begin by checking the on-disk chunk structure of my dataset by loading it with `decode_times=False`. This skips time decoding and loads only metadata without reading the data into memory." + ] + }, + { + "cell_type": "code", + "execution_count": 12, + "id": "9f12da60-e3df-4172-a9cb-e014801ba7c9", + "metadata": { + "tags": [] + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "[366 50 50]\n" + ] + } + ], + "source": [ + "import xarray as xr \n", + "\n", + "# open the daily ERA5-land dataset as an example\n", + "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", + "\n", + "# use decode_times=False to skip time decoding and load only metadata\n", + "ds = xr.open_dataset(url, decode_times=False, chunks={})\n", + "\n", + "# `._ChunkSizes` attribute shows the chunk structure of the variable\n", + "print(ds['tas']._ChunkSizes)" + ] + }, + { + "cell_type": "markdown", + "id": "1ab719a3-3de0-4344-aa9a-2836d4a099ed", + "metadata": { + "tags": [] + }, + "source": [ + "The `._ChunkSizes` attribute shows the chunk sizes of the `tas` variable on disk. I then re-open the dataset with matching Dask chunk sizes for efficient memory alignment. \n", + "\n", + "When you open a dataset using chunking, it is represented as Dask-backed arrays, where only the metadata is initially loaded, and the data itself remains on disk until needed. Dask uses lazy evalution, meaning it doesn't immediately perform the operation but instead builds a computation graph. This graph tracks the sequence of operations, delaying execution until `compute()` is explicitly called. During computation, Dask reads and processes data in chunks and loads only the necessary parts into memory. \n", + "\n", + "Here's an example showing the difference in computation times when the dataset is loaded with unaligned versus aligned chunking. For this example, I resample daily ERA5-Land `tas` data to yearly means." + ] + }, + { + "cell_type": "code", + "execution_count": 2, + "id": "4d646c97-eda5-4070-aa54-19ce91d908dc", + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "CPU times: user 26.4 s, sys: 5.6 s, total: 32 s\n", + "Wall time: 3min 6s\n" + ] + } + ], + "source": [ + "# open dataset with unaligned chunks\n", + "ds = xr.open_dataset(url, chunks={'time': 200, 'lat': 30, 'lon': 30})\n", + "\n", + "# resample the 'tas' variable to yearly means\n", + "%time tas_resampled = ds['tas'].sel(time=slice('1981-01-01', '1981-12-31')).resample(time='YS').mean().compute()" + ] + }, + { + "cell_type": "code", + "execution_count": 3, + "id": "a449349b-a339-4d96-aa33-12a97d9ec51e", + "metadata": { + "tags": [] + }, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "CPU times: user 19.7 s, sys: 7.46 s, total: 27.2 s\n", + "Wall time: 58.6 s\n" + ] + } + ], + "source": [ + "# open dataset with aligned chunks\n", + "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + "\n", + "# resample the 'tas' variable to yearly means\n", + "%time tas_resampled = ds['tas'].sel(time=slice('1981-01-01', '1981-12-31')).resample(time='YS').mean().compute()" + ] + }, + { + "cell_type": "markdown", + "id": "47a56ac9-2d80-43ef-a283-12a2e2d42bfd", + "metadata": {}, + "source": [ + "Aligning the chunks reduced the computation time from approximately 3 minutes to 1 minute, by allowing Dask to load only the required data and avoid excessive memory usage and I/O operations. " + ] + }, + { + "cell_type": "markdown", + "id": "2422a144-7765-4d29-9703-93d0c54a4c02", + "metadata": {}, + "source": [ + "### How to do parallel processing with Dask Distributed Client? \n", + "Processing large climate datasets can be computationally intensive and time-consuming due to their size and complexity. Parallel processing divides a computational task into smaller, independent subtasks that can be executed simultaneously across multiple processors or cores. This can minimize memory overhead and significantly reduce computation time. \n", + "\n", + "One way to implement parallel processing is through the Dask Distributed Client. When you initialize a Dask Client, you establish a connection to the cluster, allowing interaction with the scheduler and workers. The scheduler is the central component that manages task distribution across the workers, and the workers are the actual computational units that execute tasks using one or more CPU cores. \n", + "\n", + "Let's look at an example of resampling daily ERA5-Land data into yearly means for multiple variables using the Distributed Client. To manage system resources effectively, especially since PAVICS is a shared environment, I set the total memory limit to 20 GB. After experimenting with different configurations, I found that using 5 workers, with 2 threads per worker and 4 GB memory per worker provides a good balance between reducing overhead and optimizing computation time. I track memory usage and task progress in real time by displaying the Dask dashboard.\n", + "\n", + "To create delayed tasks, I set `compute=False` with `to_zarr`. This delays the entire pipeline, from reading the data, performing computations, to writing the output, and allows Dask to optimize the execution plan. Dask schedules tasks efficiently, reducing unnecessary data movement and enabling each worker to write data independently to Zarr format. This approach is much faster than using sequential formats like NetCDF, which don’t support parallel writes as effectively.\n", + "\n", + "To prevent memory leaks, which occur when the program retains memory it no longer needs, potentially slowing down the system or causing it to crash, I use the Client within a context manager to ensure that resources are released after execution. " + ] + }, + { + "cell_type": "code", + "execution_count": 11, + "id": "73765dbb-3bf7-4814-b44e-6246713275de", + "metadata": { + "tags": [] + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-4c5b0b8a-a124-11ef-ab33-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

f2d89cc8

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-32d45067-a2b5-4fb0-92a9-a07bb97da0ae

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:46341\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38031\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36389/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35377\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-74on4p3n\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33339\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43587/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37279\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-c6u7zge8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35613\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34169/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33505\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-2jbdsdhb\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37163\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43807/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38151\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ego2rpka\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35367\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46437/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38855\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-jhacn10g\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 800, lon: 1700, time: 26298)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
+       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
+       "  * time     (time) datetime64[ns] 1950-01-01 1950-01-02 ... 2021-12-31\n",
+       "Data variables:\n",
+       "    tas      (time, lat, lon) float32 dask.array<chunksize=(13421, 50, 50), meta=np.ndarray>\n",
+       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(13421, 50, 50), meta=np.ndarray>\n",
+       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(13421, 50, 50), meta=np.ndarray>\n",
+       "    pr       (time, lat, lon) float32 dask.array<chunksize=(13421, 50, 50), meta=np.ndarray>\n",
+       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(13421, 50, 50), meta=np.ndarray>\n",
+       "Attributes: (12/26)\n",
+       "    Conventions:          CF-1.8\n",
+       "    cell_methods:         time: mean (interval: 1 day)\n",
+       "    data_specs_version:   00.00.07\n",
+       "    domain:               NAM\n",
+       "    format:               netcdf\n",
+       "    frequency:            day\n",
+       "    ...                   ...\n",
+       "    dataset_description:  https://www.ecmwf.int/en/era5-land\n",
+       "    license_type:         permissive\n",
+       "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
+       "    attribution:          Contains modified Copernicus Climate Change Service...\n",
+       "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
+       "    doi:                  https://doi.org/10.24381/cds.e2161bac
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 800, lon: 1700, time: 26298)\n", + "Coordinates:\n", + " * lat (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n", + " * lon (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", + " * time (time) datetime64[ns] 1950-01-01 1950-01-02 ... 2021-12-31\n", + "Data variables:\n", + " tas (time, lat, lon) float32 dask.array\n", + " tasmin (time, lat, lon) float32 dask.array\n", + " tasmax (time, lat, lon) float32 dask.array\n", + " pr (time, lat, lon) float32 dask.array\n", + " prsn (time, lat, lon) float32 dask.array\n", + "Attributes: (12/26)\n", + " Conventions: CF-1.8\n", + " cell_methods: time: mean (interval: 1 day)\n", + " data_specs_version: 00.00.07\n", + " domain: NAM\n", + " format: netcdf\n", + " frequency: day\n", + " ... ...\n", + " dataset_description: https://www.ecmwf.int/en/era5-land\n", + " license_type: permissive\n", + " license: Please acknowledge the use of ERA5-Land as stated i...\n", + " attribution: Contains modified Copernicus Climate Change Service...\n", + " citation: Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n", + " doi: https://doi.org/10.24381/cds.e2161bac" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Total memory usage across all workers: 1.40 GB\n", + "Total computation time: 76.60 seconds\n" + ] + } + ], + "source": [ + "import xarray as xr\n", + "from pathlib import Path\n", + "from dask.distributed import Client\n", + "from dask import compute\n", + "import psutil \n", + "import time\n", + "\n", + "start_time = time.time()\n", + "\n", + "# function to compute the yearly mean for each variable \n", + "def compute_yearly_mean(ds, var_name):\n", + " var = ds[var_name]\n", + " yearly_mean = var.resample(time='YS').mean()\n", + " return yearly_mean\n", + "\n", + "# set up Dask client within a context manager\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", + " # display the Dask Dashboard\n", + " display(client)\n", + "\n", + " # open the dataset with on-disk chunking structure\n", + " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", + " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon':50}) \n", + " display(ds)\n", + "\n", + " # select data for the year 1981\n", + " ds = ds.sel(time=slice('1981-01-01', '1981-12-31'))\n", + "\n", + " variables = list(ds.data_vars)\n", + "\n", + " # create a list to hold delayed tasks\n", + " tasks = []\n", + " for var_name in variables:\n", + " output_path = Path(f\"var_means/{var_name}_1981_yearly_mean.zarr\")\n", + " if not output_path.exists():\n", + " yearly_mean = compute_yearly_mean(ds, var_name)\n", + " # save to Zarr with compute=False to get a delayed task object\n", + " delayed_task = yearly_mean.to_zarr(output_path, mode='w', compute=False)\n", + " tasks.append(delayed_task)\n", + "\n", + " # trigger the execution of all delayed tasks\n", + " compute(*tasks)\n", + "\n", + " # fetch memory usage from all workers and display the total usage\n", + " worker_memory = client.run(lambda: psutil.Process().memory_info().rss)\n", + " total_memory = sum(worker_memory.values())\n", + " print(f\"Total memory usage across all workers: {total_memory / 1e9:.2f} GB\")\n", + "\n", + "end_time = time.time()\n", + "elapsed_time = end_time - start_time\n", + "print(f'Total computation time: {elapsed_time:.2f} seconds')" + ] + }, + { + "cell_type": "markdown", + "id": "e9cee3ca-a916-4d16-aab8-75dc3692217c", + "metadata": {}, + "source": [ + "Let’s look at a more complex analysis involving heatwave indicators. Here, I calculate two climate indicators, `heat_wave_total_length` and `heat_wave_frequency`, using the `atmos` submodule from the `xclim` library. Both indicators rely on the same input data (`tasmin` and `tasmax`), so I create a pipeline of delayed tasks, which minimizes I/O operations by keeping the data in memory until both indicators are calculated and saved." + ] + }, + { + "cell_type": "code", + "execution_count": 18, + "id": "c24f1efe-65b8-48af-9090-ae516e6840d6", + "metadata": { + "tags": [] + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-77bed890-a207-11ef-ab33-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

ac36ff1b

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-565ce5b4-3926-4ea7-a67a-7ac7806f623d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:41377\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:45771\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45109/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40321\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-i7netdoj\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44925\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36703/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41683\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-otas7pcr\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41497\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38523/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37981\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-bdoasynd\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41131\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45485/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33285\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-2wg7r3g0\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33721\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37593/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42447\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-xm2_1gwu\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 800, lon: 1700, time: 7305)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
+       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
+       "  * time     (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n",
+       "Data variables:\n",
+       "    tas      (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    pr       (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "Attributes: (12/26)\n",
+       "    Conventions:          CF-1.8\n",
+       "    cell_methods:         time: mean (interval: 1 day)\n",
+       "    data_specs_version:   00.00.07\n",
+       "    domain:               NAM\n",
+       "    format:               netcdf\n",
+       "    frequency:            day\n",
+       "    ...                   ...\n",
+       "    dataset_description:  https://www.ecmwf.int/en/era5-land\n",
+       "    license_type:         permissive\n",
+       "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
+       "    attribution:          Contains modified Copernicus Climate Change Service...\n",
+       "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
+       "    doi:                  https://doi.org/10.24381/cds.e2161bac
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 800, lon: 1700, time: 7305)\n", + "Coordinates:\n", + " * lat (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n", + " * lon (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", + " * time (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n", + "Data variables:\n", + " tas (time, lat, lon) float32 dask.array\n", + " tasmin (time, lat, lon) float32 dask.array\n", + " tasmax (time, lat, lon) float32 dask.array\n", + " pr (time, lat, lon) float32 dask.array\n", + " prsn (time, lat, lon) float32 dask.array\n", + "Attributes: (12/26)\n", + " Conventions: CF-1.8\n", + " cell_methods: time: mean (interval: 1 day)\n", + " data_specs_version: 00.00.07\n", + " domain: NAM\n", + " format: netcdf\n", + " frequency: day\n", + " ... ...\n", + " dataset_description: https://www.ecmwf.int/en/era5-land\n", + " license_type: permissive\n", + " license: Please acknowledge the use of ERA5-Land as stated i...\n", + " attribution: Contains modified Copernicus Climate Change Service...\n", + " citation: Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n", + " doi: https://doi.org/10.24381/cds.e2161bac" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stderr", + "output_type": "stream", + "text": [ + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + " _check_cell_methods(\n" + ] + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "indicator : \n" + ] + }, + { + "name": "stderr", + "output_type": "stream", + "text": [ + "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 15.73 MiB.\n", + "This may cause some slowdown.\n", + "Consider scattering data ahead of time and using futures.\n", + " warnings.warn(\n" + ] + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Total memory usage across all workers: 1.95 GB\n", + "Total computation time: 415.93 seconds\n" + ] + } + ], + "source": [ + "from pathlib import Path\n", + "from dask.distributed import Client\n", + "import time\n", + "from xclim import atmos\n", + "import xarray as xr\n", + "from dask import compute\n", + "import psutil \n", + "\n", + "start_time = time.time()\n", + "\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", + " display(client)\n", + " \n", + " # load data using on-disk chunk sizes\n", + " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", + " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + " \n", + " # we're interested in the 1991-2010 period for our heatwave analysis\n", + " ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", + " \n", + " display(ds)\n", + "\n", + " # list of heatwave indicator functions\n", + " indicators = [atmos.heat_wave_total_length, atmos.heat_wave_frequency, atmos._precip.precip_accumulation] \n", + " tasks = []\n", + " for indicator in indicators:\n", + " ds_out = xr.Dataset(attrs=ds.attrs) # create a new dataset for each indicator\n", + "\n", + " if indicator == atmos._precip.precip_accumulation:\n", + " # calculate indicator\n", + " out = indicator(\n", + " ds=ds,\n", + " freq='YS',\n", + " )\n", + " else: \n", + " out = indicator(\n", + " ds=ds,\n", + " freq='YS',\n", + " resample_before_rl=False\n", + " )\n", + " \n", + " out = out.chunk({'time': -1, 'lat': 50, 'lon': 50})\n", + " ds_out[out.name] = out\n", + "\n", + " output_path = Path(f'heatwave_output_ex1/{out.name}_1991-2010.zarr')\n", + " output_path.parent.mkdir(parents=True, exist_ok=True)\n", + "\n", + " if not output_path.exists():\n", + " # save to Zarr with compute=False to get a delayed task object \n", + " delayed_task = ds_out.to_zarr(output_path, mode='w', compute=False)\n", + " tasks.append(delayed_task)\n", + "\n", + " # trigger computation \n", + " compute(*tasks)\n", + " \n", + " # fetch memory usage from all workers and display the total usage\n", + " worker_memory = client.run(lambda: psutil.Process().memory_info().rss)\n", + " total_memory = sum(worker_memory.values())\n", + " print(f\"Total memory usage across all workers: {total_memory / 1e9:.2f} GB\")\n", + " \n", + "end_time = time.time()\n", + "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" + ] + }, + { + "cell_type": "markdown", + "id": "12d29293-ad43-44f5-8d4a-1ecdbf8574ce", + "metadata": {}, + "source": [ + "### What can we do when we have large task graphs / large memory footprint? \n", + "One downside of using a fully delayed computation approach is that it can lead to the creation of large task graphs that are difficult to manage. This can result in excessive memory consumption as the Dask scheduler struggles to handle numerous interdependent tasks.\n", + "\n", + "To address this issue, we can simplify the task graph by computing and saving each indicator one at a time. This method ensures that Dask completes the calculation and writing of the first indicator, then releases the memory used for that computation before moving on to the second indicator. By processing each indicator sequentially, the memory footprint is reduced, and the scheduler has fewer tasks to manage at any given time." + ] + }, + { + "cell_type": "code", + "execution_count": 22, + "id": "48e59f9d-3d12-4119-9f3c-0cd694ccd871", + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-b8e9b705-a2bb-11ef-ab33-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

24ecda26

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-66a88548-a02f-4504-9661-370df089e931

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:41109\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44813\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34925/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44017\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ck_2wyzo\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42169\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44117/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35551\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-pjs6yx3h\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39133\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36913/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46425\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-l0k5a_85\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34665\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35467/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38721\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-zkv4axvy\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36431\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33927/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43431\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-e5q354bb\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 800, lon: 1700, time: 7305)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
+       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
+       "  * time     (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n",
+       "Data variables:\n",
+       "    tas      (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    pr       (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "Attributes: (12/26)\n",
+       "    Conventions:          CF-1.8\n",
+       "    cell_methods:         time: mean (interval: 1 day)\n",
+       "    data_specs_version:   00.00.07\n",
+       "    domain:               NAM\n",
+       "    format:               netcdf\n",
+       "    frequency:            day\n",
+       "    ...                   ...\n",
+       "    dataset_description:  https://www.ecmwf.int/en/era5-land\n",
+       "    license_type:         permissive\n",
+       "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
+       "    attribution:          Contains modified Copernicus Climate Change Service...\n",
+       "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
+       "    doi:                  https://doi.org/10.24381/cds.e2161bac
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 800, lon: 1700, time: 7305)\n", + "Coordinates:\n", + " * lat (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n", + " * lon (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", + " * time (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n", + "Data variables:\n", + " tas (time, lat, lon) float32 dask.array\n", + " tasmin (time, lat, lon) float32 dask.array\n", + " tasmax (time, lat, lon) float32 dask.array\n", + " pr (time, lat, lon) float32 dask.array\n", + " prsn (time, lat, lon) float32 dask.array\n", + "Attributes: (12/26)\n", + " Conventions: CF-1.8\n", + " cell_methods: time: mean (interval: 1 day)\n", + " data_specs_version: 00.00.07\n", + " domain: NAM\n", + " format: netcdf\n", + " frequency: day\n", + " ... ...\n", + " dataset_description: https://www.ecmwf.int/en/era5-land\n", + " license_type: permissive\n", + " license: Please acknowledge the use of ERA5-Land as stated i...\n", + " attribution: Contains modified Copernicus Climate Change Service...\n", + " citation: Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n", + " doi: https://doi.org/10.24381/cds.e2161bac" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stderr", + "output_type": "stream", + "text": [ + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 36.85 MiB.\n", + "This may cause some slowdown.\n", + "Consider scattering data ahead of time and using futures.\n", + " warnings.warn(\n", + "2024-11-14 19:09:25,025 - tornado.application - ERROR - Uncaught exception GET /status/ws (127.0.0.1)\n", + "HTTPServerRequest(protocol='http', host='pavics.ouranos.ca', method='GET', uri='/status/ws', version='HTTP/1.1', remote_ip='127.0.0.1')\n", + "Traceback (most recent call last):\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/web.py\", line 1790, in _execute\n", + " result = await result\n", + " ^^^^^^^^^^^^\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 273, in get\n", + " await self.ws_connection.accept_connection(self)\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 863, in accept_connection\n", + " await self._accept_connection(handler)\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 946, in _accept_connection\n", + " await self._receive_frame_loop()\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 1105, in _receive_frame_loop\n", + " self.handler.on_ws_connection_close(self.close_code, self.close_reason)\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 571, in on_ws_connection_close\n", + " self.on_connection_close()\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 563, in on_connection_close\n", + " self.on_close()\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/bokeh/server/views/ws.py\", line 308, in on_close\n", + " self.connection.session.notify_connection_lost()\n", + " ^^^^^^^^^^^^^^^^^^^^^^^\n", + " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/bokeh/server/connection.py\", line 65, in session\n", + " assert self._session is not None\n", + " ^^^^^^^^^^^^^^^^^^^^^^^^^\n", + "AssertionError\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + " _check_cell_methods(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 36.86 MiB.\n", + "This may cause some slowdown.\n", + "Consider scattering data ahead of time and using futures.\n", + " warnings.warn(\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 16.15 MiB.\n", + "This may cause some slowdown.\n", + "Consider scattering data ahead of time and using futures.\n", + " warnings.warn(\n" + ] + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Total memory usage across all workers: 2.65 GB\n", + "Total computation time: 2484.91 seconds\n" + ] + } + ], + "source": [ + "from pathlib import Path\n", + "from dask.distributed import Client\n", + "import time\n", + "from xclim import atmos\n", + "from dask import compute\n", + "import psutil \n", + "\n", + "start_time = time.time()\n", + "\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", + " display(client)\n", + " \n", + " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", + " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + " \n", + " ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", + " \n", + " display(ds)\n", + "\n", + " indicators = [atmos.heat_wave_total_length, atmos.heat_wave_frequency, atmos._precip.precip_accumulation] \n", + "\n", + " for indicator in indicators:\n", + " ds_out = xr.Dataset(attrs=ds.attrs) \n", + " \n", + " if indicator == atmos._precip.precip_accumulation:\n", + " # calculate indicator\n", + " out = indicator(\n", + " ds=ds,\n", + " freq='YS',\n", + " )\n", + " else: \n", + " out = indicator(\n", + " ds=ds,\n", + " freq='YS',\n", + " resample_before_rl=False\n", + " )\n", + " \n", + " out = out.chunk({'time': -1, 'lat': 50, 'lon': 50})\n", + " ds_out[out.name] = out\n", + "\n", + " output_path = Path(f'heatwave_output_ex2/{out.name}_1991-2010.zarr')\n", + " output_path.parent.mkdir(parents=True, exist_ok=True)\n", + "\n", + " if not output_path.exists():\n", + " # save to Zarr, triggering computation immediately\n", + " ds_out.to_zarr(output_path)\n", + " \n", + " # fetch memory usage from all workers and display the total usage\n", + " worker_memory = client.run(lambda: psutil.Process().memory_info().rss)\n", + " total_memory = sum(worker_memory.values())\n", + " print(f\"Total memory usage across all workers: {total_memory / 1e9:.2f} GB\")\n", + "\n", + "end_time = time.time()\n", + "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" + ] + }, + { + "cell_type": "markdown", + "id": "3849655b-5c02-444a-a5ce-f3e788f5242f", + "metadata": {}, + "source": [ + "Even with the sequential computation approach, there may be scenarios where the heatwave indicator data is still too large to write in a single step. In such cases, an effective strategy is to split the dataset into smaller, manageable spatial chunks, which would allow for more efficient processing and data writing.\n", + "\n", + "For example, we can calculate the heatwave indicators over the entire dataset spanning North America, and then divide the results into smaller latitude bins (e.g., groups of 50 latitudes each). Each bin can then be processed and saved individually to a temporary `.zarr` file using Dask. Once all bins are saved, they can be merged back into a single dataset and written out as a final `.zarr` file. This method distributes the data processing load and helps minimize the strain on system resources." + ] + }, + { + "cell_type": "code", + "execution_count": 7, + "id": "8b6d03a4-0e9e-4c6f-a74c-fbe77b692de3", + "metadata": { + "tags": [] + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 800, lon: 1700, time: 7305)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
+       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
+       "  * time     (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n",
+       "Data variables:\n",
+       "    tas      (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    pr       (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "Attributes: (12/26)\n",
+       "    Conventions:          CF-1.8\n",
+       "    cell_methods:         time: mean (interval: 1 day)\n",
+       "    data_specs_version:   00.00.07\n",
+       "    domain:               NAM\n",
+       "    format:               netcdf\n",
+       "    frequency:            day\n",
+       "    ...                   ...\n",
+       "    dataset_description:  https://www.ecmwf.int/en/era5-land\n",
+       "    license_type:         permissive\n",
+       "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
+       "    attribution:          Contains modified Copernicus Climate Change Service...\n",
+       "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
+       "    doi:                  https://doi.org/10.24381/cds.e2161bac
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 800, lon: 1700, time: 7305)\n", + "Coordinates:\n", + " * lat (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n", + " * lon (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", + " * time (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n", + "Data variables:\n", + " tas (time, lat, lon) float32 dask.array\n", + " tasmin (time, lat, lon) float32 dask.array\n", + " tasmax (time, lat, lon) float32 dask.array\n", + " pr (time, lat, lon) float32 dask.array\n", + " prsn (time, lat, lon) float32 dask.array\n", + "Attributes: (12/26)\n", + " Conventions: CF-1.8\n", + " cell_methods: time: mean (interval: 1 day)\n", + " data_specs_version: 00.00.07\n", + " domain: NAM\n", + " format: netcdf\n", + " frequency: day\n", + " ... ...\n", + " dataset_description: https://www.ecmwf.int/en/era5-land\n", + " license_type: permissive\n", + " license: Please acknowledge the use of ERA5-Land as stated i...\n", + " attribution: Contains modified Copernicus Climate Change Service...\n", + " citation: Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n", + " doi: https://doi.org/10.24381/cds.e2161bac" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stderr", + "output_type": "stream", + "text": [ + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n" + ] + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "16\n" + ] + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-1bcb90f2-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

a25186c9

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-4be94dbf-b42a-4daf-a70a-0dff427ccc27

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:45491\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41945\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38317/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38255\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-47ho7fnf\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42927\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45877/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33283\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-cdjd977d\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37221\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46087/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43419\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ueww31un\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41697\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41243/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45223\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ftfw8asm\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42575\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37443/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45299\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-oh2nkoju\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-42654b91-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

4b766fd0

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-a818a105-5d82-42cb-bfa3-51b24d184661

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:41401\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33835\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42475/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43829\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-_qf211p4\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34215\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39863/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40135\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-wgv37dhm\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39901\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35889/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45729\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ybqbhxnb\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43129\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42627/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46497\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-3pj_3tm0\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36897\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/32979/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46857\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-mkaagify\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-672b22c5-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

3cc03a4d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-acae489b-be57-446a-9d6c-7f6f84919e8a

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:44255\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36863\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39983/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33469\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-0gmeme2p\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39565\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38081/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35181\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-esv8hmna\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37125\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36443/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38937\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-mb28texs\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39777\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45801/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46273\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ohe3x11j\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42569\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38831/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45705\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-pv8krp2x\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-894f490a-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

f107137b

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-de49da15-6918-4cae-b21e-581d195f4b2f

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:41437\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35899\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34833/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43101\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-t4g8atac\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34295\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37449/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39121\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-467etyu_\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37243\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41379/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:32957\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-gkp39oz4\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42843\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34155/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38595\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-dp6kalj8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38545\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45427/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44981\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-rppx1rb7\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-aaaf641e-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

2c9b7384

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-0b8d2bc4-af75-4753-b51e-b1fdef4f5818

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:42553\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43635\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37313/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37809\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-objo06zp\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39619\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33349/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37067\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ud53_dvu\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39659\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/32877/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45161\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-3tnyzn8e\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46067\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34459/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33461\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-hhks3maw\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34985\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39727/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38463\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-1dkx_mxi\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-cd4ce9e0-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

8ac98491

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-80a2de8b-5497-4faf-89b1-802abdd58fa1

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:41775\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37707\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42265/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41669\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-um0qbdsh\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44805\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41513/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44251\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-qh7dw_j3\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38797\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34133/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38015\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-qx9kz_o4\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34511\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34113/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36979\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-nyothb0b\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40841\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40431/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42115\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-0ln52f8z\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-f06f7a71-a07b-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

1168d538

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-5ab79e57-1690-45e8-8b11-86b6e953f532

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:43667\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39145\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45993/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37973\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-cm3rgr1o\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33427\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46805/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45439\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-qh68ldc8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36291\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40837/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41749\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-muf6o3tk\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43881\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34945/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34323\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-f8a_2fbl\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33979\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33639/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37751\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-dwjn8ttz\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-137d24f5-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

a12bf0dd

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-2ea0a55f-8fd7-4588-8dd3-8ce131e54fa4

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:40041\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34055\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37439/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36853\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-y9ggk3dh\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42327\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44773/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39373\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-_juhm3se\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37933\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44399/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42757\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-a1vz5bcq\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35707\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41345/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40085\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-8jhu4y3m\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40773\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33107/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46409\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-wde0ov6y\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-36ad0ebe-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

84cf3107

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-17ea335a-43fb-4a2e-bb3d-6ce8b140608a

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:39591\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42969\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39093/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37833\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-l9uccjwj\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:45775\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45159/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37221\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-o8c0metp\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44013\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35365/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37401\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-dk7jkrs9\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36761\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40203/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33309\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-exqapw5t\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43859\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44149/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43913\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-dqd_ntcq\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-5ac35be2-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

a9793067

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-80aba4f6-1884-441c-9d7b-5b2fdad91104

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:32953\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37163\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44923/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35603\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-jz25ksln\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34217\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43449/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39227\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-xgw_a2cu\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44865\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43447/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34519\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-u_b9974t\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46315\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35003/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38549\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-o6zidv3x\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39753\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40707/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33365\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-5hkvykj1\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-7c150a80-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

68f9665a

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-c38ba09d-4514-4152-8104-3c08ad30ce50

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:36583\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41771\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36345/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33893\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-eea239r7\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43559\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39575/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34517\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-tr5vqxx9\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42553\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37799/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44067\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-0e5c2xwm\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43477\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33857/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39257\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-44uavz73\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41767\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40727/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42699\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-tenz6i59\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-9f48dbef-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

b5c277fc

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-d74b109a-fa94-4340-8a43-b82ceb8c0222

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:34711\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44545\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42045/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34951\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-42vwhg79\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37457\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37535/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44951\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-69fbj5qg\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39679\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39105/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42991\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-5f4ecgme\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35173\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34929/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42473\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-rstxqpbz\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44853\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44903/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34203\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-od98zcu4\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-c2adf2c4-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

03d37d5d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-e27a9240-11c2-4530-aa67-9ec506888cb2

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:39651\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38789\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36667/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45391\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-e0ydzkmr\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36407\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36221/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44583\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-0jxqlon5\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38019\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36979/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45359\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-jngykuh0\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40607\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46055/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34331\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-stdvpml4\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38999\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41701/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36441\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-zhq0yyjd\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-e750b721-a07c-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

af8c328c

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-1f28e14f-421f-403f-a18d-70fda975f6ac

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:43537\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33715\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/32857/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34633\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-w_scz39z\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34147\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35319/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39519\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-xc6ffw1j\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36245\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40043/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46657\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-e_1x5h3_\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46291\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40079/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44683\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-i3xdz2iq\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43243\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38875/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33357\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-5i79xk03\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-0a37bbd4-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

8f5a213d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-472408cd-3835-47b6-83be-68eab684c975

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:39181\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44435\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43493/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34947\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-y_qi_0go\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43637\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42059/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46111\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-w7g3vi5y\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40875\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33779/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45029\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-gifpfdd7\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44691\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43081/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42721\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-f2lmutvw\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42711\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37025/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41969\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-3s099v37\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-2e11435a-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

5c2066ad

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-4dc8b779-d4f7-4872-b6d4-f094f48bf69b

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:44939\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43557\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33235/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33915\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-rkckkevp\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39623\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41123/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41823\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-zq_qd1fi\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33927\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39905/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:32823\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-b11syynp\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35143\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41713/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37797\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-a4dwtkp_\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46045\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35193/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40521\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-mo8ff252\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-2e11435a-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

5c2066ad

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 0\n", + "
\n", + " Total threads: 0\n", + " \n", + " Total memory: 0 B\n", + "
Status: closedUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-4dc8b779-d4f7-4872-b6d4-f094f48bf69b

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:44939\n", + " \n", + " Workers: 0\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 0\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 0 B\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Total computation time: 948.99 seconds\n" + ] + } + ], + "source": [ + "import xarray as xr\n", + "from xclim import atmos\n", + "from pathlib import Path\n", + "from xscen.io import save_to_zarr\n", + "from tempfile import TemporaryDirectory\n", + "from dask.distributed import Client\n", + "import shutil\n", + "import time\n", + "\n", + "start_time = time.time()\n", + "\n", + "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", + "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + "\n", + "# we're interested in the 1991-2010 period for our heatwave analysis\n", + "ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", + "\n", + "display(ds)\n", + "\n", + "# create an empty output dataset\n", + "dsout = xr.Dataset(attrs=ds.attrs)\n", + "\n", + "# xclim calculation on entire dataset\n", + "out = atmos.heat_wave_total_length(ds=ds, freq='YS', resample_before_rl=False) # Heat_wave calculation where heat-waves could cross calendar years\n", + "dsout[out.name] = out\n", + "\n", + "outzarr = Path('output').joinpath('heat_wave_total_length_spatial_example.zarr')\n", + "\n", + "if not outzarr.exists():\n", + " \n", + " # if data is too big to write in single step - make individual datasets by binning the `lat` dim (approx n= 50 latitudes at a time)\n", + " grp_dim = 'lat'\n", + " bins = round(len(dsout.lat)/50) \n", + " _, datasets = zip(*dsout.groupby_bins(grp_dim, bins))\n", + " print(len(datasets))\n", + " assert sum([len(d[grp_dim]) for d in datasets]) == len(dsout[grp_dim])\n", + " \n", + " # export each chunk of 50-latitudes to a temporary location\n", + " with TemporaryDirectory(prefix='output', dir='.') as outtmp:\n", + " for ii, dds in enumerate(datasets):\n", + " dds = dds.chunk(time=-1, lon=50, lat=50)\n", + " filename = Path(outtmp).joinpath(f\"{ii}.zarr\")\n", + " with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client: \n", + " display(client)\n", + " save_to_zarr(\n", + " ds=dds,\n", + " filename=filename,\n", + " mode='o',\n", + " )\n", + " \n", + " # reassamble pieces and export joined\n", + " inzarrs = sorted(list(filename.parent.glob(f'*.zarr')))\n", + " \n", + " # open the files as a combined multi-file dataset\n", + " ds = xr.open_mfdataset(inzarrs, engine='zarr', decode_timedelta=False) \n", + " \n", + " # define the final chunking configuration\n", + " final_chunks = dict(time=-1, lon=50, lat=50) \n", + " \n", + " # save the final combined dataset\n", + " tmpzarr = Path(outtmp).joinpath(outzarr.name)\n", + " with Client(n_workers=10) as c:\n", + " display(client)\n", + " save_to_zarr(\n", + " ds=ds.chunk(final_chunks),\n", + " filename=tmpzarr,\n", + " mode=\"o\",\n", + " )\n", + " # move the final combined file to the output location\n", + " outzarr.parent.mkdir(exist_ok=True, parents=True)\n", + " shutil.move(tmpzarr, outzarr)\n", + " \n", + "end_time = time.time()\n", + "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" + ] + }, + { + "cell_type": "markdown", + "id": "369db0c5-ada0-45ce-b0fc-58a8500761e4", + "metadata": {}, + "source": [ + "Alternatively, if our goal is to calculate heatwave indicators over a longer time period but for a smaller spatial extent, we can manage memory more efficiently by splitting the data along the time dimension, such as into 5-year intervals." + ] + }, + { + "cell_type": "code", + "execution_count": 9, + "id": "e4b3763e-9174-47dc-9335-703d71a80cf6", + "metadata": { + "tags": [] + }, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 230, lon: 0, time: 25933)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 42.1 42.2 42.3 42.4 42.5 ... 64.6 64.7 64.8 64.9 65.0\n",
+       "  * lon      (lon) float32 \n",
+       "  * time     (time) datetime64[ns] 1950-01-01 1950-01-02 ... 2020-12-31\n",
+       "Data variables:\n",
+       "    tas      (time, lat, lon) float32 dask.array<chunksize=(366, 29, 0), meta=np.ndarray>\n",
+       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(366, 29, 0), meta=np.ndarray>\n",
+       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(366, 29, 0), meta=np.ndarray>\n",
+       "    pr       (time, lat, lon) float32 dask.array<chunksize=(366, 29, 0), meta=np.ndarray>\n",
+       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(366, 29, 0), meta=np.ndarray>\n",
+       "Attributes: (12/26)\n",
+       "    Conventions:          CF-1.8\n",
+       "    cell_methods:         time: mean (interval: 1 day)\n",
+       "    data_specs_version:   00.00.07\n",
+       "    domain:               NAM\n",
+       "    format:               netcdf\n",
+       "    frequency:            day\n",
+       "    ...                   ...\n",
+       "    dataset_description:  https://www.ecmwf.int/en/era5-land\n",
+       "    license_type:         permissive\n",
+       "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
+       "    attribution:          Contains modified Copernicus Climate Change Service...\n",
+       "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
+       "    doi:                  https://doi.org/10.24381/cds.e2161bac
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 230, lon: 0, time: 25933)\n", + "Coordinates:\n", + " * lat (lat) float32 42.1 42.2 42.3 42.4 42.5 ... 64.6 64.7 64.8 64.9 65.0\n", + " * lon (lon) float32 \n", + " * time (time) datetime64[ns] 1950-01-01 1950-01-02 ... 2020-12-31\n", + "Data variables:\n", + " tas (time, lat, lon) float32 dask.array\n", + " tasmin (time, lat, lon) float32 dask.array\n", + " tasmax (time, lat, lon) float32 dask.array\n", + " pr (time, lat, lon) float32 dask.array\n", + " prsn (time, lat, lon) float32 dask.array\n", + "Attributes: (12/26)\n", + " Conventions: CF-1.8\n", + " cell_methods: time: mean (interval: 1 day)\n", + " data_specs_version: 00.00.07\n", + " domain: NAM\n", + " format: netcdf\n", + " frequency: day\n", + " ... ...\n", + " dataset_description: https://www.ecmwf.int/en/era5-land\n", + " license_type: permissive\n", + " license: Please acknowledge the use of ERA5-Land as stated i...\n", + " attribution: Contains modified Copernicus Climate Change Service...\n", + " citation: Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n", + " doi: https://doi.org/10.24381/cds.e2161bac" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stderr", + "output_type": "stream", + "text": [ + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n" + ] + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-8e5678bb-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

96c53251

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-23a044ab-06a5-4c05-82eb-7080f2c5388c

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:39115\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40881\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40105/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46051\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-u7uopkut\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41259\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44237/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33695\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-2ckpl417\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43159\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37411/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43153\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-t2mgpnfb\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42005\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38491/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45529\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-rbd8e7xo\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42985\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42881/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35457\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-x5k56gpa\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-8f4c7754-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

f4c7750c

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-40a5f00a-ff20-4c68-96f5-6aa563cc7ee0

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:40359\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41037\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34971/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45939\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-rx_wn98o\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34899\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42541/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44755\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-rz_c7vsw\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34533\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38619/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44527\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-9rqivyz9\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35549\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44097/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36785\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-bksoxh8p\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33039\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34555/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35171\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-wofzgxpt\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-903b3e74-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

8c6eb6d9

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-91c151e8-8fa9-457b-8684-30497815685b

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:34389\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33999\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45475/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43255\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-6xb5l87b\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46809\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44377/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34409\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-2oklsv8c\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:32773\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46145/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46155\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-a360xk9l\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35551\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39377/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46701\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-4465nbxz\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46805\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44371/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45485\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-4c6wuy2b\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-9119f10f-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

876a6276

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-46c3fb77-ca74-4f33-87f3-3edb06de0a3d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:46517\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37713\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39439/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33427\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-2n2ol863\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38357\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37017/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36203\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-8vq6cga4\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33067\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45881/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45941\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-9cg5391g\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38405\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37553/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43405\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-1c2s90kj\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:45999\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38569/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40627\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-o3r7voxa\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-91fb43aa-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

fc83d812

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-e1bf9663-17d5-43cc-96f3-c0545d58066b

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:40843\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43123\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38007/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34497\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-dprgh5kd\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40575\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34503/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43307\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-gsrx4wt8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39365\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39617/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45969\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-5pdzkcm3\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46263\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38675/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38183\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-0qsswbht\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36907\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46459/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34641\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-hgbwljvo\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-92db8201-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

23ffef01

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-dd086852-29a1-42cf-9f1e-0aaebb3251d2

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:44053\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38151\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36847/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39641\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-oe3s5g7z\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36965\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45995/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42305\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ym47_hxz\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44501\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36835/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35067\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-wyd898sp\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33577\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39017/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40631\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-knu_k_32\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38239\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35585/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46505\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ac5ftesv\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-93b65bfb-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

e82ab823

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-a01335c0-fe3f-4092-97fb-a33797b61703

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:46365\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37769\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42867/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42457\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-055dnevn\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:46665\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41861/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35109\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-vqkyadfb\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36619\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41121/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36507\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-8oaklrju\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40575\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43631/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41191\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-fje5lbwu\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42695\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35717/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39777\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-qfuq_oql\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-94957fd7-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

86b0dbee

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-7b0e7815-48a4-4f20-a629-7ee1ed7176af

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:41319\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42025\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40643/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42847\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ge870ntq\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39333\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39185/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45451\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ens9unm1\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42265\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35585/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42021\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-zl0i5d5a\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44407\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37151/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45903\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-hygsg5qs\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43445\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37897/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37177\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-eau89iux\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-956f3a71-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

bfa77138

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-7b4c1b38-b105-4a4a-a394-f7c02a1e52ef

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:36861\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33113\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33143/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33229\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-jt9xlgku\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38481\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46709/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38917\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-jeakac77\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:38093\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44057/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37537\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-9ji0n5x7\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41739\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/43521/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40155\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-wh48cuaa\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44749\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37035/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33569\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-6m9cexi9\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-9648a244-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

4b5d3876

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-b2316176-eca6-4372-b220-4e542b353b0e

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:36045\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33771\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46163/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44713\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-e566cfzc\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44941\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41469/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36991\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-zsso_u8o\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42815\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45421/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38229\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-z_bxg7n5\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44689\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41077/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42151\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-b_5042x0\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:45411\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38891/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38519\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-5x1cqdij\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-97226655-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

ab9a4b66

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-8cdf5451-8e3f-4697-b55f-15e9c7bf8654

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:40293\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44579\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37893/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34965\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ql1q64a3\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41923\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33377/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38875\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-8tufgpka\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40889\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38649/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44235\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-sdm9f01e\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34733\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35445/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41293\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-22bbqnnt\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:44781\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35755/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40983\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-w_gvh68r\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-97fae801-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

da925a80

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-56ad65e2-95b1-4714-9471-63d06918bd69

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:36863\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40805\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39251/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34963\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-gm9ujw3j\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42469\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33783/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:37709\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-1cexz6mn\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:35139\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38813/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:42915\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-l93n9jub\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:39923\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36929/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36843\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ubc5z5ri\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36117\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34713/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:38303\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-9rabi6m9\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-98d41c9e-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

bac0f35f

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-92ac6b01-c73d-4196-8a02-e0b64b0d5ff0

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:36105\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36909\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40113/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:43797\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-whx2hktv\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33889\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40051/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45851\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-877oeuhu\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43045\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34465/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41043\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-zzt14i96\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:45599\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36933/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45827\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-jqsmj9r0\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37393\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37595/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36293\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-hgzyvrc8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-99aeea73-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

aa130ea9

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-4c5eb92f-47bf-4ab9-92bf-7359f062ad10

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:46673\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34345\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42573/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:34129\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-3x2jeb6g\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:37449\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39821/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:39101\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-5nf2v4h8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:42907\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/42999/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:41419\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-qq4fpijy\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:33587\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40789/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:44683\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-iiqj49c8\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:43093\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39301/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:35683\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-hk7ppjxh\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-9a8aed0a-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

fa2b84b9

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 5\n", + "
\n", + " Total threads: 10\n", + " \n", + " Total memory: 18.63 GiB\n", + "
Status: runningUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-fcb73a62-09a4-476b-85fa-13a44a7f300d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:43183\n", + " \n", + " Workers: 5\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 10\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 18.63 GiB\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 0

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:45675\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38773/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:36913\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-f0ot8t_p\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 1

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:40271\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/39287/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:33835\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-blndtchc\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 2

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:41569\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38447/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:40095\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-ggn8tjf2\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 3

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:36161\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36179/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:45287\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-sbwf62do\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "

Worker: 4

\n", + "
\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + "\n", + " \n", + "\n", + "
\n", + " Comm: tcp://127.0.0.1:34801\n", + " \n", + " Total threads: 2\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36899/status\n", + " \n", + " Memory: 3.73 GiB\n", + "
\n", + " Nanny: tcp://127.0.0.1:46847\n", + "
\n", + " Local directory: /tmp/dask-scratch-space/worker-pgbogqdo\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "data": { + "text/html": [ + "
\n", + "
\n", + "
\n", + "

Client

\n", + "

Client-9a8aed0a-a07d-11ef-9e54-0242ac12000c

\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "
Connection method: Cluster objectCluster type: distributed.LocalCluster
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + "
\n", + "\n", + " \n", + " \n", + " \n", + "\n", + " \n", + "
\n", + "

Cluster Info

\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

LocalCluster

\n", + "

fa2b84b9

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "\n", + "\n", + " \n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Workers: 0\n", + "
\n", + " Total threads: 0\n", + " \n", + " Total memory: 0 B\n", + "
Status: closedUsing processes: True
\n", + "\n", + "
\n", + " \n", + "

Scheduler Info

\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + "

Scheduler

\n", + "

Scheduler-fcb73a62-09a4-476b-85fa-13a44a7f300d

\n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + " \n", + "
\n", + " Comm: tcp://127.0.0.1:43183\n", + " \n", + " Workers: 0\n", + "
\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/8787/status\n", + " \n", + " Total threads: 0\n", + "
\n", + " Started: Just now\n", + " \n", + " Total memory: 0 B\n", + "
\n", + "
\n", + "
\n", + "\n", + "
\n", + " \n", + "

Workers

\n", + "
\n", + "\n", + " \n", + "\n", + "
\n", + "
\n", + "\n", + "
\n", + "
\n", + "
\n", + "
\n", + " \n", + "\n", + "
\n", + "
" + ], + "text/plain": [ + "" + ] + }, + "metadata": {}, + "output_type": "display_data" + }, + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Total computation time: 24.60 seconds\n" + ] + } + ], + "source": [ + "import xarray as xr\n", + "from xclim import atmos\n", + "from pathlib import Path\n", + "from xscen.io import save_to_zarr\n", + "from tempfile import TemporaryDirectory\n", + "from dask.distributed import Client\n", + "import shutil\n", + "import pandas as pd\n", + "import time\n", + "\n", + "start_time = time.time()\n", + "\n", + "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", + "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + "\n", + "# this time, let's focus our analysis on a smaller region and a longer time period \n", + "ds = ds.sel(time=slice('1950-01-01', '2020-12-31'), lat=slice(42.1, 65.0), lon=slice(-12.5, -20.0))\n", + "\n", + "display(ds)\n", + "\n", + "dsout = xr.Dataset(attrs=ds.attrs)\n", + "\n", + "out = atmos.heat_wave_total_length(ds=ds, freq='YS', resample_before_rl=False)\n", + "dsout[out.name] = out\n", + "\n", + "outzarr = Path('output').joinpath('heat_wave_total_length_temporal_example.zarr')\n", + "\n", + "if not outzarr.exists():\n", + " \n", + " # resample into 5-year intervals\n", + " _, datasets = zip(*dsout.resample(time='5YS'))\n", + "\n", + " # export each 5-year chunk to a temporary location\n", + " with TemporaryDirectory(prefix='output', dir='.') as outtmp:\n", + " for ii, dds in enumerate(datasets):\n", + " dds = dds.chunk(dict(time=-1, lon=50, lat=50)) \n", + " filename = Path(outtmp).joinpath(f\"{ii}.zarr\")\n", + " with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client: \n", + " display(client)\n", + " save_to_zarr(\n", + " ds=dds,\n", + " filename=filename,\n", + " mode='o',\n", + " )\n", + " \n", + " # reassemble the 5-year chunks into a single dataset\n", + " inzarrs = sorted(list(filename.parent.glob(f'*.zarr')))\n", + " \n", + " # open the files as a combined multi-file dataset\n", + " ds = xr.open_mfdataset(inzarrs, engine='zarr', decode_timedelta=False)\n", + " \n", + " # define the final chunking configuration\n", + " final_chunks = dict(time=12*50, lon=50, lat=50)\n", + " \n", + " # save the final combined dataset\n", + " tmpzarr = Path(outtmp).joinpath(outzarr.name)\n", + " with Client(n_workers=10) as c:\n", + " display(client)\n", + " save_to_zarr(\n", + " ds=ds.chunk(final_chunks),\n", + " filename=tmpzarr,\n", + " mode=\"o\",\n", + " )\n", + " \n", + " # move the final combined file to the output location\n", + " outzarr.parent.mkdir(exist_ok=True, parents=True)\n", + " shutil.move(tmpzarr, outzarr)\n", + " \n", + "end_time = time.time()\n", + "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "id": "971d3b5d-3c2b-4544-8fa5-20f692c39053", + "metadata": {}, + "outputs": [], + "source": [] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.11.10" + } + }, + "nbformat": 4, + "nbformat_minor": 5 +} From facb87e47203c9bf73ce5640b743f068c4a72140 Mon Sep 17 00:00:00 2001 From: "pre-commit-ci[bot]" <66853113+pre-commit-ci[bot]@users.noreply.github.com> Date: Wed, 20 Nov 2024 17:09:21 +0000 Subject: [PATCH 02/10] [pre-commit.ci] auto fixes from pre-commit.com hooks for more information, see https://pre-commit.ci --- docs/source/notebooks/FAQ_dask_parallel.ipynb | 288 ++++++++++-------- 1 file changed, 153 insertions(+), 135 deletions(-) diff --git a/docs/source/notebooks/FAQ_dask_parallel.ipynb b/docs/source/notebooks/FAQ_dask_parallel.ipynb index 7778a9f..ad0858f 100644 --- a/docs/source/notebooks/FAQ_dask_parallel.ipynb +++ b/docs/source/notebooks/FAQ_dask_parallel.ipynb @@ -2,7 +2,7 @@ "cells": [ { "cell_type": "markdown", - "id": "b898ae93-3234-44b1-a88f-5aa0c1d8f6b9", + "id": "0", "metadata": {}, "source": [ "# Processing Large Climate Datasets with Dask and Xarray\n", @@ -15,7 +15,7 @@ { "cell_type": "code", "execution_count": 12, - "id": "9f12da60-e3df-4172-a9cb-e014801ba7c9", + "id": "1", "metadata": { "tags": [] }, @@ -29,7 +29,7 @@ } ], "source": [ - "import xarray as xr \n", + "import xarray as xr\n", "\n", "# open the daily ERA5-land dataset as an example\n", "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", @@ -38,12 +38,12 @@ "ds = xr.open_dataset(url, decode_times=False, chunks={})\n", "\n", "# `._ChunkSizes` attribute shows the chunk structure of the variable\n", - "print(ds['tas']._ChunkSizes)" + "print(ds[\"tas\"]._ChunkSizes)" ] }, { "cell_type": "markdown", - "id": "1ab719a3-3de0-4344-aa9a-2836d4a099ed", + "id": "2", "metadata": { "tags": [] }, @@ -58,7 +58,7 @@ { "cell_type": "code", "execution_count": 2, - "id": "4d646c97-eda5-4070-aa54-19ce91d908dc", + "id": "3", "metadata": {}, "outputs": [ { @@ -72,7 +72,7 @@ ], "source": [ "# open dataset with unaligned chunks\n", - "ds = xr.open_dataset(url, chunks={'time': 200, 'lat': 30, 'lon': 30})\n", + "ds = xr.open_dataset(url, chunks={\"time\": 200, \"lat\": 30, \"lon\": 30})\n", "\n", "# resample the 'tas' variable to yearly means\n", "%time tas_resampled = ds['tas'].sel(time=slice('1981-01-01', '1981-12-31')).resample(time='YS').mean().compute()" @@ -81,7 +81,7 @@ { "cell_type": "code", "execution_count": 3, - "id": "a449349b-a339-4d96-aa33-12a97d9ec51e", + "id": "4", "metadata": { "tags": [] }, @@ -97,7 +97,7 @@ ], "source": [ "# open dataset with aligned chunks\n", - "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + "ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", "\n", "# resample the 'tas' variable to yearly means\n", "%time tas_resampled = ds['tas'].sel(time=slice('1981-01-01', '1981-12-31')).resample(time='YS').mean().compute()" @@ -105,7 +105,7 @@ }, { "cell_type": "markdown", - "id": "47a56ac9-2d80-43ef-a283-12a2e2d42bfd", + "id": "5", "metadata": {}, "source": [ "Aligning the chunks reduced the computation time from approximately 3 minutes to 1 minute, by allowing Dask to load only the required data and avoid excessive memory usage and I/O operations. " @@ -113,7 +113,7 @@ }, { "cell_type": "markdown", - "id": "2422a144-7765-4d29-9703-93d0c54a4c02", + "id": "6", "metadata": {}, "source": [ "### How to do parallel processing with Dask Distributed Client? \n", @@ -131,7 +131,7 @@ { "cell_type": "code", "execution_count": 11, - "id": "73765dbb-3bf7-4814-b44e-6246713275de", + "id": "7", "metadata": { "tags": [] }, @@ -1689,33 +1689,37 @@ } ], "source": [ - "import xarray as xr\n", + "import time\n", "from pathlib import Path\n", - "from dask.distributed import Client\n", + "\n", + "import psutil\n", + "import xarray as xr\n", "from dask import compute\n", - "import psutil \n", - "import time\n", + "from dask.distributed import Client\n", "\n", "start_time = time.time()\n", "\n", - "# function to compute the yearly mean for each variable \n", + "# function to compute the yearly mean for each variable\n", + "\n", + "\n", "def compute_yearly_mean(ds, var_name):\n", " var = ds[var_name]\n", - " yearly_mean = var.resample(time='YS').mean()\n", + " yearly_mean = var.resample(time=\"YS\").mean()\n", " return yearly_mean\n", "\n", + "\n", "# set up Dask client within a context manager\n", - "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit=\"4GB\") as client:\n", " # display the Dask Dashboard\n", " display(client)\n", "\n", " # open the dataset with on-disk chunking structure\n", " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon':50}) \n", + " ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", " display(ds)\n", "\n", " # select data for the year 1981\n", - " ds = ds.sel(time=slice('1981-01-01', '1981-12-31'))\n", + " ds = ds.sel(time=slice(\"1981-01-01\", \"1981-12-31\"))\n", "\n", " variables = list(ds.data_vars)\n", "\n", @@ -1726,7 +1730,7 @@ " if not output_path.exists():\n", " yearly_mean = compute_yearly_mean(ds, var_name)\n", " # save to Zarr with compute=False to get a delayed task object\n", - " delayed_task = yearly_mean.to_zarr(output_path, mode='w', compute=False)\n", + " delayed_task = yearly_mean.to_zarr(output_path, mode=\"w\", compute=False)\n", " tasks.append(delayed_task)\n", "\n", " # trigger the execution of all delayed tasks\n", @@ -1739,12 +1743,12 @@ "\n", "end_time = time.time()\n", "elapsed_time = end_time - start_time\n", - "print(f'Total computation time: {elapsed_time:.2f} seconds')" + "print(f\"Total computation time: {elapsed_time:.2f} seconds\")" ] }, { "cell_type": "markdown", - "id": "e9cee3ca-a916-4d16-aab8-75dc3692217c", + "id": "8", "metadata": {}, "source": [ "Let’s look at a more complex analysis involving heatwave indicators. Here, I calculate two climate indicators, `heat_wave_total_length` and `heat_wave_frequency`, using the `atmos` submodule from the `xclim` library. Both indicators rely on the same input data (`tasmin` and `tasmax`), so I create a pipeline of delayed tasks, which minimizes I/O operations by keeping the data in memory until both indicators are calculated and saved." @@ -1753,7 +1757,7 @@ { "cell_type": "code", "execution_count": 18, - "id": "c24f1efe-65b8-48af-9090-ae516e6840d6", + "id": "9", "metadata": { "tags": [] }, @@ -3512,30 +3516,35 @@ } ], "source": [ - "from pathlib import Path\n", - "from dask.distributed import Client\n", "import time\n", - "from xclim import atmos\n", + "from pathlib import Path\n", + "\n", + "import psutil\n", "import xarray as xr\n", "from dask import compute\n", - "import psutil \n", + "from dask.distributed import Client\n", + "from xclim import atmos\n", "\n", "start_time = time.time()\n", "\n", - "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit=\"4GB\") as client:\n", " display(client)\n", - " \n", + "\n", " # load data using on-disk chunk sizes\n", " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", - " \n", + " ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", + "\n", " # we're interested in the 1991-2010 period for our heatwave analysis\n", - " ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", - " \n", + " ds = ds.sel(time=slice(\"1991-01-01\", \"2010-12-31\"))\n", + "\n", " display(ds)\n", "\n", " # list of heatwave indicator functions\n", - " indicators = [atmos.heat_wave_total_length, atmos.heat_wave_frequency, atmos._precip.precip_accumulation] \n", + " indicators = [\n", + " atmos.heat_wave_total_length,\n", + " atmos.heat_wave_frequency,\n", + " atmos._precip.precip_accumulation,\n", + " ]\n", " tasks = []\n", " for indicator in indicators:\n", " ds_out = xr.Dataset(attrs=ds.attrs) # create a new dataset for each indicator\n", @@ -3544,41 +3553,37 @@ " # calculate indicator\n", " out = indicator(\n", " ds=ds,\n", - " freq='YS',\n", - " )\n", - " else: \n", - " out = indicator(\n", - " ds=ds,\n", - " freq='YS',\n", - " resample_before_rl=False\n", + " freq=\"YS\",\n", " )\n", - " \n", - " out = out.chunk({'time': -1, 'lat': 50, 'lon': 50})\n", + " else:\n", + " out = indicator(ds=ds, freq=\"YS\", resample_before_rl=False)\n", + "\n", + " out = out.chunk({\"time\": -1, \"lat\": 50, \"lon\": 50})\n", " ds_out[out.name] = out\n", "\n", - " output_path = Path(f'heatwave_output_ex1/{out.name}_1991-2010.zarr')\n", + " output_path = Path(f\"heatwave_output_ex1/{out.name}_1991-2010.zarr\")\n", " output_path.parent.mkdir(parents=True, exist_ok=True)\n", "\n", " if not output_path.exists():\n", - " # save to Zarr with compute=False to get a delayed task object \n", - " delayed_task = ds_out.to_zarr(output_path, mode='w', compute=False)\n", + " # save to Zarr with compute=False to get a delayed task object\n", + " delayed_task = ds_out.to_zarr(output_path, mode=\"w\", compute=False)\n", " tasks.append(delayed_task)\n", "\n", - " # trigger computation \n", + " # trigger computation\n", " compute(*tasks)\n", - " \n", + "\n", " # fetch memory usage from all workers and display the total usage\n", " worker_memory = client.run(lambda: psutil.Process().memory_info().rss)\n", " total_memory = sum(worker_memory.values())\n", " print(f\"Total memory usage across all workers: {total_memory / 1e9:.2f} GB\")\n", - " \n", + "\n", "end_time = time.time()\n", "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" ] }, { "cell_type": "markdown", - "id": "12d29293-ad43-44f5-8d4a-1ecdbf8574ce", + "id": "10", "metadata": {}, "source": [ "### What can we do when we have large task graphs / large memory footprint? \n", @@ -3590,7 +3595,7 @@ { "cell_type": "code", "execution_count": 22, - "id": "48e59f9d-3d12-4119-9f3c-0cd694ccd871", + "id": "11", "metadata": {}, "outputs": [ { @@ -5367,53 +5372,54 @@ } ], "source": [ + "import time\n", "from pathlib import Path\n", + "\n", + "import psutil\n", + "from dask import compute\n", "from dask.distributed import Client\n", - "import time\n", "from xclim import atmos\n", - "from dask import compute\n", - "import psutil \n", "\n", "start_time = time.time()\n", "\n", - "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit=\"4GB\") as client:\n", " display(client)\n", - " \n", + "\n", " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", - " \n", - " ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", - " \n", + " ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", + "\n", + " ds = ds.sel(time=slice(\"1991-01-01\", \"2010-12-31\"))\n", + "\n", " display(ds)\n", "\n", - " indicators = [atmos.heat_wave_total_length, atmos.heat_wave_frequency, atmos._precip.precip_accumulation] \n", + " indicators = [\n", + " atmos.heat_wave_total_length,\n", + " atmos.heat_wave_frequency,\n", + " atmos._precip.precip_accumulation,\n", + " ]\n", "\n", " for indicator in indicators:\n", - " ds_out = xr.Dataset(attrs=ds.attrs) \n", - " \n", + " ds_out = xr.Dataset(attrs=ds.attrs)\n", + "\n", " if indicator == atmos._precip.precip_accumulation:\n", " # calculate indicator\n", " out = indicator(\n", " ds=ds,\n", - " freq='YS',\n", + " freq=\"YS\",\n", " )\n", - " else: \n", - " out = indicator(\n", - " ds=ds,\n", - " freq='YS',\n", - " resample_before_rl=False\n", - " )\n", - " \n", - " out = out.chunk({'time': -1, 'lat': 50, 'lon': 50})\n", + " else:\n", + " out = indicator(ds=ds, freq=\"YS\", resample_before_rl=False)\n", + "\n", + " out = out.chunk({\"time\": -1, \"lat\": 50, \"lon\": 50})\n", " ds_out[out.name] = out\n", "\n", - " output_path = Path(f'heatwave_output_ex2/{out.name}_1991-2010.zarr')\n", + " output_path = Path(f\"heatwave_output_ex2/{out.name}_1991-2010.zarr\")\n", " output_path.parent.mkdir(parents=True, exist_ok=True)\n", "\n", " if not output_path.exists():\n", " # save to Zarr, triggering computation immediately\n", " ds_out.to_zarr(output_path)\n", - " \n", + "\n", " # fetch memory usage from all workers and display the total usage\n", " worker_memory = client.run(lambda: psutil.Process().memory_info().rss)\n", " total_memory = sum(worker_memory.values())\n", @@ -5425,7 +5431,7 @@ }, { "cell_type": "markdown", - "id": "3849655b-5c02-444a-a5ce-f3e788f5242f", + "id": "12", "metadata": {}, "source": [ "Even with the sequential computation approach, there may be scenarios where the heatwave indicator data is still too large to write in a single step. In such cases, an effective strategy is to split the dataset into smaller, manageable spatial chunks, which would allow for more efficient processing and data writing.\n", @@ -5436,7 +5442,7 @@ { "cell_type": "code", "execution_count": 7, - "id": "8b6d03a4-0e9e-4c6f-a74c-fbe77b692de3", + "id": "13", "metadata": { "tags": [] }, @@ -12713,22 +12719,23 @@ } ], "source": [ - "import xarray as xr\n", - "from xclim import atmos\n", + "import shutil\n", + "import time\n", "from pathlib import Path\n", - "from xscen.io import save_to_zarr\n", "from tempfile import TemporaryDirectory\n", + "\n", + "import xarray as xr\n", "from dask.distributed import Client\n", - "import shutil\n", - "import time\n", + "from xclim import atmos\n", + "from xscen.io import save_to_zarr\n", "\n", "start_time = time.time()\n", "\n", "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + "ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", "\n", "# we're interested in the 1991-2010 period for our heatwave analysis\n", - "ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", + "ds = ds.sel(time=slice(\"1991-01-01\", \"2010-12-31\"))\n", "\n", "display(ds)\n", "\n", @@ -12736,62 +12743,66 @@ "dsout = xr.Dataset(attrs=ds.attrs)\n", "\n", "# xclim calculation on entire dataset\n", - "out = atmos.heat_wave_total_length(ds=ds, freq='YS', resample_before_rl=False) # Heat_wave calculation where heat-waves could cross calendar years\n", + "out = atmos.heat_wave_total_length(\n", + " ds=ds, freq=\"YS\", resample_before_rl=False\n", + ") # Heat_wave calculation where heat-waves could cross calendar years\n", "dsout[out.name] = out\n", "\n", - "outzarr = Path('output').joinpath('heat_wave_total_length_spatial_example.zarr')\n", + "outzarr = Path(\"output\").joinpath(\"heat_wave_total_length_spatial_example.zarr\")\n", "\n", "if not outzarr.exists():\n", - " \n", + "\n", " # if data is too big to write in single step - make individual datasets by binning the `lat` dim (approx n= 50 latitudes at a time)\n", - " grp_dim = 'lat'\n", - " bins = round(len(dsout.lat)/50) \n", + " grp_dim = \"lat\"\n", + " bins = round(len(dsout.lat) / 50)\n", " _, datasets = zip(*dsout.groupby_bins(grp_dim, bins))\n", " print(len(datasets))\n", " assert sum([len(d[grp_dim]) for d in datasets]) == len(dsout[grp_dim])\n", - " \n", + "\n", " # export each chunk of 50-latitudes to a temporary location\n", - " with TemporaryDirectory(prefix='output', dir='.') as outtmp:\n", + " with TemporaryDirectory(prefix=\"output\", dir=\".\") as outtmp:\n", " for ii, dds in enumerate(datasets):\n", " dds = dds.chunk(time=-1, lon=50, lat=50)\n", " filename = Path(outtmp).joinpath(f\"{ii}.zarr\")\n", - " with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client: \n", + " with Client(\n", + " n_workers=5, threads_per_worker=2, memory_limit=\"4GB\"\n", + " ) as client:\n", " display(client)\n", " save_to_zarr(\n", " ds=dds,\n", " filename=filename,\n", - " mode='o',\n", + " mode=\"o\",\n", " )\n", - " \n", + "\n", " # reassamble pieces and export joined\n", - " inzarrs = sorted(list(filename.parent.glob(f'*.zarr')))\n", - " \n", + " inzarrs = sorted(list(filename.parent.glob(f\"*.zarr\")))\n", + "\n", " # open the files as a combined multi-file dataset\n", - " ds = xr.open_mfdataset(inzarrs, engine='zarr', decode_timedelta=False) \n", - " \n", + " ds = xr.open_mfdataset(inzarrs, engine=\"zarr\", decode_timedelta=False)\n", + "\n", " # define the final chunking configuration\n", - " final_chunks = dict(time=-1, lon=50, lat=50) \n", - " \n", + " final_chunks = dict(time=-1, lon=50, lat=50)\n", + "\n", " # save the final combined dataset\n", " tmpzarr = Path(outtmp).joinpath(outzarr.name)\n", " with Client(n_workers=10) as c:\n", " display(client)\n", " save_to_zarr(\n", - " ds=ds.chunk(final_chunks),\n", - " filename=tmpzarr,\n", - " mode=\"o\",\n", - " )\n", + " ds=ds.chunk(final_chunks),\n", + " filename=tmpzarr,\n", + " mode=\"o\",\n", + " )\n", " # move the final combined file to the output location\n", " outzarr.parent.mkdir(exist_ok=True, parents=True)\n", " shutil.move(tmpzarr, outzarr)\n", - " \n", + "\n", "end_time = time.time()\n", "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" ] }, { "cell_type": "markdown", - "id": "369db0c5-ada0-45ce-b0fc-58a8500761e4", + "id": "14", "metadata": {}, "source": [ "Alternatively, if our goal is to calculate heatwave indicators over a longer time period but for a smaller spatial extent, we can manage memory more efficiently by splitting the data along the time dimension, such as into 5-year intervals." @@ -12800,7 +12811,7 @@ { "cell_type": "code", "execution_count": 9, - "id": "e4b3763e-9174-47dc-9335-703d71a80cf6", + "id": "15", "metadata": { "tags": [] }, @@ -18996,60 +19007,67 @@ } ], "source": [ - "import xarray as xr\n", - "from xclim import atmos\n", + "import shutil\n", + "import time\n", "from pathlib import Path\n", - "from xscen.io import save_to_zarr\n", "from tempfile import TemporaryDirectory\n", - "from dask.distributed import Client\n", - "import shutil\n", + "\n", "import pandas as pd\n", - "import time\n", + "import xarray as xr\n", + "from dask.distributed import Client\n", + "from xclim import atmos\n", + "from xscen.io import save_to_zarr\n", "\n", "start_time = time.time()\n", "\n", "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon': 50})\n", + "ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", "\n", - "# this time, let's focus our analysis on a smaller region and a longer time period \n", - "ds = ds.sel(time=slice('1950-01-01', '2020-12-31'), lat=slice(42.1, 65.0), lon=slice(-12.5, -20.0))\n", + "# this time, let's focus our analysis on a smaller region and a longer time period\n", + "ds = ds.sel(\n", + " time=slice(\"1950-01-01\", \"2020-12-31\"),\n", + " lat=slice(42.1, 65.0),\n", + " lon=slice(-12.5, -20.0),\n", + ")\n", "\n", "display(ds)\n", "\n", "dsout = xr.Dataset(attrs=ds.attrs)\n", "\n", - "out = atmos.heat_wave_total_length(ds=ds, freq='YS', resample_before_rl=False)\n", + "out = atmos.heat_wave_total_length(ds=ds, freq=\"YS\", resample_before_rl=False)\n", "dsout[out.name] = out\n", "\n", - "outzarr = Path('output').joinpath('heat_wave_total_length_temporal_example.zarr')\n", + "outzarr = Path(\"output\").joinpath(\"heat_wave_total_length_temporal_example.zarr\")\n", "\n", "if not outzarr.exists():\n", - " \n", + "\n", " # resample into 5-year intervals\n", - " _, datasets = zip(*dsout.resample(time='5YS'))\n", + " _, datasets = zip(*dsout.resample(time=\"5YS\"))\n", "\n", " # export each 5-year chunk to a temporary location\n", - " with TemporaryDirectory(prefix='output', dir='.') as outtmp:\n", + " with TemporaryDirectory(prefix=\"output\", dir=\".\") as outtmp:\n", " for ii, dds in enumerate(datasets):\n", - " dds = dds.chunk(dict(time=-1, lon=50, lat=50)) \n", + " dds = dds.chunk(dict(time=-1, lon=50, lat=50))\n", " filename = Path(outtmp).joinpath(f\"{ii}.zarr\")\n", - " with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client: \n", + " with Client(\n", + " n_workers=5, threads_per_worker=2, memory_limit=\"4GB\"\n", + " ) as client:\n", " display(client)\n", " save_to_zarr(\n", " ds=dds,\n", " filename=filename,\n", - " mode='o',\n", + " mode=\"o\",\n", " )\n", - " \n", + "\n", " # reassemble the 5-year chunks into a single dataset\n", - " inzarrs = sorted(list(filename.parent.glob(f'*.zarr')))\n", - " \n", + " inzarrs = sorted(list(filename.parent.glob(f\"*.zarr\")))\n", + "\n", " # open the files as a combined multi-file dataset\n", - " ds = xr.open_mfdataset(inzarrs, engine='zarr', decode_timedelta=False)\n", - " \n", + " ds = xr.open_mfdataset(inzarrs, engine=\"zarr\", decode_timedelta=False)\n", + "\n", " # define the final chunking configuration\n", - " final_chunks = dict(time=12*50, lon=50, lat=50)\n", - " \n", + " final_chunks = dict(time=12 * 50, lon=50, lat=50)\n", + "\n", " # save the final combined dataset\n", " tmpzarr = Path(outtmp).joinpath(outzarr.name)\n", " with Client(n_workers=10) as c:\n", @@ -19059,11 +19077,11 @@ " filename=tmpzarr,\n", " mode=\"o\",\n", " )\n", - " \n", + "\n", " # move the final combined file to the output location\n", " outzarr.parent.mkdir(exist_ok=True, parents=True)\n", " shutil.move(tmpzarr, outzarr)\n", - " \n", + "\n", "end_time = time.time()\n", "print(f\"Total computation time: {end_time - start_time:.2f} seconds\")" ] @@ -19071,7 +19089,7 @@ { "cell_type": "code", "execution_count": null, - "id": "971d3b5d-3c2b-4544-8fa5-20f692c39053", + "id": "16", "metadata": {}, "outputs": [], "source": [] From 4faa90a6089ed99bfb6e3efecfe0c8c8521e3a0f Mon Sep 17 00:00:00 2001 From: Asli Bese Date: Sat, 18 Jan 2025 00:51:32 -0500 Subject: [PATCH 03/10] Replaced FAQ_dask_parallel.ipynb with an updated version --- docs/source/notebooks/FAQ_dask_parallel.ipynb | 3650 +++++++++-------- 1 file changed, 1858 insertions(+), 1792 deletions(-) diff --git a/docs/source/notebooks/FAQ_dask_parallel.ipynb b/docs/source/notebooks/FAQ_dask_parallel.ipynb index ad0858f..10944cc 100644 --- a/docs/source/notebooks/FAQ_dask_parallel.ipynb +++ b/docs/source/notebooks/FAQ_dask_parallel.ipynb @@ -2,20 +2,20 @@ "cells": [ { "cell_type": "markdown", - "id": "0", + "id": "b898ae93-3234-44b1-a88f-5aa0c1d8f6b9", "metadata": {}, "source": [ "# Processing Large Climate Datasets with Dask and Xarray\n", "### How to align memory chunks with on-disk chunks?\n", - "Datasets on PAVICS are stored on disk in chunks (small blocks of data) which Dask processes one at a time. To efficiently process large datasets and prevent memory overloads, aligning in-memory chunks with the on-disk chunk structure is essential. This allows Dask to load and process data in a way that minimizes memory usage and I/O operations, speeding up computation. \n", + "Efficient data processing on PAVICS, or using xarray/Dask in general, typically requires understanding two different notions of data *chunking*: 1) *on-disk* chunking or the way the data is stored in the filesystem and 2) *in-memory* chunking, or the way that Dask will break up a large dataset into manageable portions and process the data in parallel. In the case of *memory* chunks it is important to note that users can specify any size of in-memory chunking they feel is appropriate with respect to their system resources and the analysis at hand. However, this does not mean that the calculation will be efficient. Indeed, to efficiently process large datasets and prevent memory overloads, aligning in-memory chunks with the on-disk chunk structure is essential. This allows Dask to load and process data in a way that minimizes memory usage and I/O operations, speeding up computation.\n", "\n", "I begin by checking the on-disk chunk structure of my dataset by loading it with `decode_times=False`. This skips time decoding and loads only metadata without reading the data into memory." ] }, { "cell_type": "code", - "execution_count": 12, - "id": "1", + "execution_count": 1, + "id": "9f12da60-e3df-4172-a9cb-e014801ba7c9", "metadata": { "tags": [] }, @@ -29,7 +29,7 @@ } ], "source": [ - "import xarray as xr\n", + "import xarray as xr \n", "\n", "# open the daily ERA5-land dataset as an example\n", "url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", @@ -38,12 +38,12 @@ "ds = xr.open_dataset(url, decode_times=False, chunks={})\n", "\n", "# `._ChunkSizes` attribute shows the chunk structure of the variable\n", - "print(ds[\"tas\"]._ChunkSizes)" + "print(ds['tas']._ChunkSizes)" ] }, { "cell_type": "markdown", - "id": "2", + "id": "1ab719a3-3de0-4344-aa9a-2836d4a099ed", "metadata": { "tags": [] }, @@ -52,36 +52,44 @@ "\n", "When you open a dataset using chunking, it is represented as Dask-backed arrays, where only the metadata is initially loaded, and the data itself remains on disk until needed. Dask uses lazy evalution, meaning it doesn't immediately perform the operation but instead builds a computation graph. This graph tracks the sequence of operations, delaying execution until `compute()` is explicitly called. During computation, Dask reads and processes data in chunks and loads only the necessary parts into memory. \n", "\n", - "Here's an example showing the difference in computation times when the dataset is loaded with unaligned versus aligned chunking. For this example, I resample daily ERA5-Land `tas` data to yearly means." + "Here's an example showing the difference in computation times when the dataset is loaded with unaligned versus aligned chunking. For this example, I resample daily ERA5-Land `tas` data to yearly means. Since we are calculating a single year but for all of North America, a logical choice might be to chunk the data in memory for a few days over the `time` dimension, while keeping the entire spatial domain in the `lat` and `lon` dimensions:" ] }, { "cell_type": "code", "execution_count": 2, - "id": "3", + "id": "4d646c97-eda5-4070-aa54-19ce91d908dc", "metadata": {}, "outputs": [ { "name": "stdout", "output_type": "stream", "text": [ - "CPU times: user 26.4 s, sys: 5.6 s, total: 32 s\n", - "Wall time: 3min 6s\n" + "CPU times: user 13.6 s, sys: 5.85 s, total: 19.5 s\n", + "Wall time: 5min 26s\n" ] } ], "source": [ "# open dataset with unaligned chunks\n", - "ds = xr.open_dataset(url, chunks={\"time\": 200, \"lat\": 30, \"lon\": 30})\n", + "ds = xr.open_dataset(url, chunks={\"time\":10, \"lat\":-1, \"lon\":-1})\n", "\n", "# resample the 'tas' variable to yearly means\n", "%time tas_resampled = ds['tas'].sel(time=slice('1981-01-01', '1981-12-31')).resample(time='YS').mean().compute()" ] }, + { + "cell_type": "markdown", + "id": "224cca61-81e9-4c69-9fcd-23d5421a351c", + "metadata": {}, + "source": [ + "However, this original chunking choice resulted in inefficient computation because the on-disk chunk size for the `time` dimension was much larger than 10. Processing such small chunks led to excessive overhead and slow performance. Instead of arbitrary chunking, I align the chunks with the on-disk structure by setting the `time` chunk size to match the smallest on-disk chunk and choosing spatial chunks in multiples of 50 to avoid creating too many small chunks:" + ] + }, { "cell_type": "code", - "execution_count": 3, - "id": "4", + "execution_count": 4, + "id": "a449349b-a339-4d96-aa33-12a97d9ec51e", "metadata": { "tags": [] }, @@ -90,14 +98,14 @@ "name": "stdout", "output_type": "stream", "text": [ - "CPU times: user 19.7 s, sys: 7.46 s, total: 27.2 s\n", - "Wall time: 58.6 s\n" + "CPU times: user 11.9 s, sys: 6.95 s, total: 18.8 s\n", + "Wall time: 1min 26s\n" ] } ], "source": [ "# open dataset with aligned chunks\n", - "ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", + "ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50*5, 'lon': 50*5})\n", "\n", "# resample the 'tas' variable to yearly means\n", "%time tas_resampled = ds['tas'].sel(time=slice('1981-01-01', '1981-12-31')).resample(time='YS').mean().compute()" @@ -105,15 +113,15 @@ }, { "cell_type": "markdown", - "id": "5", + "id": "47a56ac9-2d80-43ef-a283-12a2e2d42bfd", "metadata": {}, "source": [ - "Aligning the chunks reduced the computation time from approximately 3 minutes to 1 minute, by allowing Dask to load only the required data and avoid excessive memory usage and I/O operations. " + "The computation time dropped from approximately 5 minutes to just over 1 minute after aligning the chunks, as this approach better managed memory and I/O operations, leading to faster execution." ] }, { "cell_type": "markdown", - "id": "6", + "id": "2422a144-7765-4d29-9703-93d0c54a4c02", "metadata": {}, "source": [ "### How to do parallel processing with Dask Distributed Client? \n", @@ -131,7 +139,7 @@ { "cell_type": "code", "execution_count": 11, - "id": "7", + "id": "73765dbb-3bf7-4814-b44e-6246713275de", "metadata": { "tags": [] }, @@ -1689,37 +1697,33 @@ } ], "source": [ - "import time\n", - "from pathlib import Path\n", - "\n", - "import psutil\n", "import xarray as xr\n", - "from dask import compute\n", + "from pathlib import Path\n", "from dask.distributed import Client\n", + "from dask import compute\n", + "import psutil \n", + "import time\n", "\n", "start_time = time.time()\n", "\n", - "# function to compute the yearly mean for each variable\n", - "\n", - "\n", + "# function to compute the yearly mean for each variable \n", "def compute_yearly_mean(ds, var_name):\n", " var = ds[var_name]\n", - " yearly_mean = var.resample(time=\"YS\").mean()\n", + " yearly_mean = var.resample(time='YS').mean()\n", " return yearly_mean\n", "\n", - "\n", "# set up Dask client within a context manager\n", - "with Client(n_workers=5, threads_per_worker=2, memory_limit=\"4GB\") as client:\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", " # display the Dask Dashboard\n", " display(client)\n", "\n", " # open the dataset with on-disk chunking structure\n", " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - " ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", + " ds = xr.open_dataset(url, chunks={'time': 366, 'lat': 50, 'lon':50}) \n", " display(ds)\n", "\n", " # select data for the year 1981\n", - " ds = ds.sel(time=slice(\"1981-01-01\", \"1981-12-31\"))\n", + " ds = ds.sel(time=slice('1981-01-01', '1981-12-31'))\n", "\n", " variables = list(ds.data_vars)\n", "\n", @@ -1730,7 +1734,7 @@ " if not output_path.exists():\n", " yearly_mean = compute_yearly_mean(ds, var_name)\n", " # save to Zarr with compute=False to get a delayed task object\n", - " delayed_task = yearly_mean.to_zarr(output_path, mode=\"w\", compute=False)\n", + " delayed_task = yearly_mean.to_zarr(output_path, mode='w', compute=False)\n", " tasks.append(delayed_task)\n", "\n", " # trigger the execution of all delayed tasks\n", @@ -1743,21 +1747,23 @@ "\n", "end_time = time.time()\n", "elapsed_time = end_time - start_time\n", - "print(f\"Total computation time: {elapsed_time:.2f} seconds\")" + "print(f'Total computation time: {elapsed_time:.2f} seconds')" ] }, { "cell_type": "markdown", - "id": "8", + "id": "e9cee3ca-a916-4d16-aab8-75dc3692217c", "metadata": {}, "source": [ - "Let’s look at a more complex analysis involving heatwave indicators. Here, I calculate two climate indicators, `heat_wave_total_length` and `heat_wave_frequency`, using the `atmos` submodule from the `xclim` library. Both indicators rely on the same input data (`tasmin` and `tasmax`), so I create a pipeline of delayed tasks, which minimizes I/O operations by keeping the data in memory until both indicators are calculated and saved." + "Let’s look at a more complex analysis involving heatwave indicators. Here, I calculate two climate indicators, `heat_wave_total_length` and `heat_wave_frequency`, using the `atmos` submodule from the `xclim` library. Both indicators rely on the same input data (`tasmin` and `tasmax`), so I create a pipeline of delayed tasks, which minimizes I/O operations by keeping the data in memory until both indicators are calculated and saved.\n", + "\n", + "In this case, I chunk the `time` dimension into 4-year intervals to optimize memory usage, as we are analyzing a multi-year period (e.g., 1991–2010). An additional day is added to each 4-year chunk to account for leap years." ] }, { "cell_type": "code", - "execution_count": 18, - "id": "9", + "execution_count": 4, + "id": "c24f1efe-65b8-48af-9090-ae516e6840d6", "metadata": { "tags": [] }, @@ -1769,7 +1775,7 @@ "
\n", "
\n", "

Client

\n", - "

Client-77bed890-a207-11ef-ab33-0242ac12000c

\n", + "

Client-10c81637-cd09-11ef-8541-0242ac12000e

\n", " \n", "\n", " \n", @@ -1804,7 +1810,7 @@ " \n", "
\n", "

LocalCluster

\n", - "

ac36ff1b

\n", + "

0fd3ca97

\n", "
\n", " \n", "
\n", @@ -1841,11 +1847,11 @@ "
\n", "
\n", "

Scheduler

\n", - "

Scheduler-565ce5b4-3926-4ea7-a67a-7ac7806f623d

\n", + "

Scheduler-54a08564-628c-4c95-943b-3997ab43a15e

\n", " \n", " \n", " \n", "
\n", - " Comm: tcp://127.0.0.1:41377\n", + " Comm: tcp://127.0.0.1:38499\n", " \n", " Workers: 5\n", @@ -1887,7 +1893,7 @@ " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -1932,7 +1938,7 @@ "
\n", - " Comm: tcp://127.0.0.1:45771\n", + " Comm: tcp://127.0.0.1:38711\n", " \n", " Total threads: 2\n", @@ -1895,7 +1901,7 @@ "
\n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45109/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36357/status\n", " \n", " Memory: 3.73 GiB\n", @@ -1903,13 +1909,13 @@ "
\n", - " Nanny: tcp://127.0.0.1:40321\n", + " Nanny: tcp://127.0.0.1:39627\n", "
\n", - " Local directory: /tmp/dask-scratch-space/worker-i7netdoj\n", + " Local directory: /tmp/dask-scratch-space/worker-5xhh721m\n", "
\n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -1977,7 +1983,7 @@ "
\n", - " Comm: tcp://127.0.0.1:44925\n", + " Comm: tcp://127.0.0.1:33397\n", " \n", " Total threads: 2\n", @@ -1940,7 +1946,7 @@ "
\n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36703/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/46843/status\n", " \n", " Memory: 3.73 GiB\n", @@ -1948,13 +1954,13 @@ "
\n", - " Nanny: tcp://127.0.0.1:41683\n", + " Nanny: tcp://127.0.0.1:45601\n", "
\n", - " Local directory: /tmp/dask-scratch-space/worker-otas7pcr\n", + " Local directory: /tmp/dask-scratch-space/worker-n9q_9ljp\n", "
\n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -2022,7 +2028,7 @@ "
\n", - " Comm: tcp://127.0.0.1:41497\n", + " Comm: tcp://127.0.0.1:37897\n", " \n", " Total threads: 2\n", @@ -1985,7 +1991,7 @@ "
\n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/38523/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37901/status\n", " \n", " Memory: 3.73 GiB\n", @@ -1993,13 +1999,13 @@ "
\n", - " Nanny: tcp://127.0.0.1:37981\n", + " Nanny: tcp://127.0.0.1:33467\n", "
\n", - " Local directory: /tmp/dask-scratch-space/worker-bdoasynd\n", + " Local directory: /tmp/dask-scratch-space/worker-_5g51k5x\n", "
\n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -2067,7 +2073,7 @@ "
\n", - " Comm: tcp://127.0.0.1:41131\n", + " Comm: tcp://127.0.0.1:40611\n", " \n", " Total threads: 2\n", @@ -2030,7 +2036,7 @@ "
\n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45485/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/45227/status\n", " \n", " Memory: 3.73 GiB\n", @@ -2038,13 +2044,13 @@ "
\n", - " Nanny: tcp://127.0.0.1:33285\n", + " Nanny: tcp://127.0.0.1:34073\n", "
\n", - " Local directory: /tmp/dask-scratch-space/worker-2wg7r3g0\n", + " Local directory: /tmp/dask-scratch-space/worker-vdrrwwlg\n", "
\n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -2116,7 +2122,7 @@ "" ], "text/plain": [ - "" + "" ] }, "metadata": {}, @@ -2156,6 +2162,7 @@ "}\n", "\n", "html[theme=dark],\n", + "html[data-theme=dark],\n", "body[data-theme=dark],\n", "body.vscode-dark {\n", " --xr-font-color0: rgba(255, 255, 255, 1);\n", @@ -2206,7 +2213,7 @@ ".xr-sections {\n", " padding-left: 0 !important;\n", " display: grid;\n", - " grid-template-columns: 150px auto auto 1fr 20px 20px;\n", + " grid-template-columns: 150px auto auto 1fr 0 20px 0 20px;\n", "}\n", "\n", ".xr-section-item {\n", @@ -2214,7 +2221,8 @@ "}\n", "\n", ".xr-section-item input {\n", - " display: none;\n", + " display: inline-block;\n", + " opacity: 0;\n", "}\n", "\n", ".xr-section-item input + label {\n", @@ -2226,6 +2234,10 @@ " color: var(--xr-font-color2);\n", "}\n", "\n", + ".xr-section-item input:focus + label {\n", + " border: 2px solid var(--xr-font-color0);\n", + "}\n", + "\n", ".xr-section-item input:enabled + label:hover {\n", " color: var(--xr-font-color0);\n", "}\n", @@ -2488,18 +2500,18 @@ " stroke: currentColor;\n", " fill: currentColor;\n", "}\n", - "
<xarray.Dataset>\n",
+       "
<xarray.Dataset> Size: 199GB\n",
        "Dimensions:  (lat: 800, lon: 1700, time: 7305)\n",
        "Coordinates:\n",
-       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
-       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
-       "  * time     (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n",
+       "  * lat      (lat) float32 3kB 10.0 10.1 10.2 10.3 10.4 ... 89.6 89.7 89.8 89.9\n",
+       "  * lon      (lon) float32 7kB -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
+       "  * time     (time) datetime64[ns] 58kB 1991-01-01 1991-01-02 ... 2010-12-31\n",
        "Data variables:\n",
-       "    tas      (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
-       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
-       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
-       "    pr       (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
-       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
+       "    tas      (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
+       "    tasmin   (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
+       "    tasmax   (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
+       "    pr       (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
+       "    prsn     (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
        "Attributes: (12/26)\n",
        "    Conventions:          CF-1.8\n",
        "    cell_methods:         time: mean (interval: 1 day)\n",
@@ -2513,10 +2525,10 @@
        "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
        "    attribution:          Contains modified Copernicus Climate Change Service...\n",
        "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
-       "    doi:                  https://doi.org/10.24381/cds.e2161bac
\n", - " Comm: tcp://127.0.0.1:33721\n", + " Comm: tcp://127.0.0.1:37757\n", " \n", " Total threads: 2\n", @@ -2075,7 +2081,7 @@ "
\n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/37593/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/41929/status\n", " \n", " Memory: 3.73 GiB\n", @@ -2083,13 +2089,13 @@ "
\n", - " Nanny: tcp://127.0.0.1:42447\n", + " Nanny: tcp://127.0.0.1:38803\n", "
\n", - " Local directory: /tmp/dask-scratch-space/worker-xm2_1gwu\n", + " Local directory: /tmp/dask-scratch-space/worker-9bjl6880\n", "
\n", + " dtype='datetime64[ns]')
    • tas
      (time, lat, lon)
      float32
      dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
      long_name :
      2 metre temperature
      original_variable :
      t2m
      standard_name :
      air_temperature
      units :
      K
      cell_methods :
      time: point
      _ChunkSizes :
      [366 50 50]
  • \n", " \n", "
    \n", " \n", @@ -2532,17 +2544,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -2575,49 +2587,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -2696,7 +2682,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • tasmin
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • tasmin
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -2712,17 +2698,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -2755,49 +2741,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -2876,7 +2836,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • tasmax
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • tasmax
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -2892,17 +2852,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -2935,49 +2895,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -3056,7 +2990,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • pr
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Total Precipitation using a density of 1000 kg/m³.
    long_name :
    Precipitation
    original_long_name :
    Total Precipitation
    original_variable :
    tp
    standard_name :
    precipitation_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • pr
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    Precipitation
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Total Precipitation using a density of 1000 kg/m³.
    original_long_name :
    Total Precipitation
    original_variable :
    tp
    standard_name :
    precipitation_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -3072,17 +3006,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -3115,49 +3049,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -3236,7 +3144,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • prsn
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Snowfall (water equivalent) using a density of 1000 kg/m³.
    long_name :
    Snowfall flux
    original_long_name :
    Snowfall
    original_variable :
    sf
    standard_name :
    snowfall_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • prsn
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    Snowfall flux
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Snowfall (water equivalent) using a density of 1000 kg/m³.
    original_long_name :
    Snowfall
    original_variable :
    sf
    standard_name :
    snowfall_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -3252,17 +3160,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -3295,49 +3203,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -3416,7 +3298,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
    • lat
      PandasIndex
      PandasIndex(Index([              10.0, 10.100000381469727, 10.199999809265137,\n",
      +       "
    • lat
      PandasIndex
      PandasIndex(Index([              10.0, 10.100000381469727, 10.199999809265137,\n",
              "       10.300000190734863, 10.399999618530273,               10.5,\n",
              "       10.600000381469727, 10.699999809265137, 10.800000190734863,\n",
              "       10.899999618530273,\n",
      @@ -3425,7 +3307,7 @@
              "        89.30000305175781,   89.4000015258789,               89.5,\n",
              "         89.5999984741211,  89.69999694824219,  89.80000305175781,\n",
              "         89.9000015258789],\n",
      -       "      dtype='float32', name='lat', length=800))
    • lon
      PandasIndex
      PandasIndex(Index([-179.89999389648438,  -179.8000030517578,  -179.6999969482422,\n",
      +       "      dtype='float32', name='lat', length=800))
    • lon
      PandasIndex
      PandasIndex(Index([-179.89999389648438,  -179.8000030517578,  -179.6999969482422,\n",
              "       -179.60000610351562,              -179.5, -179.39999389648438,\n",
              "        -179.3000030517578,  -179.1999969482422, -179.10000610351562,\n",
              "                    -179.0,\n",
      @@ -3434,28 +3316,28 @@
              "       -10.600000381469727,               -10.5, -10.399999618530273,\n",
              "       -10.300000190734863, -10.199999809265137, -10.100000381469727,\n",
              "                     -10.0],\n",
      -       "      dtype='float32', name='lon', length=1700))
    • time
      PandasIndex
      PandasIndex(DatetimeIndex(['1991-01-01', '1991-01-02', '1991-01-03', '1991-01-04',\n",
      +       "      dtype='float32', name='lon', length=1700))
    • time
      PandasIndex
      PandasIndex(DatetimeIndex(['1991-01-01', '1991-01-02', '1991-01-03', '1991-01-04',\n",
              "               '1991-01-05', '1991-01-06', '1991-01-07', '1991-01-08',\n",
              "               '1991-01-09', '1991-01-10',\n",
              "               ...\n",
              "               '2010-12-22', '2010-12-23', '2010-12-24', '2010-12-25',\n",
              "               '2010-12-26', '2010-12-27', '2010-12-28', '2010-12-29',\n",
              "               '2010-12-30', '2010-12-31'],\n",
      -       "              dtype='datetime64[ns]', name='time', length=7305, freq=None))
  • Conventions :
    CF-1.8
    cell_methods :
    time: mean (interval: 1 day)
    data_specs_version :
    00.00.07
    domain :
    NAM
    format :
    netcdf
    frequency :
    day
    history :
    [2022-07-16 11:22:42.650045] Converted from original data to zarr with modified metadata for CF-like compliance.
    institution :
    ECMWF
    processing_level :
    raw
    project :
    era5-land
    realm :
    atmos
    source :
    ERA5-Land
    table_date :
    13 June 2022
    type :
    reanalysis
    version :
    v2022.07.15
    title :
    ERA5-Land : daily
    institute :
    European Centre for Medium-Range Weather Forecasts
    institute_id :
    ECMWF
    dataset_id :
    ERA5-Land
    abstract :
    ERA5-Land provides hourly high resolution information of surface variables. The data is a replay of the land component of the ERA5 climate reanalysis with a finer spatial resolution: ~9km grid spacing. ERA5-Land includes information about uncertainties for all variables at reduced spatial and temporal resolutions. The model used in the production of ERA5-Land is the tiled ECMWF Scheme for Surface Exchanges over Land incorporating land surface hydrology (H-TESSEL).
    dataset_description :
    https://www.ecmwf.int/en/era5-land
    license_type :
    permissive
    license :
    Please acknowledge the use of ERA5-Land as stated in the Copernicus C3S/CAMS License agreement http://apps.ecmwf.int/datasets/licences/copernicus/
    attribution :
    Contains modified Copernicus Climate Change Service Information 2022. Neither the European Commission nor ECMWF is responsible for any use that may be made of the Copernicus Information or Data it contains.
    citation :
    Muñoz Sabater, J., (2021): ERA5-Land hourly data from 1950 to 1980. Copernicus Climate Change Service (C3S) Climate Data Store (CDS). (Accessed on 2022.06.07)
    doi :
    https://doi.org/10.24381/cds.e2161bac
  • " + " dtype='datetime64[ns]', name='time', length=7305, freq=None))
  • Conventions :
    CF-1.8
    cell_methods :
    time: mean (interval: 1 day)
    data_specs_version :
    00.00.07
    domain :
    NAM
    format :
    netcdf
    frequency :
    day
    history :
    [2022-07-16 11:22:42.650045] Converted from original data to zarr with modified metadata for CF-like compliance.
    institution :
    ECMWF
    processing_level :
    raw
    project :
    era5-land
    realm :
    atmos
    source :
    ERA5-Land
    table_date :
    13 June 2022
    type :
    reanalysis
    version :
    v2022.07.15
    title :
    ERA5-Land : daily
    institute :
    European Centre for Medium-Range Weather Forecasts
    institute_id :
    ECMWF
    dataset_id :
    ERA5-Land
    abstract :
    ERA5-Land provides hourly high resolution information of surface variables. The data is a replay of the land component of the ERA5 climate reanalysis with a finer spatial resolution: ~9km grid spacing. ERA5-Land includes information about uncertainties for all variables at reduced spatial and temporal resolutions. The model used in the production of ERA5-Land is the tiled ECMWF Scheme for Surface Exchanges over Land incorporating land surface hydrology (H-TESSEL).
    dataset_description :
    https://www.ecmwf.int/en/era5-land
    license_type :
    permissive
    license :
    Please acknowledge the use of ERA5-Land as stated in the Copernicus C3S/CAMS License agreement http://apps.ecmwf.int/datasets/licences/copernicus/
    attribution :
    Contains modified Copernicus Climate Change Service Information 2022. Neither the European Commission nor ECMWF is responsible for any use that may be made of the Copernicus Information or Data it contains.
    citation :
    Muñoz Sabater, J., (2021): ERA5-Land hourly data from 1950 to 1980. Copernicus Climate Change Service (C3S) Climate Data Store (CDS). (Accessed on 2022.06.07)
    doi :
    https://doi.org/10.24381/cds.e2161bac
  • " ], "text/plain": [ - "\n", + " Size: 199GB\n", "Dimensions: (lat: 800, lon: 1700, time: 7305)\n", "Coordinates:\n", - " * lat (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n", - " * lon (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", - " * time (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n", + " * lat (lat) float32 3kB 10.0 10.1 10.2 10.3 10.4 ... 89.6 89.7 89.8 89.9\n", + " * lon (lon) float32 7kB -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", + " * time (time) datetime64[ns] 58kB 1991-01-01 1991-01-02 ... 2010-12-31\n", "Data variables:\n", - " tas (time, lat, lon) float32 dask.array\n", - " tasmin (time, lat, lon) float32 dask.array\n", - " tasmax (time, lat, lon) float32 dask.array\n", - " pr (time, lat, lon) float32 dask.array\n", - " prsn (time, lat, lon) float32 dask.array\n", + " tas (time, lat, lon) float32 40GB dask.array\n", + " tasmin (time, lat, lon) float32 40GB dask.array\n", + " tasmax (time, lat, lon) float32 40GB dask.array\n", + " pr (time, lat, lon) float32 40GB dask.array\n", + " prsn (time, lat, lon) float32 40GB dask.array\n", "Attributes: (12/26)\n", " Conventions: CF-1.8\n", " cell_methods: time: mean (interval: 1 day)\n", @@ -3479,13 +3361,13 @@ "name": "stderr", "output_type": "stream", "text": [ - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`.\n", " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`.\n", " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`.\n", " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`.\n", " _check_cell_methods(\n" ] }, @@ -3493,25 +3375,8 @@ "name": "stdout", "output_type": "stream", "text": [ - "indicator : \n" - ] - }, - { - "name": "stderr", - "output_type": "stream", - "text": [ - "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 15.73 MiB.\n", - "This may cause some slowdown.\n", - "Consider scattering data ahead of time and using futures.\n", - " warnings.warn(\n" - ] - }, - { - "name": "stdout", - "output_type": "stream", - "text": [ - "Total memory usage across all workers: 1.95 GB\n", - "Total computation time: 415.93 seconds\n" + "Total memory usage across all workers: 2.14 GB\n", + "Total computation time: 1608.42 seconds\n" ] } ], @@ -3532,7 +3397,7 @@ "\n", " # load data using on-disk chunk sizes\n", " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - " ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", + " ds = xr.open_dataset(url, chunks={\"time\": (365*4)+1, \"lat\": 50, \"lon\": 50})\n", "\n", " # we're interested in the 1991-2010 period for our heatwave analysis\n", " ds = ds.sel(time=slice(\"1991-01-01\", \"2010-12-31\"))\n", @@ -3583,8 +3448,10 @@ }, { "cell_type": "markdown", - "id": "10", - "metadata": {}, + "id": "12d29293-ad43-44f5-8d4a-1ecdbf8574ce", + "metadata": { + "tags": [] + }, "source": [ "### What can we do when we have large task graphs / large memory footprint? \n", "One downside of using a fully delayed computation approach is that it can lead to the creation of large task graphs that are difficult to manage. This can result in excessive memory consumption as the Dask scheduler struggles to handle numerous interdependent tasks.\n", @@ -3594,8 +3461,8 @@ }, { "cell_type": "code", - "execution_count": 22, - "id": "11", + "execution_count": 5, + "id": "48e59f9d-3d12-4119-9f3c-0cd694ccd871", "metadata": {}, "outputs": [ { @@ -3605,7 +3472,7 @@ "
    \n", "
    \n", "

    Client

    \n", - "

    Client-b8e9b705-a2bb-11ef-ab33-0242ac12000c

    \n", + "

    Client-06e631ff-d518-11ef-809f-0242ac12001f

    \n", " \n", "\n", " \n", @@ -3640,7 +3507,7 @@ " \n", "
    \n", "

    LocalCluster

    \n", - "

    24ecda26

    \n", + "

    af514157

    \n", "
    \n", " \n", "
    \n", @@ -3677,11 +3544,11 @@ "
    \n", "
    \n", "

    Scheduler

    \n", - "

    Scheduler-66a88548-a02f-4504-9661-370df089e931

    \n", + "

    Scheduler-22125e5c-f659-4613-be3b-f0770d2e8eb1

    \n", " \n", " \n", " \n", "
    \n", - " Comm: tcp://127.0.0.1:41109\n", + " Comm: tcp://127.0.0.1:34229\n", " \n", " Workers: 5\n", @@ -3723,7 +3590,7 @@ " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -3768,7 +3635,7 @@ "
    \n", - " Comm: tcp://127.0.0.1:44813\n", + " Comm: tcp://127.0.0.1:34223\n", " \n", " Total threads: 2\n", @@ -3731,7 +3598,7 @@ "
    \n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/34925/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40455/status\n", " \n", " Memory: 3.73 GiB\n", @@ -3739,13 +3606,13 @@ "
    \n", - " Nanny: tcp://127.0.0.1:44017\n", + " Nanny: tcp://127.0.0.1:39307\n", "
    \n", - " Local directory: /tmp/dask-scratch-space/worker-ck_2wyzo\n", + " Local directory: /tmp/dask-scratch-space/worker-qt5_xqkp\n", "
    \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -3813,7 +3680,7 @@ "
    \n", - " Comm: tcp://127.0.0.1:42169\n", + " Comm: tcp://127.0.0.1:44209\n", " \n", " Total threads: 2\n", @@ -3776,7 +3643,7 @@ "
    \n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/44117/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36523/status\n", " \n", " Memory: 3.73 GiB\n", @@ -3784,13 +3651,13 @@ "
    \n", - " Nanny: tcp://127.0.0.1:35551\n", + " Nanny: tcp://127.0.0.1:34513\n", "
    \n", - " Local directory: /tmp/dask-scratch-space/worker-pjs6yx3h\n", + " Local directory: /tmp/dask-scratch-space/worker-4y__vkhu\n", "
    \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -3858,7 +3725,7 @@ "
    \n", - " Comm: tcp://127.0.0.1:39133\n", + " Comm: tcp://127.0.0.1:38925\n", " \n", " Total threads: 2\n", @@ -3821,7 +3688,7 @@ "
    \n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/36913/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/40639/status\n", " \n", " Memory: 3.73 GiB\n", @@ -3829,13 +3696,13 @@ "
    \n", - " Nanny: tcp://127.0.0.1:46425\n", + " Nanny: tcp://127.0.0.1:38497\n", "
    \n", - " Local directory: /tmp/dask-scratch-space/worker-l0k5a_85\n", + " Local directory: /tmp/dask-scratch-space/worker-uptflgsn\n", "
    \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -3903,7 +3770,7 @@ "
    \n", - " Comm: tcp://127.0.0.1:34665\n", + " Comm: tcp://127.0.0.1:39845\n", " \n", " Total threads: 2\n", @@ -3866,7 +3733,7 @@ "
    \n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35467/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33317/status\n", " \n", " Memory: 3.73 GiB\n", @@ -3874,13 +3741,13 @@ "
    \n", - " Nanny: tcp://127.0.0.1:38721\n", + " Nanny: tcp://127.0.0.1:35209\n", "
    \n", - " Local directory: /tmp/dask-scratch-space/worker-zkv4axvy\n", + " Local directory: /tmp/dask-scratch-space/worker-1i_l5uvg\n", "
    \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", " \n", "\n", @@ -3952,7 +3819,7 @@ "" ], "text/plain": [ - "" + "" ] }, "metadata": {}, @@ -3992,6 +3859,7 @@ "}\n", "\n", "html[theme=dark],\n", + "html[data-theme=dark],\n", "body[data-theme=dark],\n", "body.vscode-dark {\n", " --xr-font-color0: rgba(255, 255, 255, 1);\n", @@ -4042,7 +3910,7 @@ ".xr-sections {\n", " padding-left: 0 !important;\n", " display: grid;\n", - " grid-template-columns: 150px auto auto 1fr 20px 20px;\n", + " grid-template-columns: 150px auto auto 1fr 0 20px 0 20px;\n", "}\n", "\n", ".xr-section-item {\n", @@ -4050,7 +3918,8 @@ "}\n", "\n", ".xr-section-item input {\n", - " display: none;\n", + " display: inline-block;\n", + " opacity: 0;\n", "}\n", "\n", ".xr-section-item input + label {\n", @@ -4062,6 +3931,10 @@ " color: var(--xr-font-color2);\n", "}\n", "\n", + ".xr-section-item input:focus + label {\n", + " border: 2px solid var(--xr-font-color0);\n", + "}\n", + "\n", ".xr-section-item input:enabled + label:hover {\n", " color: var(--xr-font-color0);\n", "}\n", @@ -4324,18 +4197,18 @@ " stroke: currentColor;\n", " fill: currentColor;\n", "}\n", - "
    <xarray.Dataset>\n",
    +       "
    <xarray.Dataset> Size: 199GB\n",
            "Dimensions:  (lat: 800, lon: 1700, time: 7305)\n",
            "Coordinates:\n",
    -       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
    -       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
    -       "  * time     (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n",
    +       "  * lat      (lat) float32 3kB 10.0 10.1 10.2 10.3 10.4 ... 89.6 89.7 89.8 89.9\n",
    +       "  * lon      (lon) float32 7kB -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
    +       "  * time     (time) datetime64[ns] 58kB 1991-01-01 1991-01-02 ... 2010-12-31\n",
            "Data variables:\n",
    -       "    tas      (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    pr       (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    +       "    tas      (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    tasmin   (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    tasmax   (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    pr       (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    prsn     (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
            "Attributes: (12/26)\n",
            "    Conventions:          CF-1.8\n",
            "    cell_methods:         time: mean (interval: 1 day)\n",
    @@ -4349,10 +4222,10 @@
            "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
            "    attribution:          Contains modified Copernicus Climate Change Service...\n",
            "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
    -       "    doi:                  https://doi.org/10.24381/cds.e2161bac
    \n", - " Comm: tcp://127.0.0.1:36431\n", + " Comm: tcp://127.0.0.1:46049\n", " \n", " Total threads: 2\n", @@ -3911,7 +3778,7 @@ "
    \n", - " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/33927/status\n", + " Dashboard: https://pavics.ouranos.ca/jupyter/user/aslibese/proxy/35959/status\n", " \n", " Memory: 3.73 GiB\n", @@ -3919,13 +3786,13 @@ "
    \n", - " Nanny: tcp://127.0.0.1:43431\n", + " Nanny: tcp://127.0.0.1:45731\n", "
    \n", - " Local directory: /tmp/dask-scratch-space/worker-e5q354bb\n", + " Local directory: /tmp/dask-scratch-space/worker-vf6_0fn9\n", "
    \n", + " dtype='datetime64[ns]')
    • tas
      (time, lat, lon)
      float32
      dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
      long_name :
      2 metre temperature
      original_variable :
      t2m
      standard_name :
      air_temperature
      units :
      K
      cell_methods :
      time: point
      _ChunkSizes :
      [366 50 50]
  • \n", " \n", "
    \n", " \n", @@ -4368,17 +4241,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -4411,49 +4284,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -4532,7 +4379,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • tasmin
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • tasmin
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -4548,17 +4395,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -4591,49 +4438,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -4712,7 +4533,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • tasmax
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • tasmax
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    2 metre temperature
    original_variable :
    t2m
    standard_name :
    air_temperature
    units :
    K
    cell_methods :
    time: point
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -4728,17 +4549,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -4771,49 +4592,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -4892,7 +4687,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • pr
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Total Precipitation using a density of 1000 kg/m³.
    long_name :
    Precipitation
    original_long_name :
    Total Precipitation
    original_variable :
    tp
    standard_name :
    precipitation_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • pr
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    Precipitation
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Total Precipitation using a density of 1000 kg/m³.
    original_long_name :
    Total Precipitation
    original_variable :
    tp
    standard_name :
    precipitation_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -4908,17 +4703,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -4951,49 +4746,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -5072,7 +4841,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
  • prsn
    (time, lat, lon)
    float32
    dask.array<chunksize=(31, 50, 50), meta=np.ndarray>
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Snowfall (water equivalent) using a density of 1000 kg/m³.
    long_name :
    Snowfall flux
    original_long_name :
    Snowfall
    original_variable :
    sf
    standard_name :
    snowfall_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", + "
  • prsn
    (time, lat, lon)
    float32
    dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>
    long_name :
    Snowfall flux
    cell_methods :
    time: mean (interval: 1 hour)
    comments :
    Converted from Snowfall (water equivalent) using a density of 1000 kg/m³.
    original_long_name :
    Snowfall
    original_variable :
    sf
    standard_name :
    snowfall_flux
    units :
    kg m-2 s-1
    _ChunkSizes :
    [366 50 50]
    \n", " \n", "
    \n", " \n", @@ -5088,17 +4857,17 @@ " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", - " \n", + " \n", " \n", " \n", " \n", @@ -5131,49 +4900,23 @@ "\n", " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", - " \n", + " \n", "\n", " \n", " \n", - " \n", - " \n", - " \n", " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", - " \n", + " \n", + " \n", + " \n", + " \n", " \n", "\n", " \n", @@ -5252,7 +4995,7 @@ "\n", " \n", " \n", - "
    Bytes 37.01 GiB 3.49 MiB 13.93 MiB
    Shape (7305, 800, 1700) (366, 50, 50) (1461, 50, 50)
    Dask graph 11424 chunks in 3 graph layers 3264 chunks in 3 graph layers
    Data type
    • lat
      PandasIndex
      PandasIndex(Index([              10.0, 10.100000381469727, 10.199999809265137,\n",
      +       "
    • lat
      PandasIndex
      PandasIndex(Index([              10.0, 10.100000381469727, 10.199999809265137,\n",
              "       10.300000190734863, 10.399999618530273,               10.5,\n",
              "       10.600000381469727, 10.699999809265137, 10.800000190734863,\n",
              "       10.899999618530273,\n",
      @@ -5261,7 +5004,7 @@
              "        89.30000305175781,   89.4000015258789,               89.5,\n",
              "         89.5999984741211,  89.69999694824219,  89.80000305175781,\n",
              "         89.9000015258789],\n",
      -       "      dtype='float32', name='lat', length=800))
    • lon
      PandasIndex
      PandasIndex(Index([-179.89999389648438,  -179.8000030517578,  -179.6999969482422,\n",
      +       "      dtype='float32', name='lat', length=800))
    • lon
      PandasIndex
      PandasIndex(Index([-179.89999389648438,  -179.8000030517578,  -179.6999969482422,\n",
              "       -179.60000610351562,              -179.5, -179.39999389648438,\n",
              "        -179.3000030517578,  -179.1999969482422, -179.10000610351562,\n",
              "                    -179.0,\n",
      @@ -5270,28 +5013,28 @@
              "       -10.600000381469727,               -10.5, -10.399999618530273,\n",
              "       -10.300000190734863, -10.199999809265137, -10.100000381469727,\n",
              "                     -10.0],\n",
      -       "      dtype='float32', name='lon', length=1700))
    • time
      PandasIndex
      PandasIndex(DatetimeIndex(['1991-01-01', '1991-01-02', '1991-01-03', '1991-01-04',\n",
      +       "      dtype='float32', name='lon', length=1700))
    • time
      PandasIndex
      PandasIndex(DatetimeIndex(['1991-01-01', '1991-01-02', '1991-01-03', '1991-01-04',\n",
              "               '1991-01-05', '1991-01-06', '1991-01-07', '1991-01-08',\n",
              "               '1991-01-09', '1991-01-10',\n",
              "               ...\n",
              "               '2010-12-22', '2010-12-23', '2010-12-24', '2010-12-25',\n",
              "               '2010-12-26', '2010-12-27', '2010-12-28', '2010-12-29',\n",
              "               '2010-12-30', '2010-12-31'],\n",
      -       "              dtype='datetime64[ns]', name='time', length=7305, freq=None))
  • Conventions :
    CF-1.8
    cell_methods :
    time: mean (interval: 1 day)
    data_specs_version :
    00.00.07
    domain :
    NAM
    format :
    netcdf
    frequency :
    day
    history :
    [2022-07-16 11:22:42.650045] Converted from original data to zarr with modified metadata for CF-like compliance.
    institution :
    ECMWF
    processing_level :
    raw
    project :
    era5-land
    realm :
    atmos
    source :
    ERA5-Land
    table_date :
    13 June 2022
    type :
    reanalysis
    version :
    v2022.07.15
    title :
    ERA5-Land : daily
    institute :
    European Centre for Medium-Range Weather Forecasts
    institute_id :
    ECMWF
    dataset_id :
    ERA5-Land
    abstract :
    ERA5-Land provides hourly high resolution information of surface variables. The data is a replay of the land component of the ERA5 climate reanalysis with a finer spatial resolution: ~9km grid spacing. ERA5-Land includes information about uncertainties for all variables at reduced spatial and temporal resolutions. The model used in the production of ERA5-Land is the tiled ECMWF Scheme for Surface Exchanges over Land incorporating land surface hydrology (H-TESSEL).
    dataset_description :
    https://www.ecmwf.int/en/era5-land
    license_type :
    permissive
    license :
    Please acknowledge the use of ERA5-Land as stated in the Copernicus C3S/CAMS License agreement http://apps.ecmwf.int/datasets/licences/copernicus/
    attribution :
    Contains modified Copernicus Climate Change Service Information 2022. Neither the European Commission nor ECMWF is responsible for any use that may be made of the Copernicus Information or Data it contains.
    citation :
    Muñoz Sabater, J., (2021): ERA5-Land hourly data from 1950 to 1980. Copernicus Climate Change Service (C3S) Climate Data Store (CDS). (Accessed on 2022.06.07)
    doi :
    https://doi.org/10.24381/cds.e2161bac
  • " + " dtype='datetime64[ns]', name='time', length=7305, freq=None))
  • Conventions :
    CF-1.8
    cell_methods :
    time: mean (interval: 1 day)
    data_specs_version :
    00.00.07
    domain :
    NAM
    format :
    netcdf
    frequency :
    day
    history :
    [2022-07-16 11:22:42.650045] Converted from original data to zarr with modified metadata for CF-like compliance.
    institution :
    ECMWF
    processing_level :
    raw
    project :
    era5-land
    realm :
    atmos
    source :
    ERA5-Land
    table_date :
    13 June 2022
    type :
    reanalysis
    version :
    v2022.07.15
    title :
    ERA5-Land : daily
    institute :
    European Centre for Medium-Range Weather Forecasts
    institute_id :
    ECMWF
    dataset_id :
    ERA5-Land
    abstract :
    ERA5-Land provides hourly high resolution information of surface variables. The data is a replay of the land component of the ERA5 climate reanalysis with a finer spatial resolution: ~9km grid spacing. ERA5-Land includes information about uncertainties for all variables at reduced spatial and temporal resolutions. The model used in the production of ERA5-Land is the tiled ECMWF Scheme for Surface Exchanges over Land incorporating land surface hydrology (H-TESSEL).
    dataset_description :
    https://www.ecmwf.int/en/era5-land
    license_type :
    permissive
    license :
    Please acknowledge the use of ERA5-Land as stated in the Copernicus C3S/CAMS License agreement http://apps.ecmwf.int/datasets/licences/copernicus/
    attribution :
    Contains modified Copernicus Climate Change Service Information 2022. Neither the European Commission nor ECMWF is responsible for any use that may be made of the Copernicus Information or Data it contains.
    citation :
    Muñoz Sabater, J., (2021): ERA5-Land hourly data from 1950 to 1980. Copernicus Climate Change Service (C3S) Climate Data Store (CDS). (Accessed on 2022.06.07)
    doi :
    https://doi.org/10.24381/cds.e2161bac
  • " ], "text/plain": [ - "\n", + " Size: 199GB\n", "Dimensions: (lat: 800, lon: 1700, time: 7305)\n", "Coordinates:\n", - " * lat (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n", - " * lon (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", - " * time (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n", + " * lat (lat) float32 3kB 10.0 10.1 10.2 10.3 10.4 ... 89.6 89.7 89.8 89.9\n", + " * lon (lon) float32 7kB -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n", + " * time (time) datetime64[ns] 58kB 1991-01-01 1991-01-02 ... 2010-12-31\n", "Data variables:\n", - " tas (time, lat, lon) float32 dask.array\n", - " tasmin (time, lat, lon) float32 dask.array\n", - " tasmax (time, lat, lon) float32 dask.array\n", - " pr (time, lat, lon) float32 dask.array\n", - " prsn (time, lat, lon) float32 dask.array\n", + " tas (time, lat, lon) float32 40GB dask.array\n", + " tasmin (time, lat, lon) float32 40GB dask.array\n", + " tasmax (time, lat, lon) float32 40GB dask.array\n", + " pr (time, lat, lon) float32 40GB dask.array\n", + " prsn (time, lat, lon) float32 40GB dask.array\n", "Attributes: (12/26)\n", " Conventions: CF-1.8\n", " cell_methods: time: mean (interval: 1 day)\n", @@ -5315,111 +5058,73 @@ "name": "stderr", "output_type": "stream", "text": [ - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`.\n", " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`.\n", " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 36.85 MiB.\n", - "This may cause some slowdown.\n", - "Consider scattering data ahead of time and using futures.\n", - " warnings.warn(\n", - "2024-11-14 19:09:25,025 - tornado.application - ERROR - Uncaught exception GET /status/ws (127.0.0.1)\n", - "HTTPServerRequest(protocol='http', host='pavics.ouranos.ca', method='GET', uri='/status/ws', version='HTTP/1.1', remote_ip='127.0.0.1')\n", - "Traceback (most recent call last):\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/web.py\", line 1790, in _execute\n", - " result = await result\n", - " ^^^^^^^^^^^^\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 273, in get\n", - " await self.ws_connection.accept_connection(self)\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 863, in accept_connection\n", - " await self._accept_connection(handler)\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 946, in _accept_connection\n", - " await self._receive_frame_loop()\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 1105, in _receive_frame_loop\n", - " self.handler.on_ws_connection_close(self.close_code, self.close_reason)\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 571, in on_ws_connection_close\n", - " self.on_connection_close()\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/tornado/websocket.py\", line 563, in on_connection_close\n", - " self.on_close()\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/bokeh/server/views/ws.py\", line 308, in on_close\n", - " self.connection.session.notify_connection_lost()\n", - " ^^^^^^^^^^^^^^^^^^^^^^^\n", - " File \"/opt/conda/envs/birdy/lib/python3.11/site-packages/bokeh/server/connection.py\", line 65, in session\n", - " assert self._session is not None\n", - " ^^^^^^^^^^^^^^^^^^^^^^^^^\n", - "AssertionError\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`\n", + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: minimum`.\n", " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:41: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`\n", - " _check_cell_methods(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 36.86 MiB.\n", - "This may cause some slowdown.\n", - "Consider scattering data ahead of time and using futures.\n", - " warnings.warn(\n", - "/opt/conda/envs/birdy/lib/python3.11/site-packages/distributed/client.py:3161: UserWarning: Sending large graph of size 16.15 MiB.\n", - "This may cause some slowdown.\n", - "Consider scattering data ahead of time and using futures.\n", - " warnings.warn(\n" + "/opt/conda/envs/birdy/lib/python3.11/site-packages/xclim/core/cfchecks.py:43: UserWarning: Variable has a non-conforming cell_methods: Got `time: point`, which do not include the expected `time: maximum`.\n", + " _check_cell_methods(\n" ] }, { "name": "stdout", "output_type": "stream", "text": [ - "Total memory usage across all workers: 2.65 GB\n", - "Total computation time: 2484.91 seconds\n" + "Total memory usage across all workers: 1.95 GB\n", + "Total computation time: 2820.25 seconds\n" ] } ], "source": [ - "import time\n", "from pathlib import Path\n", - "\n", - "import psutil\n", - "from dask import compute\n", "from dask.distributed import Client\n", + "import time\n", "from xclim import atmos\n", + "from dask import compute\n", + "import psutil \n", "\n", "start_time = time.time()\n", "\n", - "with Client(n_workers=5, threads_per_worker=2, memory_limit=\"4GB\") as client:\n", + "with Client(n_workers=5, threads_per_worker=2, memory_limit='4GB') as client:\n", " display(client)\n", - "\n", + " \n", " url = \"https://pavics.ouranos.ca/twitcher/ows/proxy/thredds/dodsC/datasets/reanalyses/day_ERA5-Land_NAM.ncml\"\n", - " ds = xr.open_dataset(url, chunks={\"time\": 366, \"lat\": 50, \"lon\": 50})\n", - "\n", - " ds = ds.sel(time=slice(\"1991-01-01\", \"2010-12-31\"))\n", - "\n", + " ds = xr.open_dataset(url, chunks={'time': (365*4)+1, 'lat': 50, 'lon': 50})\n", + " \n", + " ds = ds.sel(time=slice('1991-01-01', '2010-12-31'))\n", + " \n", " display(ds)\n", "\n", - " indicators = [\n", - " atmos.heat_wave_total_length,\n", - " atmos.heat_wave_frequency,\n", - " atmos._precip.precip_accumulation,\n", - " ]\n", + " indicators = [atmos.heat_wave_total_length, atmos.heat_wave_frequency, atmos._precip.precip_accumulation] \n", "\n", " for indicator in indicators:\n", - " ds_out = xr.Dataset(attrs=ds.attrs)\n", - "\n", + " ds_out = xr.Dataset(attrs=ds.attrs) \n", + " \n", " if indicator == atmos._precip.precip_accumulation:\n", " # calculate indicator\n", " out = indicator(\n", " ds=ds,\n", - " freq=\"YS\",\n", + " freq='YS',\n", " )\n", - " else:\n", - " out = indicator(ds=ds, freq=\"YS\", resample_before_rl=False)\n", - "\n", - " out = out.chunk({\"time\": -1, \"lat\": 50, \"lon\": 50})\n", + " else: \n", + " out = indicator(\n", + " ds=ds,\n", + " freq='YS',\n", + " resample_before_rl=False\n", + " )\n", + " \n", + " out = out.chunk({'time': -1, 'lat': 50, 'lon': 50})\n", " ds_out[out.name] = out\n", "\n", - " output_path = Path(f\"heatwave_output_ex2/{out.name}_1991-2010.zarr\")\n", + " output_path = Path(f'heatwave_output_ex2/{out.name}_1991-2010.zarr')\n", " output_path.parent.mkdir(parents=True, exist_ok=True)\n", "\n", " if not output_path.exists():\n", " # save to Zarr, triggering computation immediately\n", " ds_out.to_zarr(output_path)\n", - "\n", + " \n", " # fetch memory usage from all workers and display the total usage\n", " worker_memory = client.run(lambda: psutil.Process().memory_info().rss)\n", " total_memory = sum(worker_memory.values())\n", @@ -5431,7 +5136,7 @@ }, { "cell_type": "markdown", - "id": "12", + "id": "3849655b-5c02-444a-a5ce-f3e788f5242f", "metadata": {}, "source": [ "Even with the sequential computation approach, there may be scenarios where the heatwave indicator data is still too large to write in a single step. In such cases, an effective strategy is to split the dataset into smaller, manageable spatial chunks, which would allow for more efficient processing and data writing.\n", @@ -5441,8 +5146,8 @@ }, { "cell_type": "code", - "execution_count": 7, - "id": "13", + "execution_count": 6, + "id": "8b6d03a4-0e9e-4c6f-a74c-fbe77b692de3", "metadata": { "tags": [] }, @@ -5481,6 +5186,7 @@ "}\n", "\n", "html[theme=dark],\n", + "html[data-theme=dark],\n", "body[data-theme=dark],\n", "body.vscode-dark {\n", " --xr-font-color0: rgba(255, 255, 255, 1);\n", @@ -5531,7 +5237,7 @@ ".xr-sections {\n", " padding-left: 0 !important;\n", " display: grid;\n", - " grid-template-columns: 150px auto auto 1fr 20px 20px;\n", + " grid-template-columns: 150px auto auto 1fr 0 20px 0 20px;\n", "}\n", "\n", ".xr-section-item {\n", @@ -5539,7 +5245,8 @@ "}\n", "\n", ".xr-section-item input {\n", - " display: none;\n", + " display: inline-block;\n", + " opacity: 0;\n", "}\n", "\n", ".xr-section-item input + label {\n", @@ -5551,6 +5258,10 @@ " color: var(--xr-font-color2);\n", "}\n", "\n", + ".xr-section-item input:focus + label {\n", + " border: 2px solid var(--xr-font-color0);\n", + "}\n", + "\n", ".xr-section-item input:enabled + label:hover {\n", " color: var(--xr-font-color0);\n", "}\n", @@ -5813,18 +5524,18 @@ " stroke: currentColor;\n", " fill: currentColor;\n", "}\n", - "
    <xarray.Dataset>\n",
    +       "
    <xarray.Dataset> Size: 199GB\n",
            "Dimensions:  (lat: 800, lon: 1700, time: 7305)\n",
            "Coordinates:\n",
    -       "  * lat      (lat) float32 10.0 10.1 10.2 10.3 10.4 ... 89.5 89.6 89.7 89.8 89.9\n",
    -       "  * lon      (lon) float32 -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
    -       "  * time     (time) datetime64[ns] 1991-01-01 1991-01-02 ... 2010-12-31\n",
    +       "  * lat      (lat) float32 3kB 10.0 10.1 10.2 10.3 10.4 ... 89.6 89.7 89.8 89.9\n",
    +       "  * lon      (lon) float32 7kB -179.9 -179.8 -179.7 -179.6 ... -10.2 -10.1 -10.0\n",
    +       "  * time     (time) datetime64[ns] 58kB 1991-01-01 1991-01-02 ... 2010-12-31\n",
            "Data variables:\n",
    -       "    tas      (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    tasmin   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    tasmax   (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    pr       (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    -       "    prsn     (time, lat, lon) float32 dask.array<chunksize=(31, 50, 50), meta=np.ndarray>\n",
    +       "    tas      (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    tasmin   (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    tasmax   (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    pr       (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
    +       "    prsn     (time, lat, lon) float32 40GB dask.array<chunksize=(1096, 50, 50), meta=np.ndarray>\n",
            "Attributes: (12/26)\n",
            "    Conventions:          CF-1.8\n",
            "    cell_methods:         time: mean (interval: 1 day)\n",
    @@ -5838,10 +5549,10 @@
            "    license:              Please acknowledge the use of ERA5-Land as stated i...\n",
            "    attribution:          Contains modified Copernicus Climate Change Service...\n",
            "    citation:             Muñoz Sabater, J., (2021): ERA5-Land hourly data fr...\n",
    -       "    doi:                  https://doi.org/10.24381/cds.e2161bac