From 36faf3ea447c18c942aafdba1080cb40136907f0 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Sun, 22 Nov 2020 17:49:36 -0500 Subject: [PATCH 01/34] wip: new recipe syntax --- pangeo_forge/recipe/recipe.py | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 pangeo_forge/recipe/recipe.py diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe/recipe.py new file mode 100644 index 00000000..266b0506 --- /dev/null +++ b/pangeo_forge/recipe/recipe.py @@ -0,0 +1,28 @@ +""" +A Pangeo Forge Recipe +""" + +import xarray as xr +import fsspec + + +class Recipe: + datasets = [] + keys = [] + + def filenames_for_chunk(**chunk_keys): + raise NotImplementedError + + def open_file(self, fname): + # todo: caching + return fsspec.open(fname) + + def open_chunk(self, filenames): + files_to_open = [self.open_file(f) for f in filenames] + ds_chunk = xr.open_mfdataset(filenames, **self.open_chunk_kwargs) + return ds_chunk + + def store_chunk(self, store_target, **chunk_keys): + filenames = self.filenames_for_chunk(**chunk_keys) + ds_chunk = self.open_chunk(filenames) + ds_chunk.to_zarr(store_target, mode='a') From 7ee78f22c5108fc10009b9bcf8f7bb668b041a4d Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Sat, 28 Nov 2020 21:32:01 -0500 Subject: [PATCH 02/34] messy wip --- pangeo_forge/__init__.py | 10 +- pangeo_forge/pipelines/__init__.py | 5 - pangeo_forge/recipe/__init__.py | 0 pangeo_forge/recipe/pipeline.py | 0 pangeo_forge/recipe/recipe.py | 154 +++++++++++++++++++++++++++-- pangeo_forge/recipe/target.py | 22 +++++ pangeo_forge/utils.py | 18 ---- requirements.txt | 1 - tests/__init__.py | 0 tests/fixtures.py | 88 +++++++++++++++++ tests/test_recipe.py | 48 +++++++++ tests/test_utils.py | 17 +++- 12 files changed, 322 insertions(+), 41 deletions(-) create mode 100644 pangeo_forge/recipe/__init__.py create mode 100644 pangeo_forge/recipe/pipeline.py create mode 100644 pangeo_forge/recipe/target.py create mode 100644 tests/__init__.py create mode 100644 tests/fixtures.py create mode 100644 tests/test_recipe.py diff --git a/pangeo_forge/__init__.py b/pangeo_forge/__init__.py index a9528c6e..722da05d 100644 --- a/pangeo_forge/__init__.py +++ b/pangeo_forge/__init__.py @@ -1,6 +1,6 @@ from pkg_resources import DistributionNotFound, get_distribution -from pangeo_forge.pipelines import AbstractPipeline +#from pangeo_forge.pipelines import AbstractPipeline try: __version__ = get_distribution(__name__).version @@ -10,7 +10,7 @@ del get_distribution, DistributionNotFound - -__all__ = [ - "AbstractPipeline", -] +# +# __all__ = [ +# "AbstractPipeline", +# ] diff --git a/pangeo_forge/pipelines/__init__.py b/pangeo_forge/pipelines/__init__.py index 122c0873..e69de29b 100644 --- a/pangeo_forge/pipelines/__init__.py +++ b/pangeo_forge/pipelines/__init__.py @@ -1,5 +0,0 @@ -from .base import AbstractPipeline - -__all__ = [ - "AbstractPipeline", -] diff --git a/pangeo_forge/recipe/__init__.py b/pangeo_forge/recipe/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/pangeo_forge/recipe/pipeline.py b/pangeo_forge/recipe/pipeline.py new file mode 100644 index 00000000..e69de29b diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe/recipe.py index 266b0506..3cc86180 100644 --- a/pangeo_forge/recipe/recipe.py +++ b/pangeo_forge/recipe/recipe.py @@ -4,25 +4,161 @@ import xarray as xr import fsspec +from ..utils import chunked_iterable +from .target import Target +### How to manually execute a recipe: ### +# +# r = PangeoForgeTarget() +# r = MyRecipe(**opts) # 1 +# r.set_target(tmp_dir) # 2 +# # manual execution of recipe +# r.prepare() # 3 +# for input_key in r.iter_inputs(): +# r.cache_input(input_key) # 4 +# for chunk_key in r.iter_chunks(): +# r.store_chunk(chunk_key) # 5 +# r.finalize() # 6 +# +### -class Recipe: - datasets = [] - keys = [] - def filenames_for_chunk(**chunk_keys): - raise NotImplementedError +# 1) Initialize the Recipe object +# 2) Point the Recipe at its Target +# 3) Initialize the recipe. +# Check if the target exists; if not, create it. +# + + +#@dataclass +class DatasetRecipe(): + target: Target + + +class FSSpecInputOpenerMixin: def open_file(self, fname): # todo: caching return fsspec.open(fname) - def open_chunk(self, filenames): + +class XarrayChunkOpenerMixin: + + def open_chunk(self, filenames, load=True): files_to_open = [self.open_file(f) for f in filenames] ds_chunk = xr.open_mfdataset(filenames, **self.open_chunk_kwargs) + if load: + ds_chunk.load() return ds_chunk - def store_chunk(self, store_target, **chunk_keys): - filenames = self.filenames_for_chunk(**chunk_keys) + +class ZarrWriterMixin: + + + def store_chunk(self, chunk_key): + filenames = self.filenames_for_chunk(chunk_key) ds_chunk = self.open_chunk(filenames) - ds_chunk.to_zarr(store_target, mode='a') + target_mapper = self.target.get_mapper() + write_region = self.get_write_region(chunk_key) + ds_chunk.to_zarr(target_mapper, region=write_region, **target_kwargs) + + + def open_target(self): + target_mapper = self.target.get_mapper() + return xr.open_zarr(target_mapper) + + + + + +class FileSequenceRecipe(DatasetRecipe): + + + def __init__(self, file_urls, sequence_dim, files_per_chunk=1, nitems_per_file=1, chunksize_within_file=None): + self.file_urls = file_urls + + if chunksize_within_file: + assert file_per_chunk is None + assert nitems_per_file >= chunksize_within_file + + self.files_per_chunk = files_per_chunk + self.nitems_per_file = nitems_per_file + self.sequence_dim = sequence_dim + + # mapping between chunks and file names + self._chunks_files = {k: v for k, v in + enumerate(chunked_iterable(file_urls, files_per_chunk))} + + + def filenames_for_chunk(self, chunk_key): + return self._chunks_files[chunk_key] + + + def nitems_for_chunk(self, chunk_key): + return self.nitems_per_file * len(self.filenames_for_chunk(chunk_key)) + + + def region_for_chunk(self, chunk_key): + # return a dict suitable to pass to xr.to_zarr(region=...) + # specifies where in the overall array to put this chunk's data + stride = self.nitems_per_file * self.files_per_chunk + start = chunk_key * stride + return { + self.sequence_dim: + slice(start, start + self.nitems_for_chunk(chunk_key)) + } + + + def sequence_dim(self): + # tells the total size of dataset along the sequence dimension + return { + self.sequence_dim: + sum([self.nitems_for_chunk(k) for k in self.iter_chunks()]) + } + + + def sequence_chunks(self): + # chunking + return {self.sequence_dim: self.files_per_chunk * self.nitems_per_file} + + + def iter_chunks(self): + for k in self._chunks_files: + yield k + + def prepare(self): + + target_store = self.get_store_target() + + try: + ds = self.open_target(target_store) + + except IOError: + first_chunk_key = next(self.iter_chunks()) + ds = self.open_chunk(first_chunk_key).chunk() + ds.to_zarr(path, compute=False, consolidated=False) + + encoding = {v: ds[v].encoding for v in ds} + + # now resize the sequence dim at the zarr level + sequence_axes = {v: ds[v].get_axis_num(self.sequence_dim) + for v in ds + if self.sequence_dim in ds[v].dims} + N = self.sequence_dim() + + zgroup = zarr.open_group(target_store) + + for v, axis in sequence_axes.items(): + arr = zgroup[v] + shape = list(arr.shape) + shape[axis] = N + arr.resize(shape) + + +class StandardSequentialRecipe( + FileSequenceRecipe, + FSSpecInputOpenerMixin, + XarrayChunkOpenerMixin, + ZarrWriterMixin, + ): + pass diff --git a/pangeo_forge/recipe/target.py b/pangeo_forge/recipe/target.py new file mode 100644 index 00000000..b7274d94 --- /dev/null +++ b/pangeo_forge/recipe/target.py @@ -0,0 +1,22 @@ +from dataclasses import dataclass +import fsspec + + +@dataclass +class Target: + """Representation of a storage target for Pangeo Forge. + Attributes + ---------- + url : FileSystemSpec.AbtractFileSystem + The filesystem we are writing to. Should be instantiated outside this + class. + path : str + The path where the target data will be saved. + """ + + fs: fsspec.AbstractFileSystem + path: str + + def get_mapper(self): + # don't want to use this because we want to use a fancier Zarr FSStore + return self.fs.get_mapper(self.path) diff --git a/pangeo_forge/utils.py b/pangeo_forge/utils.py index f4b76625..c49b3014 100644 --- a/pangeo_forge/utils.py +++ b/pangeo_forge/utils.py @@ -1,8 +1,4 @@ import itertools -from typing import Any, List, Tuple - -from prefect import task - # https://alexwlchan.net/2018/12/iterating-in-fixed-size-chunks/ def chunked_iterable(iterable, size): @@ -12,17 +8,3 @@ def chunked_iterable(iterable, size): if not chunk: break yield chunk - - -@task -def chunk(sources: List[Any], size: int) -> List[Tuple[Any, ...]]: - """ - Prefect task to chunk a list of sources into batches. - - Examples - -------- - >>> import pangeo_forge.utils - >>> pangeo_forge.utils.chunk.run([1, 2, 3, 4, 5], size=2) - [(1, 2), (3, 4), (5,)] - """ - return list(chunked_iterable(sources, size)) diff --git a/requirements.txt b/requirements.txt index c6a2743f..dbf84c5f 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,7 +2,6 @@ setuptools click dask distributed -prefect xarray zarr fsspec[http] diff --git a/tests/__init__.py b/tests/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/tests/fixtures.py b/tests/fixtures.py new file mode 100644 index 00000000..0e54d899 --- /dev/null +++ b/tests/fixtures.py @@ -0,0 +1,88 @@ +import subprocess +import time + +import fsspec +import numpy as np +import pandas as pd +import pytest +import xarray as xr + + + +# where to run the http server +_PORT = "8080" +_ADDRESS = "127.0.0.1" + + +@pytest.fixture(scope="session") +def daily_xarray_dataset(): + """Return a synthetic random xarray dataset.""" + np.random.seed(1) + nt, ny, nx = 10, 18, 36 + time = pd.date_range(start="2010-01-01", periods=nt, freq="D") + lon = (np.arange(nx) + 0.5) * 360 / nx + lon_attrs = {"units": "degrees_east", "long_name": "longitude"} + lat = (np.arange(ny) + 0.5) * 180 / ny + lat_attrs = {"units": "degrees_north", "long_name": "latitude"} + foo = np.random.rand(nt, ny, nx) + foo_attrs = {"long_name": "Fantastic Foo"} + bar = np.random.rand(nt, ny, nx) + bar_attrs = {"long_name": "Beautiful Bar"} + dims = ("time", "lat", "lon") + ds = xr.Dataset( + {"foo": (dims, foo, foo_attrs), "bar": (dims, bar, bar_attrs)}, + coords={ + "time": ("time", time), + "lat": ("lat", lat, lat_attrs), + "lon": ("lon", lon, lon_attrs), + }, + attrs={"conventions": "CF 1.6"}, + ) + return ds + + +@pytest.fixture(scope="session", params=["D", "2D"]) +def netcdf_local_paths(daily_xarray_dataset, tmpdir_factory, request): + """Return a list of paths pointing to netcdf files.""" + tmp_path = tmpdir_factory.mktemp("netcdf_data") + gb = daily_xarray_dataset.resample(time=request.param) + _, datasets = zip(*gb) + fnames = [f"{n:03d}.nc" for n in range(len(datasets))] + paths = [tmp_path.join(fname) for fname in fnames] + print(len(paths)) + xr.save_mfdataset(datasets, [str(path) for path in paths]) + return paths + + +@pytest.fixture(scope="session") +def netcdf_http_server(netcdf_local_paths): + first_path = netcdf_local_paths[0] + # assume that all files are in the same directory + basedir = first_path.dirpath() + print(basedir) + fnames = [path.basename for path in netcdf_local_paths] + + # this feels very hacky + command_list = ["python", "-m", "http.server", _PORT, "--bind", _ADDRESS] + p = subprocess.Popen(command_list, cwd=basedir) + url = f"http://{_ADDRESS}:{_PORT}" + time.sleep(0.1) # let the server start up + yield url, fnames + p.kill() + + +# tests that our fixtures work + + +def test_fixture_local_files(daily_xarray_dataset, netcdf_local_paths): + paths = [str(path) for path in netcdf_local_paths] + ds = xr.open_mfdataset(paths, combine="nested", concat_dim="time") + assert ds.identical(daily_xarray_dataset) + + +def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): + url, paths = netcdf_http_server + urls = ["/".join([url, str(path)]) for path in paths] + open_files = [fsspec.open(url).open() for url in urls] + ds = xr.open_mfdataset(open_files, combine="nested", concat_dim="time") + assert ds.identical(daily_xarray_dataset) diff --git a/tests/test_recipe.py b/tests/test_recipe.py new file mode 100644 index 00000000..7ca003a6 --- /dev/null +++ b/tests/test_recipe.py @@ -0,0 +1,48 @@ +import pytest + +from pangeo_forge.recipe import recipe + +from .fixtures import daily_xarray_dataset, netcdf_local_paths + +dummy_fnames = ["a.nc", "b.nc", "c.nc"] +@pytest.mark.parametrize( + "file_urls, files_per_chunk, expected_keys, expected_filenames", + [ + (dummy_fnames, 1, [0, 1, 2], [("a.nc",), ("b.nc",), ("c.nc",)]), + (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",)]) + ] +) +def test_file_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames): + + r = recipe.FileSequenceRecipe( + file_urls=file_urls, + sequence_dim="time", + files_per_chunk=files_per_chunk + ) + + chunk_keys = list(r.iter_chunks()) + assert chunk_keys == expected_keys + + for k, expected in zip(r.iter_chunks(), expected_filenames): + fnames = r.filenames_for_chunk(k) + assert fnames == expected + + +def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_dir): + + r = StandardSequentialRecipe( + file_urls=netcdf_local_paths, + sequence_dim='time', + ) + + r.set_target(tmp_dir) + # manual execution of recipe + r.prepare() + for input_key in r.iter_inputs(): + r.cache_input(input_key) + for chunk_key in r.iter_chunks(): + r.store_chunk(chunk_key) + r.finalize() + + ds_target = xr.open_dataset(tmp_dir) + assert ds_target.identical(daily_xarray_dataset) diff --git a/tests/test_utils.py b/tests/test_utils.py index 1b4b8d74..03e6be7f 100644 --- a/tests/test_utils.py +++ b/tests/test_utils.py @@ -1,6 +1,17 @@ +import pytest + import pangeo_forge.utils -def test_chunk(): - result = pangeo_forge.utils.chunk.run([1, 2, 3], 2) - assert result == [(1, 2), (3,)] +@pytest.mark.parametrize( + "iterable, size, expected", + [ + ([1, 2, 3], 1, [(1,), (2,), (3,)]), + ([1, 2, 3], 2, [(1, 2), (3,)]), + ([1, 2, 3], 3, [(1, 2, 3,)]), + ([1, 2, 3], 4, [(1, 2, 3,)]) + ] +) +def test_chunked_iterable(iterable, size, expected): + actual = list(pangeo_forge.utils.chunked_iterable(iterable, size)) + assert actual == expected From f980e8f8cff0f01b72ad4db1cd0ae36214b02c22 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Sat, 28 Nov 2020 22:08:05 -0500 Subject: [PATCH 03/34] made target fixture --- pangeo_forge/recipe/recipe.py | 34 +++++++++++++++++++--------------- tests/fixtures.py | 19 ++++++++++++++++++- tests/test_recipe.py | 3 ++- 3 files changed, 39 insertions(+), 17 deletions(-) diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe/recipe.py index 3cc86180..38c67d95 100644 --- a/pangeo_forge/recipe/recipe.py +++ b/pangeo_forge/recipe/recipe.py @@ -2,11 +2,16 @@ A Pangeo Forge Recipe """ +from dataclasses import dataclass + import xarray as xr import fsspec from ..utils import chunked_iterable from .target import Target +from typing import Optional, Iterable + + ### How to manually execute a recipe: ### # # r = PangeoForgeTarget() @@ -27,10 +32,15 @@ # 2) Point the Recipe at its Target # 3) Initialize the recipe. # Check if the target exists; if not, create it. -# +# 4) cache the inputs to proximate storage (OPTIONAL) +# Some recipes won't need this (e.g. cloud to cloud) +# If so, iter_inputs is just an empty iterator +# 5) Load each chunk from the inputs and store it in the target +# Might be coming from the cache or might be read directly. +# 6) -#@dataclass +@dataclass class DatasetRecipe(): target: Target @@ -71,23 +81,17 @@ def open_target(self): +@dataclass class FileSequenceRecipe(DatasetRecipe): + file_urls: Iterable[str] + sequence_dim: str + files_per_chunk: int = 1 + nitems_per_file: int = 1 - def __init__(self, file_urls, sequence_dim, files_per_chunk=1, nitems_per_file=1, chunksize_within_file=None): - self.file_urls = file_urls - - if chunksize_within_file: - assert file_per_chunk is None - assert nitems_per_file >= chunksize_within_file - - self.files_per_chunk = files_per_chunk - self.nitems_per_file = nitems_per_file - self.sequence_dim = sequence_dim - - # mapping between chunks and file names + def __post_init__(self): self._chunks_files = {k: v for k, v in - enumerate(chunked_iterable(file_urls, files_per_chunk))} + enumerate(chunked_iterable(self.file_urls, self.files_per_chunk))} def filenames_for_chunk(self, chunk_key): diff --git a/tests/fixtures.py b/tests/fixtures.py index 0e54d899..7f0f1d92 100644 --- a/tests/fixtures.py +++ b/tests/fixtures.py @@ -7,7 +7,7 @@ import pytest import xarray as xr - +from pangeo_forge.recipe.target import Target # where to run the http server _PORT = "8080" @@ -71,6 +71,14 @@ def netcdf_http_server(netcdf_local_paths): p.kill() +@pytest.fixture() +def tmp_target(tmpdir_factory): + import fsspec + fs = fsspec.get_filesystem_class("file")() + path = str(tmpdir_factory.mktemp("target")) + return Target(fs, path) + + # tests that our fixtures work @@ -86,3 +94,12 @@ def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): open_files = [fsspec.open(url).open() for url in urls] ds = xr.open_mfdataset(open_files, combine="nested", concat_dim="time") assert ds.identical(daily_xarray_dataset) + + +def test_target(tmp_target): + mapper = tmp_target.get_mapper() + mybytes = b'bar' + mapper['foo'] = b'bar' + with open(tmp_target.path + '/foo') as f: + res = f.read() + assert res == 'bar' diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 7ca003a6..b797d73f 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -17,7 +17,8 @@ def test_file_sequence_recipe(file_urls, files_per_chunk, expected_keys, expecte r = recipe.FileSequenceRecipe( file_urls=file_urls, sequence_dim="time", - files_per_chunk=files_per_chunk + files_per_chunk=files_per_chunk, + target= ) chunk_keys = list(r.iter_chunks()) From 0f311415f0802121ec2a85c66c015bcfcc7df99f Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Sat, 28 Nov 2020 22:09:25 -0500 Subject: [PATCH 04/34] made target fixture --- tests/test_recipe.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/test_recipe.py b/tests/test_recipe.py index b797d73f..11a0f41d 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -2,7 +2,7 @@ from pangeo_forge.recipe import recipe -from .fixtures import daily_xarray_dataset, netcdf_local_paths +from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_target dummy_fnames = ["a.nc", "b.nc", "c.nc"] @pytest.mark.parametrize( @@ -12,13 +12,13 @@ (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",)]) ] ) -def test_file_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames): +def test_file_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames, tmp_target): r = recipe.FileSequenceRecipe( file_urls=file_urls, sequence_dim="time", files_per_chunk=files_per_chunk, - target= + target=tmp_target ) chunk_keys = list(r.iter_chunks()) @@ -29,7 +29,7 @@ def test_file_sequence_recipe(file_urls, files_per_chunk, expected_keys, expecte assert fnames == expected -def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_dir): +def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target): r = StandardSequentialRecipe( file_urls=netcdf_local_paths, From 15e240d2d19967534510ae9a3448965954f86b8f Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Sun, 29 Nov 2020 22:24:59 -0500 Subject: [PATCH 05/34] spaghetti at this point --- pangeo_forge/recipe/recipe.py | 120 ++++++++++++++++++++++++++------- pangeo_forge/recipe/storage.py | 59 ++++++++++++++++ pangeo_forge/recipe/target.py | 22 ------ 3 files changed, 154 insertions(+), 47 deletions(-) create mode 100644 pangeo_forge/recipe/storage.py delete mode 100644 pangeo_forge/recipe/target.py diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe/recipe.py index 38c67d95..142fe849 100644 --- a/pangeo_forge/recipe/recipe.py +++ b/pangeo_forge/recipe/recipe.py @@ -9,14 +9,13 @@ from ..utils import chunked_iterable from .target import Target -from typing import Optional, Iterable +from typing import Optional, Iterable, Callable ### How to manually execute a recipe: ### # -# r = PangeoForgeTarget() -# r = MyRecipe(**opts) # 1 -# r.set_target(tmp_dir) # 2 +# t = PangeoForgeTarget() +# r = MyRecipe(target=t, **opts) # 1 # # manual execution of recipe # r.prepare() # 3 # for input_key in r.iter_inputs(): @@ -44,39 +43,110 @@ class DatasetRecipe(): target: Target + def prepare(self): + pass + + def iter_inputs(self): + return [] + + def cache_input(self, input_key): + raise NotImplementedError + + def iter_chunks(self): + raise NotImplementedError + + def store_chunk(self, chunk_key): + raise NotImplementedError + + def finalize(self): + raise NotImplementedError + + +@dataclass +class FSSpecFileOpenerMixin: + input_open_kwargs: dict = {} + + @contextmanager + def input_opener(self, fname): + with fsspec.open(fname, **input_open_kwargs) as f: + yield f + -class FSSpecInputOpenerMixin: +@dataclass +class InputCachingMixin(FSSpecFileOpenerMixin): + input_cache: InputCache + require_cache: bool -> False + + # returns a function that takes one input, the input_key + # this allows us to parallelize these operations + @property + def cache_input(self): - def open_file(self, fname): - # todo: caching - return fsspec.open(fname) + def cache_func(fname: str) -> None: + with super().input_opener(fname, mode="rb") as source: + with self.input_cache.open(fname, mode="wb") as target: + target.write(source.read()) + return cache_func -class XarrayChunkOpenerMixin: + @contextmanager + def input_opener(self, fname): + if self.input_cache.exists(fname): + with self.input_chache.open(fname, mode='rb') as f: + yield f + elif self.require_cache: + raise IOError("Input can only be opened from cache. Call .cache_input first.") + else: + # This will bypass the cache. May be slow. + with super().input_opener(fname, mode="rb") as f: + yield f - def open_chunk(self, filenames, load=True): - files_to_open = [self.open_file(f) for f in filenames] - ds_chunk = xr.open_mfdataset(filenames, **self.open_chunk_kwargs) - if load: - ds_chunk.load() - return ds_chunk + + +@dataclass +class XarrayInputOpener: + xarray_open_kwargs: Any + + def open_input(self, fname): + with self.input_opener(fname) as f: + ds = xr.open_dataset(f, **self.xarray_open_kwargs) + return ds + + +@dataclass +class XarrayConcatChunkOpener(XarrayInputOpener): + xarray_concat_kwargs: Any + + def open_chunk(self, chunk_key): + inputs = self.inputs_for_chunk(chunk_key) + dsets = [self.open_input(i) for i in inputs] + combined = xr.concat(dsets, **xarray_concat_kwargs) + # TODO: maybe do some chunking here? + return combined class ZarrWriterMixin: + @property + def store_chunk(self) -> Callable: + + def _store_chunk(chunk_key): + ds_chunk = self.open_chunk(chunk_key) + target_mapper = self.target.get_mapper() + write_region = self.get_write_region(chunk_key) + ds_chunk.to_zarr(target_mapper, region=write_region) + + return _store_chunk - def store_chunk(self, chunk_key): - filenames = self.filenames_for_chunk(chunk_key) - ds_chunk = self.open_chunk(filenames) - target_mapper = self.target.get_mapper() - write_region = self.get_write_region(chunk_key) - ds_chunk.to_zarr(target_mapper, region=write_region, **target_kwargs) + @property + def open_target(self) -> Callable: - def open_target(self): - target_mapper = self.target.get_mapper() - return xr.open_zarr(target_mapper) + def _open_target(): + target_mapper = self.target.get_mapper() + return xr.open_zarr(target_mapper) + return _open_target @@ -94,7 +164,7 @@ def __post_init__(self): enumerate(chunked_iterable(self.file_urls, self.files_per_chunk))} - def filenames_for_chunk(self, chunk_key): + def inputs_for_chunk(self, chunk_key): return self._chunks_files[chunk_key] diff --git a/pangeo_forge/recipe/storage.py b/pangeo_forge/recipe/storage.py new file mode 100644 index 00000000..474597a9 --- /dev/null +++ b/pangeo_forge/recipe/storage.py @@ -0,0 +1,59 @@ +from dataclasses import dataclass +from contextlib import closing, contextmanager +import fsspec + + +@dataclass +class Target: + """Representation of a storage target for Pangeo Forge. + Attributes + ---------- + fs : FileSystemSpec.AbtractFileSystem + The filesystem we are writing to. Should be instantiated outside this + class. + path : str + The path where the target data will be saved. + """ + + fs: fsspec.AbstractFileSystem + path: str + + def get_mapper(self): + # don't want to use this because we want to use a fancier Zarr FSStore + return self.fs.get_mapper(self.path) + + +def _hash_path(path: str) -> str: + return str(hash(path)) + + +@dataclass +class InputCache: + """Representation of an intermediate storage location where remote files + Can be cached locally. + + Attributes + ---------- + fs : FileSystemSpec.AbtractFileSystem + The filesystem we are writing to. Should be instantiated outside this + class. + prefix : str + A path prepended to all paths. + """ + + fs: fsspec.AbstractFileSystem + prefix: str = "" + + def _full_path(self, path): + return os.path.join(self.prefix, _hash_path(path)) + + def exists(self, path): + return self.fs.exists(self._full_path(path)) + + def rm(self, path): + self.fs.rm(self._full_path(path)) + + @contextmanager + def open(self, path): + with self.fs.open(self._full_path(path)) as f: + yield f diff --git a/pangeo_forge/recipe/target.py b/pangeo_forge/recipe/target.py deleted file mode 100644 index b7274d94..00000000 --- a/pangeo_forge/recipe/target.py +++ /dev/null @@ -1,22 +0,0 @@ -from dataclasses import dataclass -import fsspec - - -@dataclass -class Target: - """Representation of a storage target for Pangeo Forge. - Attributes - ---------- - url : FileSystemSpec.AbtractFileSystem - The filesystem we are writing to. Should be instantiated outside this - class. - path : str - The path where the target data will be saved. - """ - - fs: fsspec.AbstractFileSystem - path: str - - def get_mapper(self): - # don't want to use this because we want to use a fancier Zarr FSStore - return self.fs.get_mapper(self.path) From 18a895a65e1037da863e4ecbaf7b7c5ea6370b04 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 15:34:05 -0500 Subject: [PATCH 06/34] working storage classes --- pangeo_forge/recipe/storage.py | 9 +++++++-- tests/fixtures.py | 20 +++++++++++++++++++- 2 files changed, 26 insertions(+), 3 deletions(-) diff --git a/pangeo_forge/recipe/storage.py b/pangeo_forge/recipe/storage.py index 474597a9..04e949a0 100644 --- a/pangeo_forge/recipe/storage.py +++ b/pangeo_forge/recipe/storage.py @@ -1,6 +1,7 @@ from dataclasses import dataclass from contextlib import closing, contextmanager import fsspec +import os @dataclass @@ -54,6 +55,10 @@ def rm(self, path): self.fs.rm(self._full_path(path)) @contextmanager - def open(self, path): - with self.fs.open(self._full_path(path)) as f: + def open(self, path, **kwargs): + with self.fs.open(self._full_path(path), **kwargs) as f: yield f + + def __post_init__(self): + if not self.fs.isdir(self.prefix): + self.fs.mkdir(self.prefix) diff --git a/tests/fixtures.py b/tests/fixtures.py index 7f0f1d92..28b2ead5 100644 --- a/tests/fixtures.py +++ b/tests/fixtures.py @@ -7,7 +7,7 @@ import pytest import xarray as xr -from pangeo_forge.recipe.target import Target +from pangeo_forge.recipe.storage import Target, InputCache # where to run the http server _PORT = "8080" @@ -79,6 +79,13 @@ def tmp_target(tmpdir_factory): return Target(fs, path) +@pytest.fixture() +def tmp_cache(tmpdir_factory): + path = str(tmpdir_factory.mktemp("cache")) + fs = fsspec.get_filesystem_class("file")() + cache = InputCache(fs, prefix='cache') + return cache + # tests that our fixtures work @@ -103,3 +110,14 @@ def test_target(tmp_target): with open(tmp_target.path + '/foo') as f: res = f.read() assert res == 'bar' + + +def test_cache(tmp_cache): + assert not tmp_cache.exists('foo') + with tmp_cache.open('foo', mode='w') as f: + f.write('bar') + assert tmp_cache.exists('foo') + with tmp_cache.open('foo', mode='r') as f: + assert f.read() == 'bar' + tmp_cache.rm('foo') + assert not tmp_cache.exists('foo') From 301206b4bc01565f7f2b1dc5e02d7477c4cc2840 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 16:39:13 -0500 Subject: [PATCH 07/34] recipe working pretty well --- pangeo_forge/recipe/recipe.py | 255 +++++++++++++++++++++++++--------- 1 file changed, 186 insertions(+), 69 deletions(-) diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe/recipe.py index 142fe849..25702df1 100644 --- a/pangeo_forge/recipe/recipe.py +++ b/pangeo_forge/recipe/recipe.py @@ -2,15 +2,21 @@ A Pangeo Forge Recipe """ -from dataclasses import dataclass +import logging +from dataclasses import dataclass, field +from contextlib import contextmanager +from typing import Optional, Iterable, Callable, Any +import numpy as np import xarray as xr import fsspec -from ..utils import chunked_iterable -from .target import Target +import zarr -from typing import Optional, Iterable, Callable +from ..utils import chunked_iterable +from .storage import Target, InputCache +#logger = logging.getLogger(__name__) +logger = logging.getLogger("recipe") ### How to manually execute a recipe: ### # @@ -42,48 +48,76 @@ @dataclass class DatasetRecipe(): target: Target + chunk_preprocess_funcs: Iterable[Callable] + @property def prepare(self): - pass + def _prepare(): + pass + return _prepare def iter_inputs(self): return [] - def cache_input(self, input_key): - raise NotImplementedError + # need to figure out what's going on with these methods and inheritance + # @property + # def cache_input(self): + # def _cache_input(input_key): + # raise NotImplementedError + # return _cache_input + + # this only gets run when iterating, not preparing! + def preprocess_chunk(self, ds): + for f in self.chunk_preprocess_funcs: + ds = f(ds) + return ds def iter_chunks(self): raise NotImplementedError - def store_chunk(self, chunk_key): - raise NotImplementedError + # @property + # def store_chunk(self): + # def _store_chunk(chunk_key): + # raise NotImplementedError + # return _store_chunk - def finalize(self): - raise NotImplementedError + # @property + # def finalize(self): + # + # def _finalize(): + # pass + # return _finalize +# Notes about dataclasses: +# - https://www.python.org/dev/peps/pep-0557/#inheritance +# - https://stackoverflow.com/questions/51575931/class-inheritance-in-python-3-7-dataclasses +# This means that, for now, I can't get default arguments to work. @dataclass class FSSpecFileOpenerMixin: - input_open_kwargs: dict = {} + #input_open_kwargs: dict #= field(default_factory=dict) @contextmanager - def input_opener(self, fname): - with fsspec.open(fname, **input_open_kwargs) as f: + def input_opener(self, fname, **kwargs): + logger.info(f"Opening input '{fname}'") + with fsspec.open(fname, **kwargs) as f: yield f @dataclass class InputCachingMixin(FSSpecFileOpenerMixin): + require_cache: bool #= False input_cache: InputCache - require_cache: bool -> False # returns a function that takes one input, the input_key # this allows us to parallelize these operations @property def cache_input(self): + opener = super().input_opener def cache_func(fname: str) -> None: - with super().input_opener(fname, mode="rb") as source: + logger.info(f"Caching input '{fname}'") + with opener(fname, mode="rb") as source: with self.input_cache.open(fname, mode="wb") as target: target.write(source.read()) @@ -92,11 +126,14 @@ def cache_func(fname: str) -> None: @contextmanager def input_opener(self, fname): if self.input_cache.exists(fname): - with self.input_chache.open(fname, mode='rb') as f: + logger.info(f"Input '{fname}' found in cache") + with self.input_cache.open(fname, mode='rb') as f: yield f elif self.require_cache: + # this creates an error on prepare because nothing is cached raise IOError("Input can only be opened from cache. Call .cache_input first.") else: + logger.info(f"Input '{fname}' not found in cache. Opening directly.") # This will bypass the cache. May be slow. with super().input_opener(fname, mode="rb") as f: yield f @@ -105,26 +142,39 @@ def input_opener(self, fname): @dataclass class XarrayInputOpener: - xarray_open_kwargs: Any + xarray_open_kwargs: dict def open_input(self, fname): - with self.input_opener(fname) as f: - ds = xr.open_dataset(f, **self.xarray_open_kwargs) + with self.input_opener(fname) as f: + logger.info(f"Opening input with Xarray '{fname}'") + ds = xr.open_dataset(f, **self.xarray_open_kwargs).load() + # do we always want to remove encoding? I think so. + ds = _fix_scalar_attr_encoding(ds) + logger.debug(f"{ds}") return ds @dataclass class XarrayConcatChunkOpener(XarrayInputOpener): - xarray_concat_kwargs: Any + xarray_concat_kwargs: dict def open_chunk(self, chunk_key): + logger.info(f"Concatenating inputs for chunk '{chunk_key}'") inputs = self.inputs_for_chunk(chunk_key) dsets = [self.open_input(i) for i in inputs] - combined = xr.concat(dsets, **xarray_concat_kwargs) + # CONCAT DELETES ENCODING!!! + ds = xr.concat(dsets, self.sequence_dim, **self.xarray_concat_kwargs) + logger.debug(f"{ds}") + + # do we really want to just delete all encoding? + #for v in ds.variables: + # ds[v].encoding = {} + # TODO: maybe do some chunking here? - return combined + return ds +@dataclass class ZarrWriterMixin: @property @@ -132,50 +182,99 @@ def store_chunk(self) -> Callable: def _store_chunk(chunk_key): ds_chunk = self.open_chunk(chunk_key) + ds_chunk = self.preprocess_chunk(ds_chunk) target_mapper = self.target.get_mapper() - write_region = self.get_write_region(chunk_key) + write_region = self.region_for_chunk(chunk_key) + logger.info(f"Storing chunk '{chunk_key}' to Zarr region {write_region}") ds_chunk.to_zarr(target_mapper, region=write_region) return _store_chunk - @property - def open_target(self) -> Callable: + def open_target(self): + target_mapper = self.target.get_mapper() + return xr.open_zarr(target_mapper) - def _open_target(): - target_mapper = self.target.get_mapper() - return xr.open_zarr(target_mapper) - return _open_target + def initialize_target(self, ds, **expand_dims): + logger.info(f"Creating a new dataset in target") + target_mapper = self.target.get_mapper() + ds.to_zarr(target_mapper, mode='w', compute=False) + + + def expand_target_dim(self, dim, dimsize): + target_mapper = self.target.get_mapper() + zgroup = zarr.open_group(target_mapper) + + ds = self.open_target() + sequence_axes = {v: ds[v].get_axis_num(dim) + for v in ds.variables + if dim in ds[v].dims} + + for v, axis in sequence_axes.items(): + arr = zgroup[v] + shape = list(arr.shape) + shape[axis] = dimsize + arr.resize(shape) @dataclass -class FileSequenceRecipe(DatasetRecipe): - file_urls: Iterable[str] +class ZarrConsolidatorMixin(): + consolidate_zarr: bool #= True + + @property + def finalize(self): + + def _finalize(): + if self.consolidate_zarr: + logger.info(f"Consolidating Zarr metadata") + target_mapper = self.target.get_mapper() + zarr.consolidate_metadata(target_mapper) + + return _finalize + + +@dataclass +class SequenceRecipe(DatasetRecipe): + input_urls: Iterable[str] sequence_dim: str - files_per_chunk: int = 1 - nitems_per_file: int = 1 + inputs_per_chunk: int = 1 + nitems_per_input: int = 1 def __post_init__(self): - self._chunks_files = {k: v for k, v in - enumerate(chunked_iterable(self.file_urls, self.files_per_chunk))} + self._chunks_inputs = {k: v for k, v in + enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk))} + + def drop_vars(ds): + # writing a region means that all the variables MUST have sequence_dim + to_drop = [v for v in ds.variables + if self.sequence_dim not in ds[v].dims] + return ds.drop(to_drop) + + self.chunk_preprocess_funcs.append(drop_vars) def inputs_for_chunk(self, chunk_key): - return self._chunks_files[chunk_key] + return self._chunks_inputs[chunk_key] + + + def iter_inputs(self): + for chunk_key in self.iter_chunks(): + for input in self.inputs_for_chunk(chunk_key): + yield input def nitems_for_chunk(self, chunk_key): - return self.nitems_per_file * len(self.filenames_for_chunk(chunk_key)) + return self.nitems_per_input * len(self.inputs_for_chunk(chunk_key)) def region_for_chunk(self, chunk_key): # return a dict suitable to pass to xr.to_zarr(region=...) # specifies where in the overall array to put this chunk's data - stride = self.nitems_per_file * self.files_per_chunk + stride = self.nitems_per_input * self.inputs_per_chunk start = chunk_key * stride return { self.sequence_dim: @@ -183,56 +282,74 @@ def region_for_chunk(self, chunk_key): } - def sequence_dim(self): + def sequence_len(self): # tells the total size of dataset along the sequence dimension - return { - self.sequence_dim: - sum([self.nitems_for_chunk(k) for k in self.iter_chunks()]) - } + return sum([self.nitems_for_chunk(k) for k in self.iter_chunks()]) def sequence_chunks(self): # chunking - return {self.sequence_dim: self.files_per_chunk * self.nitems_per_file} + return {self.sequence_dim: self.inputs_per_chunk * self.nitems_per_input} def iter_chunks(self): - for k in self._chunks_files: + for k in self._chunks_inputs: yield k + @property def prepare(self): - target_store = self.get_store_target() - - try: - ds = self.open_target(target_store) + def _prepare(): - except IOError: - first_chunk_key = next(self.iter_chunks()) - ds = self.open_chunk(first_chunk_key).chunk() - ds.to_zarr(path, compute=False, consolidated=False) + target_store = self.target.get_mapper() - encoding = {v: ds[v].encoding for v in ds} + try: + ds = self.open_target() + logger.info(f"Found an existing dataset in target") + logger.debug(f"{ds}") + except (IOError, zarr.errors.GroupNotFoundError): + first_chunk_key = next(self.iter_chunks()) + ds = self.open_chunk(first_chunk_key).chunk() - # now resize the sequence dim at the zarr level - sequence_axes = {v: ds[v].get_axis_num(self.sequence_dim) - for v in ds - if self.sequence_dim in ds[v].dims} - N = self.sequence_dim() + # make sure the concat dim has a valid fill_value to avoid + # overruns when writing chunk + #ds[self.sequence_dim].encoding = {'_FillValue': 0} + # actually not necessary if we use decode_times=False + self.initialize_target(ds) - zgroup = zarr.open_group(target_store) + self.expand_target_dim(self.sequence_dim, self.sequence_len()) - for v, axis in sequence_axes.items(): - arr = zgroup[v] - shape = list(arr.shape) - shape[axis] = N - arr.resize(shape) + return _prepare +@dataclass class StandardSequentialRecipe( - FileSequenceRecipe, - FSSpecInputOpenerMixin, - XarrayChunkOpenerMixin, + SequenceRecipe, + InputCachingMixin, + XarrayConcatChunkOpener, ZarrWriterMixin, + ZarrConsolidatorMixin ): pass + + +# helper utilities + +# only needed because of +# https://github.com/pydata/xarray/issues/4631 +def _fix_scalar_attr_encoding(ds): + + def _fixed_attrs(d): + fixed = {} + for k, v in d.items(): + if isinstance(v, np.ndarray) and len(v) == 1: + fixed[k] = v[0] + return fixed + + ds = ds.copy() + ds.attrs.update(_fixed_attrs(ds.attrs)) + ds.encoding.update(_fixed_attrs(ds.encoding)) + for v in ds.variables: + ds[v].attrs.update(_fixed_attrs(ds[v].attrs)) + ds[v].encoding.update(_fixed_attrs(ds[v].encoding)) + return ds From a11fdc35923f1f7815cfe40ee9b1342db765447c Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 21:21:32 -0500 Subject: [PATCH 08/34] recipe tests pass --- pangeo_forge/recipe/recipe.py | 6 ++--- tests/fixtures.py | 2 ++ tests/test_recipe.py | 44 +++++++++++++++++++++++------------ 3 files changed, 34 insertions(+), 18 deletions(-) diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe/recipe.py index 25702df1..2c828500 100644 --- a/pangeo_forge/recipe/recipe.py +++ b/pangeo_forge/recipe/recipe.py @@ -175,7 +175,7 @@ def open_chunk(self, chunk_key): @dataclass -class ZarrWriterMixin: +class ZarrXarrayWriterMixin: @property def store_chunk(self) -> Callable: @@ -313,7 +313,7 @@ def _prepare(): # make sure the concat dim has a valid fill_value to avoid # overruns when writing chunk - #ds[self.sequence_dim].encoding = {'_FillValue': 0} + ds[self.sequence_dim].encoding = {'_FillValue': -1} # actually not necessary if we use decode_times=False self.initialize_target(ds) @@ -327,7 +327,7 @@ class StandardSequentialRecipe( SequenceRecipe, InputCachingMixin, XarrayConcatChunkOpener, - ZarrWriterMixin, + ZarrXarrayWriterMixin, ZarrConsolidatorMixin ): pass diff --git a/tests/fixtures.py b/tests/fixtures.py index 28b2ead5..4ac47d9f 100644 --- a/tests/fixtures.py +++ b/tests/fixtures.py @@ -45,6 +45,8 @@ def daily_xarray_dataset(): def netcdf_local_paths(daily_xarray_dataset, tmpdir_factory, request): """Return a list of paths pointing to netcdf files.""" tmp_path = tmpdir_factory.mktemp("netcdf_data") + items_per_file = {"D": 1, "2D": 2} + daily_xarray_dataset.attrs['items_per_file'] = items_per_file[request.param] gb = daily_xarray_dataset.resample(time=request.param) _, datasets = zip(*gb) fnames = [f"{n:03d}.nc" for n in range(len(datasets))] diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 11a0f41d..7b266473 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -1,8 +1,10 @@ import pytest +import xarray as xr +import zarr from pangeo_forge.recipe import recipe -from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_target +from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache dummy_fnames = ["a.nc", "b.nc", "c.nc"] @pytest.mark.parametrize( @@ -12,32 +14,43 @@ (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",)]) ] ) -def test_file_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames, tmp_target): +def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames, tmp_target): - r = recipe.FileSequenceRecipe( - file_urls=file_urls, + r = recipe.SequenceRecipe( + input_urls=file_urls, sequence_dim="time", - files_per_chunk=files_per_chunk, - target=tmp_target + inputs_per_chunk=files_per_chunk, + target=tmp_target, + chunk_preprocess_funcs=[], ) + assert r.sequence_len() == len(file_urls) + chunk_keys = list(r.iter_chunks()) assert chunk_keys == expected_keys for k, expected in zip(r.iter_chunks(), expected_filenames): - fnames = r.filenames_for_chunk(k) + fnames = r.inputs_for_chunk(k) assert fnames == expected -def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target): +def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): - r = StandardSequentialRecipe( - file_urls=netcdf_local_paths, - sequence_dim='time', + r = recipe.StandardSequentialRecipe( + consolidate_zarr=True, + xarray_open_kwargs={}, + xarray_concat_kwargs={}, + require_cache=False, + input_cache=tmp_cache, + target=tmp_target, + chunk_preprocess_funcs=[], + input_urls=netcdf_local_paths, + sequence_dim="time", + inputs_per_chunk=1, + nitems_per_input=daily_xarray_dataset.attrs['items_per_file'] ) - r.set_target(tmp_dir) - # manual execution of recipe + # this is the cannonical way to manually execute a recipe r.prepare() for input_key in r.iter_inputs(): r.cache_input(input_key) @@ -45,5 +58,6 @@ def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target): r.store_chunk(chunk_key) r.finalize() - ds_target = xr.open_dataset(tmp_dir) - assert ds_target.identical(daily_xarray_dataset) + ds_target = xr.open_zarr(tmp_target.get_mapper(), consolidated=True).load() + ds_expected = daily_xarray_dataset.compute() + assert ds_target.identical(ds_expected) From b1cc65b06b3356e41749300cdc35a833fcf0b640 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 21:24:07 -0500 Subject: [PATCH 09/34] prune old stuff --- pangeo_forge/pipelines/README.md | 8 - pangeo_forge/pipelines/__init__.py | 0 pangeo_forge/pipelines/base.py | 169 --------------------- pangeo_forge/pipelines/http_xarray_zarr.py | 35 ----- pangeo_forge/pipelines/job.yaml | 28 ---- pangeo_forge/pipelines/worker_pod.yaml | 41 ----- pangeo_forge/tasks/README.md | 8 - pangeo_forge/tasks/__init__.py | 0 pangeo_forge/tasks/http.py | 36 ----- pangeo_forge/tasks/xarray.py | 73 --------- pangeo_forge/tasks/zarr.py | 23 --- 11 files changed, 421 deletions(-) delete mode 100644 pangeo_forge/pipelines/README.md delete mode 100644 pangeo_forge/pipelines/__init__.py delete mode 100644 pangeo_forge/pipelines/base.py delete mode 100644 pangeo_forge/pipelines/http_xarray_zarr.py delete mode 100644 pangeo_forge/pipelines/job.yaml delete mode 100644 pangeo_forge/pipelines/worker_pod.yaml delete mode 100644 pangeo_forge/tasks/README.md delete mode 100644 pangeo_forge/tasks/__init__.py delete mode 100644 pangeo_forge/tasks/http.py delete mode 100644 pangeo_forge/tasks/xarray.py delete mode 100644 pangeo_forge/tasks/zarr.py diff --git a/pangeo_forge/pipelines/README.md b/pangeo_forge/pipelines/README.md deleted file mode 100644 index a1738027..00000000 --- a/pangeo_forge/pipelines/README.md +++ /dev/null @@ -1,8 +0,0 @@ -# `pangeo_forge.pipelines` - -A collection of Pangeo-forge _Pipelines_. - -## Details - -- An abstract Pipeline class is defined in `base.py`. -- A general file naming convention might be `{source}_{container}_{target}.py`. For example, a pipeline that downloads netCDF files from a remote http server and publishes a Zarr store would be `http_xarray_zarr.py`. The main class in each module would follow a similar convention, e.g. `HttpXarrayZarr`. diff --git a/pangeo_forge/pipelines/__init__.py b/pangeo_forge/pipelines/__init__.py deleted file mode 100644 index e69de29b..00000000 diff --git a/pangeo_forge/pipelines/base.py b/pangeo_forge/pipelines/base.py deleted file mode 100644 index ebf5d3ea..00000000 --- a/pangeo_forge/pipelines/base.py +++ /dev/null @@ -1,169 +0,0 @@ -""" -Base class for most pipelines. - -Design ------- - -To the extent possible, we want recipe maintainers to focus on the code -needed to express the data transformations. In particular, we don't -want them to worry about things like the execution environment and how -their code is loaded into it. - -The Flow -======== - -We use prefect_ to express ETL pipelines. It provides us a few things: - -1. A Highlevel API for writing data transformations. -2. A robust scheduling and orchestration system for executing pipelines. - -Every pangeo-forge recipe *must* have a ``prefect.Flow`` instance at the -top-level of their pipeline module. - - -.. code-block:: python - - class Pipeline(pangeo_forge.AbstractPipeline): - @property - def flow(self) -> Flow: - with Flow( - self.name, - environment=self.environment, - storage=self.storage - ) as flow: - # Your pipeline goes here. - ... - - return flow - - pipeline = Pipeline() - flow = pipeline.flow - -The ``storage`` keyword controls how your source code is loaded into -pangeo-forge, and the ``environment`` keyword controls where it's run. -""" -import textwrap -from abc import ABC, abstractmethod -from pathlib import Path -from typing import List - -from prefect import Flow -from prefect.environments import DaskKubernetesEnvironment, Environment -from prefect.environments.storage import Storage -from prefect.environments.storage.github import GitHub - -HERE = Path(__file__).parent.absolute() - - -def finalize_flow(cls): - # Nothing to see here, move along. - # OK, fine... this is a small hack to hide Prefect's orchestration stuff - # (environments, storage) from the recipe writer. Prefect currently - # requires that the module stored on GitHub have - # 1. A `Flow` instance at the top-level of the module. - # 2. A properly set `storage` and `environment`. - # Which would require this at the bottom of every python module - # >>> pipeline = MyPipeline() - # >>> flow = pipeline.flow - # >>> flow.storage = pipeline.storage - # >>> flow.environment = pipeline.environment - # Which is ugly. To avoid that, we just override how `.` works on - # AbstractPipeline. If we're getting `.flow`, this little decorator - # takes over and attaches the storage and environment. - orig_getattribute = cls.__getattribute__ - - def new_getattribute(self, name): - result = orig_getattribute(self, name) - if name == "flow": - result.storage = self.storage - result.environment = self.environment - return result - - cls.__getattribute__ = new_getattribute - return cls - - -@finalize_flow -class AbstractPipeline(ABC): - name = "AbstractPipeline" - path = "recipe/pipeline.py" - - @property - @abstractmethod - def repo(self): - """The GitHub repository containing the pipeline definition.""" - - @property - @abstractmethod - def sources(self) -> List[str]: - """A list of source URLs containing the original data.""" - pass - - @property - @abstractmethod - def targets(self) -> List[str]: - """A list of target URLs where the transformed data is written.""" - pass - - @abstractmethod - def flow(self) -> Flow: - """The """ - pass - - @property - def environment(self) -> Environment: - """ - The pipeline runtime environment. - - Returns - ------- - prefect.environments.Environment - An instance of a Prefect Environment. By default - a :class:`prefect.environments.DaskKubernetesEnvironment` - is used. - """ - scheduler_spec_file = str(HERE / "job.yaml") - worker_spec_file = str(HERE / "worker_pod.yaml") - - environment = DaskKubernetesEnvironment( - min_workers=1, - max_workers=30, - scheduler_spec_file=scheduler_spec_file, - worker_spec_file=worker_spec_file, - metadata=dict(image="pangeoforge/default-image"), - ) - return environment - - @property - def storage(self) -> Storage: - """ - The pipeline storage. - - Returns - ------- - prefect.environments.storage.Storage - By default a :class:`prefect.environments.storage.github.GitHub` - environment is used with ``self.repo`` as the repository - and ``self.path`` as the path. - """ - return GitHub(self.repo, path=self.path) - - def _generate_run(self, source): - name = type(self).__name__ - with open(source, encoding="utf-8") as f: - source = f.read() - - return textwrap.dedent( - """\ - # file: run.py - # generated by pangeo-forge. - - {source} - - # ---------------------------------------------------------------- - pipe = {name}() - flow = pipe.flow - flow.storage = pipe.storage - flow.environment = pipe.environment - """ - ).format(source=source, name=name) diff --git a/pangeo_forge/pipelines/http_xarray_zarr.py b/pangeo_forge/pipelines/http_xarray_zarr.py deleted file mode 100644 index 4b72703a..00000000 --- a/pangeo_forge/pipelines/http_xarray_zarr.py +++ /dev/null @@ -1,35 +0,0 @@ -from prefect import Flow - -from ..tasks.http import download -from ..tasks.xarray import combine_and_write -from ..tasks.zarr import consolidate_metadata -from ..utils import chunked_iterable - - -class HttpXarrayZarrMixin: - @property - def flow(self): - - if len(self.targets) == 1: - target = self.targets[0] - else: - raise ValueError("Zarr target requires self.targets be a length one list") - - with Flow(self.name) as _flow: - - cached_sources = [download(k, self.cache_location) for k in self.sources] - - write_tasks = [] - for source_group in chunked_iterable(cached_sources, self.files_per_chunk): - write_task = combine_and_write( - source_group, target, self.append_dim, self.concat_dim, - ) - write_tasks.append(write_task) - cm = consolidate_metadata(target, write_tasks) - - # create dependencies in imperative mode - for n in range(1, len(write_tasks)): - write_tasks[n].set_upstream(write_tasks[n - 1], flow=_flow) - cm.set_upstream(write_tasks[-1], flow=_flow) - - return _flow diff --git a/pangeo_forge/pipelines/job.yaml b/pangeo_forge/pipelines/job.yaml deleted file mode 100644 index 5fa08ca8..00000000 --- a/pangeo_forge/pipelines/job.yaml +++ /dev/null @@ -1,28 +0,0 @@ -apiVersion: batch/v1 -kind: Job -metadata: - name: prefect-dask-job - labels: - app: prefect-dask-job -spec: - template: - metadata: - labels: - app: prefect-dask-job - spec: - serviceAccountName: pangeo-forge - serviceAccount: pangeo-forge - containers: - - name: flow - imagePullPolicy: Always - command: ["/bin/sh", "-c"] - args: - [ - 'python -c "import prefect; prefect.environments.execution.load_and_run_flow()"', - ] - resources: - requests: - cpu: "100m" - limits: - cpu: "100m" - restartPolicy: Never diff --git a/pangeo_forge/pipelines/worker_pod.yaml b/pangeo_forge/pipelines/worker_pod.yaml deleted file mode 100644 index d7a16440..00000000 --- a/pangeo_forge/pipelines/worker_pod.yaml +++ /dev/null @@ -1,41 +0,0 @@ -kind: Pod -metadata: - labels: - app: prefect-dask-worker -spec: - restartPolicy: Never - serviceAccountName: pangeo-forge - serviceAccount: pangeo-forge - containers: - - image: pangeoforge/noaa-oisst-avhrr - imagePullPolicy: Always - args: [dask-worker, --no-bokeh, --death-timeout, '60'] - name: dask-worker - env: - - name: PREFECT__CLOUD__GRAPHQL - value: PREFECT__CLOUD__GRAPHQL - - name: PREFECT__CLOUD__AUTH_TOKEN - value: PREFECT__CLOUD__AUTH_TOKEN - - name: PREFECT__CONTEXT__FLOW_RUN_ID - value: PREFECT__CONTEXT__FLOW_RUN_ID - - name: PREFECT__CLOUD__USE_LOCAL_SECRETS - value: "false" - - name: PREFECT__ENGINE__FLOW_RUNNER__DEFAULT_CLASS - value: "prefect.engine.cloud.CloudFlowRunner" - - name: PREFECT__ENGINE__TASK_RUNNER__DEFAULT_CLASS - value: "prefect.engine.cloud.CloudTaskRunner" - - name: PREFECT__ENGINE__EXECUTOR__DEFAULT_CLASS - value: "prefect.engine.executors.DaskExecutor" - - name: PREFECT__LOGGING__LOG_TO_CLOUD - value: "true" - - name: PREFECT__LOGGING__LEVEL - value: "DEBUG" - - name: PREFECT__DEBUG - value: "true" - - name: PREFECT__LOGGING__EXTRA_LOGGERS - value: PREFECT__LOGGING__EXTRA_LOGGERS - resources: - requests: - cpu: "500m" - limits: - cpu: "500m" diff --git a/pangeo_forge/tasks/README.md b/pangeo_forge/tasks/README.md deleted file mode 100644 index 8a77c5e1..00000000 --- a/pangeo_forge/tasks/README.md +++ /dev/null @@ -1,8 +0,0 @@ -# `pangeo_forge.tasks` - -A collection of Prefect Tasks for Pangeo-Forge. - -## Details - -- This is simply a collection of Prefect Tasks. Much like https://docs.prefect.io/api/latest/#task-library or https://github.com/PrefectHQ/prefect/tree/master/src/prefect/tasks. -- This should be considered low-level public API. diff --git a/pangeo_forge/tasks/__init__.py b/pangeo_forge/tasks/__init__.py deleted file mode 100644 index e69de29b..00000000 diff --git a/pangeo_forge/tasks/http.py b/pangeo_forge/tasks/http.py deleted file mode 100644 index 32078d0c..00000000 --- a/pangeo_forge/tasks/http.py +++ /dev/null @@ -1,36 +0,0 @@ -import os - -import fsspec -from prefect import task - - -@task -def download(source_url, cache_location): - """ - Download a remote file to a cache. - - Parameters - ---------- - source_url : str - Path or url to the source file. - cache_location : str - Path or url to the target location for the source file. - - Returns - ------- - target_url : str - Path or url in the form of `{cache_location}/hash({source_url})`. - """ - target_url = os.path.join(cache_location, str(hash(source_url))) - - # there is probably a better way to do caching! - try: - fsspec.open(target_url).open() - return target_url - except FileNotFoundError: - pass - - with fsspec.open(source_url, mode="rb") as source: - with fsspec.open(target_url, mode="wb") as target: - target.write(source.read()) - return target_url diff --git a/pangeo_forge/tasks/xarray.py b/pangeo_forge/tasks/xarray.py deleted file mode 100644 index 20f65f2c..00000000 --- a/pangeo_forge/tasks/xarray.py +++ /dev/null @@ -1,73 +0,0 @@ -from typing import List - -import fsspec -import xarray as xr -from prefect import task - - -@task -def combine_and_write( - sources: List[str], target: str, append_dim: str, concat_dim: str -) -> List[str]: - """ - Write a batch of intermediate files to a combined zarr store. - - Parameters - ---------- - sources : List[str] - A list of URLs pointing to the intermediate files. - target : str - The URL for the target combined store. - append_dim : str - Name of the dimension of which datasets should be appended during write. - concat_dim : str - The dimension to concatenate along. - - Returns - ------- - target : str - The URL of the written combined Zarr store (same as target). - - Examples - -------- - >>> import pangeo_forge.tasks.xarray - >>> import fsspec - >>> import xarray as xr - >>> from prefect import Flow - - >>> # Load sample data into `sources`. - >>> ds = xr.tutorial.open_dataset('rasm').load() - >>> fs = fsspec.get_filesystem_class("memory")() - >>> dsets = ds.isel(time=slice(18)), ds.isel(time=slice(18, None)) - >>> for i, dset in enumerate(dsets): - ... as_bytes = dset.to_netcdf() - ... with fs.open(f"cache/{i}.nc", "wb") as f: - ... f.write(as_bytes) - - >>> sources = [f"memory://{dset}" for dset in fs.ls("cache")] - >>> with Flow("my-flow") as flow: - ... result = pangeo_forge.tasks.xarray.combine_and_write( - ... sources, "memory://target.zarr", concat_dim="time" - ... ) - >>> result - - - We can run that outside of a flow context with ``.run()`` - >>> pangeo_forge.tasks.xarray.combine_and_write.run( - ... sources, "memory://target.zarr", concat_dim="time" - ... ) - 'memory://target.zarr' - """ - double_open_files = [fsspec.open(url).open() for url in sources] - ds = xr.open_mfdataset(double_open_files, combine="nested", concat_dim=concat_dim) - # by definition, this should be a contiguous chunk - ds = ds.chunk({append_dim: len(sources)}) - mapper = fsspec.get_mapper(target) - - if not len(mapper): - # The first write, . - kwargs = dict(mode="w") - else: - kwargs = dict(mode="a", append_dim=append_dim) - ds.to_zarr(mapper, **kwargs) - return target diff --git a/pangeo_forge/tasks/zarr.py b/pangeo_forge/tasks/zarr.py deleted file mode 100644 index 9fd60149..00000000 --- a/pangeo_forge/tasks/zarr.py +++ /dev/null @@ -1,23 +0,0 @@ -from typing import List, Optional - -import fsspec -import zarr -from prefect import task - - -@task -def consolidate_metadata(target, writes: Optional[List[str]] = None) -> None: - """ - Consolidate the metadata the Zarr group at `target`. - - Parameters - ---------- - target : str - The URL for the (combined) Zarr group. - writes : list of strings, optional - The URLs the combined stores were written to. This is only a - parameter to introduce a dependency in the pipeline execution graph. - The actual value isn't used. - """ - mapper = fsspec.get_mapper(target) - zarr.consolidate_metadata(mapper) From 35e0c9f59d6e7aa64aaff5e4830c1705255d9fc8 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 21:34:11 -0500 Subject: [PATCH 10/34] big cleanup --- pangeo_forge/__init__.py | 5 - pangeo_forge/{recipe => }/recipe.py | 4 +- pangeo_forge/recipe/__init__.py | 0 pangeo_forge/recipe/pipeline.py | 0 pangeo_forge/{recipe => }/storage.py | 0 tests/fixtures.py | 2 +- tests/tasks/__init__.py | 0 tests/tasks/test_xarray.py | 36 ------ tests/test_pipeline.py | 161 --------------------------- tests/test_recipe.py | 2 +- 10 files changed, 4 insertions(+), 206 deletions(-) rename pangeo_forge/{recipe => }/recipe.py (99%) delete mode 100644 pangeo_forge/recipe/__init__.py delete mode 100644 pangeo_forge/recipe/pipeline.py rename pangeo_forge/{recipe => }/storage.py (100%) delete mode 100644 tests/tasks/__init__.py delete mode 100644 tests/tasks/test_xarray.py delete mode 100644 tests/test_pipeline.py diff --git a/pangeo_forge/__init__.py b/pangeo_forge/__init__.py index 722da05d..30bf1a56 100644 --- a/pangeo_forge/__init__.py +++ b/pangeo_forge/__init__.py @@ -9,8 +9,3 @@ pass del get_distribution, DistributionNotFound - -# -# __all__ = [ -# "AbstractPipeline", -# ] diff --git a/pangeo_forge/recipe/recipe.py b/pangeo_forge/recipe.py similarity index 99% rename from pangeo_forge/recipe/recipe.py rename to pangeo_forge/recipe.py index 2c828500..88a13b1b 100644 --- a/pangeo_forge/recipe/recipe.py +++ b/pangeo_forge/recipe.py @@ -12,7 +12,7 @@ import fsspec import zarr -from ..utils import chunked_iterable +from .utils import chunked_iterable from .storage import Target, InputCache #logger = logging.getLogger(__name__) @@ -252,7 +252,7 @@ def drop_vars(ds): # writing a region means that all the variables MUST have sequence_dim to_drop = [v for v in ds.variables if self.sequence_dim not in ds[v].dims] - return ds.drop(to_drop) + return ds.drop_vars(to_drop) self.chunk_preprocess_funcs.append(drop_vars) diff --git a/pangeo_forge/recipe/__init__.py b/pangeo_forge/recipe/__init__.py deleted file mode 100644 index e69de29b..00000000 diff --git a/pangeo_forge/recipe/pipeline.py b/pangeo_forge/recipe/pipeline.py deleted file mode 100644 index e69de29b..00000000 diff --git a/pangeo_forge/recipe/storage.py b/pangeo_forge/storage.py similarity index 100% rename from pangeo_forge/recipe/storage.py rename to pangeo_forge/storage.py diff --git a/tests/fixtures.py b/tests/fixtures.py index 4ac47d9f..1ee9427e 100644 --- a/tests/fixtures.py +++ b/tests/fixtures.py @@ -7,7 +7,7 @@ import pytest import xarray as xr -from pangeo_forge.recipe.storage import Target, InputCache +from pangeo_forge.storage import Target, InputCache # where to run the http server _PORT = "8080" diff --git a/tests/tasks/__init__.py b/tests/tasks/__init__.py deleted file mode 100644 index e69de29b..00000000 diff --git a/tests/tasks/test_xarray.py b/tests/tasks/test_xarray.py deleted file mode 100644 index ae22014a..00000000 --- a/tests/tasks/test_xarray.py +++ /dev/null @@ -1,36 +0,0 @@ -import fsspec -import xarray as xr -from prefect import Flow, Task - -import pangeo_forge.tasks.xarray - - -def test_combine_and_write(): - ds = xr.tutorial.open_dataset("rasm").load() - dsets = ds.isel(time=slice(18)), ds.isel(time=slice(18, None)) - fs = fsspec.get_filesystem_class("memory")() - - for i, dset in enumerate(dsets): - as_bytes = dset.to_netcdf() - - with fs.open(f"cache/{i}.nc", "wb") as f: - f.write(as_bytes) - - sources = [f"memory://{dset}" for dset in fs.ls("cache")] - - # In a flow context - - target = "memory://target.zarr" - with Flow("test") as flow: - result = pangeo_forge.tasks.xarray.combine_and_write( - sources, target, concat_dim="time", append_dim="time" - ) - assert isinstance(result, Task) - flow.validate() - - result = pangeo_forge.tasks.xarray.combine_and_write.run( - sources, target, concat_dim="time", append_dim="time" - ) - assert result == target - result = xr.open_zarr(fs.get_mapper("target.zarr")) - xr.testing.assert_equal(ds, result) diff --git a/tests/test_pipeline.py b/tests/test_pipeline.py deleted file mode 100644 index abfa20d7..00000000 --- a/tests/test_pipeline.py +++ /dev/null @@ -1,161 +0,0 @@ -"""Integration testing of pipelines.""" - -import subprocess -import time - -import fsspec -import numpy as np -import pandas as pd -import pytest -import xarray as xr - -# classes tested here -from pangeo_forge.pipelines.base import AbstractPipeline -from pangeo_forge.pipelines.http_xarray_zarr import HttpXarrayZarrMixin - -# where to run the http server -_PORT = "8080" -_ADDRESS = "127.0.0.1" - - -@pytest.fixture(scope="session") -def daily_xarray_dataset(): - """Return a synthetic random xarray dataset.""" - np.random.seed(1) - nt, ny, nx = 10, 18, 36 - time = pd.date_range(start="2010-01-01", periods=nt, freq="D") - lon = (np.arange(nx) + 0.5) * 360 / nx - lon_attrs = {"units": "degrees_east", "long_name": "longitude"} - lat = (np.arange(ny) + 0.5) * 180 / ny - lat_attrs = {"units": "degrees_north", "long_name": "latitude"} - foo = np.random.rand(nt, ny, nx) - foo_attrs = {"long_name": "Fantastic Foo"} - bar = np.random.rand(nt, ny, nx) - bar_attrs = {"long_name": "Beautiful Bar"} - dims = ("time", "lat", "lon") - ds = xr.Dataset( - {"foo": (dims, foo, foo_attrs), "bar": (dims, bar, bar_attrs)}, - coords={ - "time": ("time", time), - "lat": ("lat", lat, lat_attrs), - "lon": ("lon", lon, lon_attrs), - }, - attrs={"conventions": "CF 1.6"}, - ) - return ds - - -@pytest.fixture(scope="session", params=["D", "2D"]) -def netcdf_local_paths(daily_xarray_dataset, tmpdir_factory, request): - """Return a list of paths pointing to netcdf files.""" - tmp_path = tmpdir_factory.mktemp("netcdf_data") - gb = daily_xarray_dataset.resample(time=request.param) - _, datasets = zip(*gb) - fnames = [f"{n:03d}.nc" for n in range(len(datasets))] - paths = [tmp_path.join(fname) for fname in fnames] - print(len(paths)) - xr.save_mfdataset(datasets, [str(path) for path in paths]) - return paths - - -@pytest.fixture(scope="session") -def netcdf_http_server(netcdf_local_paths): - first_path = netcdf_local_paths[0] - # assume that all files are in the same directory - basedir = first_path.dirpath() - print(basedir) - fnames = [path.basename for path in netcdf_local_paths] - - # this feels very hacky - command_list = ["python", "-m", "http.server", _PORT, "--bind", _ADDRESS] - p = subprocess.Popen(command_list, cwd=basedir) - url = f"http://{_ADDRESS}:{_PORT}" - time.sleep(0.1) # let the server start up - yield url, fnames - p.kill() - - -# tests that our fixtures work - - -def test_fixture_local_files(daily_xarray_dataset, netcdf_local_paths): - paths = [str(path) for path in netcdf_local_paths] - ds = xr.open_mfdataset(paths, combine="nested", concat_dim="time") - assert ds.identical(daily_xarray_dataset) - - -def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): - url, paths = netcdf_http_server - urls = ["/".join([url, str(path)]) for path in paths] - open_files = [fsspec.open(url).open() for url in urls] - ds = xr.open_mfdataset(open_files, combine="nested", concat_dim="time") - assert ds.identical(daily_xarray_dataset) - - -# a pipeline to load that data - - -class MyPipeline(HttpXarrayZarrMixin, AbstractPipeline): - repo = "pangeo-forge/pangeo-forge" - - def __init__( - self, - name, - cache_path, - target_path, - concat_dim, - append_dim, - files_per_chunk, - url_base, - nfiles, - ): - self.name = name - self.cache_location = f"{cache_path}/{name}-cache/" - self.target_location = f"{target_path}/{name}.zarr" - self.append_dim = append_dim - self.concat_dim = concat_dim - self.files_per_chunk = files_per_chunk - - # needed to build up sources - self._url_base = url_base - self._nfiles = nfiles - - @property - def sources(self): - keys = range(self._nfiles) - source_url_pattern = self._url_base + "/{n:03d}.nc" - source_urls = [source_url_pattern.format(n=key) for key in keys] - return source_urls - - @property - def targets(self): - return [self.target_location] - - -# a basic pipeline test -def test_pipeline(daily_xarray_dataset, netcdf_http_server, tmpdir): - name = "TEST_DATASET" - cache_dir = tmpdir.mkdir("cache") - target_dir = tmpdir.mkdir("target") - concat_dim = "time" - append_dim = "time" - files_per_chunk = 5 - - url_base, paths = netcdf_http_server - nfiles = len(paths) - - pipeline = MyPipeline( - name, cache_dir, target_dir, concat_dim, append_dim, files_per_chunk, url_base, nfiles - ) - pipeline.flow.run() - - ds_test = xr.open_zarr(pipeline.targets[0]) - assert ds_test.identical(daily_xarray_dataset) - - -def test_storage_environment_set(): - pipeline = MyPipeline( - "name", "cache", "target", "concat", "append", "files_per_chunk", "url_base", "nfiles" - ) - assert pipeline.storage - assert pipeline.environment diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 7b266473..b91ea2c5 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -2,7 +2,7 @@ import xarray as xr import zarr -from pangeo_forge.recipe import recipe +from pangeo_forge import recipe from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache From 885634450a2d04c8e6a5ae5ea498ed995d530f24 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 21:36:34 -0500 Subject: [PATCH 11/34] lint and fix tests --- .github/workflows/main.yaml | 2 +- pangeo_forge/__init__.py | 2 +- pangeo_forge/recipe.py | 82 ++++++++++++++----------------------- pangeo_forge/storage.py | 5 ++- pangeo_forge/utils.py | 1 + 5 files changed, 37 insertions(+), 55 deletions(-) diff --git a/.github/workflows/main.yaml b/.github/workflows/main.yaml index f77811c3..ef75c257 100644 --- a/.github/workflows/main.yaml +++ b/.github/workflows/main.yaml @@ -29,7 +29,7 @@ jobs: path: ~/conda_pkgs_dir key: ${{ runner.os }}-conda-${{ env.CACHE_NUMBER }}-${{ hashFiles('ci/py${{ matrix.python-version }}.yml') }} - name: setup miniconda - uses: goanpeca/setup-miniconda@v1 + uses: conda-incubator/setup-miniconda@v2 with: activate-environment: pangeo-forge environment-file: ci/py${{ matrix.python-version }}.yml diff --git a/pangeo_forge/__init__.py b/pangeo_forge/__init__.py index 30bf1a56..7e3d3772 100644 --- a/pangeo_forge/__init__.py +++ b/pangeo_forge/__init__.py @@ -1,6 +1,6 @@ from pkg_resources import DistributionNotFound, get_distribution -#from pangeo_forge.pipelines import AbstractPipeline +# from pangeo_forge.pipelines import AbstractPipeline try: __version__ = get_distribution(__name__).version diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index 88a13b1b..d265be8e 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -3,19 +3,19 @@ """ import logging -from dataclasses import dataclass, field from contextlib import contextmanager -from typing import Optional, Iterable, Callable, Any +from dataclasses import dataclass, field +from typing import Any, Callable, Iterable, Optional +import fsspec import numpy as np import xarray as xr -import fsspec import zarr +from .storage import InputCache, Target from .utils import chunked_iterable -from .storage import Target, InputCache -#logger = logging.getLogger(__name__) +# logger = logging.getLogger(__name__) logger = logging.getLogger("recipe") ### How to manually execute a recipe: ### @@ -46,7 +46,7 @@ @dataclass -class DatasetRecipe(): +class DatasetRecipe: target: Target chunk_preprocess_funcs: Iterable[Callable] @@ -54,6 +54,7 @@ class DatasetRecipe(): def prepare(self): def _prepare(): pass + return _prepare def iter_inputs(self): @@ -69,7 +70,7 @@ def iter_inputs(self): # this only gets run when iterating, not preparing! def preprocess_chunk(self, ds): for f in self.chunk_preprocess_funcs: - ds = f(ds) + ds = f(ds) return ds def iter_chunks(self): @@ -88,14 +89,16 @@ def iter_chunks(self): # pass # return _finalize + # Notes about dataclasses: # - https://www.python.org/dev/peps/pep-0557/#inheritance # - https://stackoverflow.com/questions/51575931/class-inheritance-in-python-3-7-dataclasses # This means that, for now, I can't get default arguments to work. + @dataclass class FSSpecFileOpenerMixin: - #input_open_kwargs: dict #= field(default_factory=dict) + # input_open_kwargs: dict #= field(default_factory=dict) @contextmanager def input_opener(self, fname, **kwargs): @@ -106,7 +109,7 @@ def input_opener(self, fname, **kwargs): @dataclass class InputCachingMixin(FSSpecFileOpenerMixin): - require_cache: bool #= False + require_cache: bool # = False input_cache: InputCache # returns a function that takes one input, the input_key @@ -115,6 +118,7 @@ class InputCachingMixin(FSSpecFileOpenerMixin): def cache_input(self): opener = super().input_opener + def cache_func(fname: str) -> None: logger.info(f"Caching input '{fname}'") with opener(fname, mode="rb") as source: @@ -127,7 +131,7 @@ def cache_func(fname: str) -> None: def input_opener(self, fname): if self.input_cache.exists(fname): logger.info(f"Input '{fname}' found in cache") - with self.input_cache.open(fname, mode='rb') as f: + with self.input_cache.open(fname, mode="rb") as f: yield f elif self.require_cache: # this creates an error on prepare because nothing is cached @@ -139,7 +143,6 @@ def input_opener(self, fname): yield f - @dataclass class XarrayInputOpener: xarray_open_kwargs: dict @@ -167,7 +170,7 @@ def open_chunk(self, chunk_key): logger.debug(f"{ds}") # do we really want to just delete all encoding? - #for v in ds.variables: + # for v in ds.variables: # ds[v].encoding = {} # TODO: maybe do some chunking here? @@ -176,10 +179,8 @@ def open_chunk(self, chunk_key): @dataclass class ZarrXarrayWriterMixin: - @property def store_chunk(self) -> Callable: - def _store_chunk(chunk_key): ds_chunk = self.open_chunk(chunk_key) ds_chunk = self.preprocess_chunk(ds_chunk) @@ -190,26 +191,21 @@ def _store_chunk(chunk_key): return _store_chunk - def open_target(self): target_mapper = self.target.get_mapper() return xr.open_zarr(target_mapper) - def initialize_target(self, ds, **expand_dims): logger.info(f"Creating a new dataset in target") target_mapper = self.target.get_mapper() - ds.to_zarr(target_mapper, mode='w', compute=False) - + ds.to_zarr(target_mapper, mode="w", compute=False) def expand_target_dim(self, dim, dimsize): target_mapper = self.target.get_mapper() zgroup = zarr.open_group(target_mapper) ds = self.open_target() - sequence_axes = {v: ds[v].get_axis_num(dim) - for v in ds.variables - if dim in ds[v].dims} + sequence_axes = {v: ds[v].get_axis_num(dim) for v in ds.variables if dim in ds[v].dims} for v, axis in sequence_axes.items(): arr = zgroup[v] @@ -218,15 +214,12 @@ def expand_target_dim(self, dim, dimsize): arr.resize(shape) - - @dataclass -class ZarrConsolidatorMixin(): - consolidate_zarr: bool #= True +class ZarrConsolidatorMixin: + consolidate_zarr: bool # = True @property def finalize(self): - def _finalize(): if self.consolidate_zarr: logger.info(f"Consolidating Zarr metadata") @@ -243,62 +236,50 @@ class SequenceRecipe(DatasetRecipe): inputs_per_chunk: int = 1 nitems_per_input: int = 1 - def __post_init__(self): - self._chunks_inputs = {k: v for k, v in - enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk))} + self._chunks_inputs = { + k: v for k, v in enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk)) + } def drop_vars(ds): # writing a region means that all the variables MUST have sequence_dim - to_drop = [v for v in ds.variables - if self.sequence_dim not in ds[v].dims] + to_drop = [v for v in ds.variables if self.sequence_dim not in ds[v].dims] return ds.drop_vars(to_drop) self.chunk_preprocess_funcs.append(drop_vars) - def inputs_for_chunk(self, chunk_key): return self._chunks_inputs[chunk_key] - def iter_inputs(self): for chunk_key in self.iter_chunks(): for input in self.inputs_for_chunk(chunk_key): yield input - def nitems_for_chunk(self, chunk_key): return self.nitems_per_input * len(self.inputs_for_chunk(chunk_key)) - def region_for_chunk(self, chunk_key): # return a dict suitable to pass to xr.to_zarr(region=...) # specifies where in the overall array to put this chunk's data stride = self.nitems_per_input * self.inputs_per_chunk start = chunk_key * stride - return { - self.sequence_dim: - slice(start, start + self.nitems_for_chunk(chunk_key)) - } - + return {self.sequence_dim: slice(start, start + self.nitems_for_chunk(chunk_key))} def sequence_len(self): # tells the total size of dataset along the sequence dimension return sum([self.nitems_for_chunk(k) for k in self.iter_chunks()]) - def sequence_chunks(self): # chunking return {self.sequence_dim: self.inputs_per_chunk * self.nitems_per_input} - def iter_chunks(self): for k in self._chunks_inputs: yield k @property def prepare(self): - def _prepare(): target_store = self.target.get_mapper() @@ -313,7 +294,7 @@ def _prepare(): # make sure the concat dim has a valid fill_value to avoid # overruns when writing chunk - ds[self.sequence_dim].encoding = {'_FillValue': -1} + ds[self.sequence_dim].encoding = {"_FillValue": -1} # actually not necessary if we use decode_times=False self.initialize_target(ds) @@ -324,12 +305,12 @@ def _prepare(): @dataclass class StandardSequentialRecipe( - SequenceRecipe, - InputCachingMixin, - XarrayConcatChunkOpener, - ZarrXarrayWriterMixin, - ZarrConsolidatorMixin - ): + SequenceRecipe, + InputCachingMixin, + XarrayConcatChunkOpener, + ZarrXarrayWriterMixin, + ZarrConsolidatorMixin, +): pass @@ -338,7 +319,6 @@ class StandardSequentialRecipe( # only needed because of # https://github.com/pydata/xarray/issues/4631 def _fix_scalar_attr_encoding(ds): - def _fixed_attrs(d): fixed = {} for k, v in d.items(): diff --git a/pangeo_forge/storage.py b/pangeo_forge/storage.py index 04e949a0..b3777097 100644 --- a/pangeo_forge/storage.py +++ b/pangeo_forge/storage.py @@ -1,7 +1,8 @@ -from dataclasses import dataclass +import os from contextlib import closing, contextmanager +from dataclasses import dataclass + import fsspec -import os @dataclass diff --git a/pangeo_forge/utils.py b/pangeo_forge/utils.py index c49b3014..c865e521 100644 --- a/pangeo_forge/utils.py +++ b/pangeo_forge/utils.py @@ -1,5 +1,6 @@ import itertools + # https://alexwlchan.net/2018/12/iterating-in-fixed-size-chunks/ def chunked_iterable(iterable, size): it = iter(iterable) From b3a42ed950716775a391d4030396a3dc2a327786 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 21:45:22 -0500 Subject: [PATCH 12/34] update requirements --- ci/py3.7.yml | 3 ++- ci/py3.8.yml | 3 ++- requirements.txt | 4 ++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/ci/py3.7.yml b/ci/py3.7.yml index 8d5f0961..c310ff4c 100644 --- a/ci/py3.7.yml +++ b/ci/py3.7.yml @@ -25,4 +25,5 @@ dependencies: - scipy - setuptools - toolz - - zarr + - xarray>=0.16.2 + - zarr>=2.6.0 diff --git a/ci/py3.8.yml b/ci/py3.8.yml index ada7aae7..739ce1ef 100644 --- a/ci/py3.8.yml +++ b/ci/py3.8.yml @@ -25,4 +25,5 @@ dependencies: - scipy - setuptools - toolz - - zarr + - xarray>=0.16.2 + - zarr>=2.6.0 diff --git a/requirements.txt b/requirements.txt index dbf84c5f..11a36cbe 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,6 +2,6 @@ setuptools click dask distributed -xarray -zarr +xarray >= 0.16.2 +zarr >= 2.6.0 fsspec[http] From ec95d9d7bb23606ecc9f6f5bb90568286fca096d Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 1 Dec 2020 21:51:07 -0500 Subject: [PATCH 13/34] more linting --- tests/fixtures.py | 32 +++++++++++++++++--------------- tests/test_recipe.py | 21 +++++++++++++++++---- tests/test_utils.py | 26 +++++++++++++++++++++++--- 3 files changed, 57 insertions(+), 22 deletions(-) diff --git a/tests/fixtures.py b/tests/fixtures.py index 1ee9427e..5ac6738d 100644 --- a/tests/fixtures.py +++ b/tests/fixtures.py @@ -7,7 +7,7 @@ import pytest import xarray as xr -from pangeo_forge.storage import Target, InputCache +from pangeo_forge.storage import InputCache, Target # where to run the http server _PORT = "8080" @@ -46,7 +46,7 @@ def netcdf_local_paths(daily_xarray_dataset, tmpdir_factory, request): """Return a list of paths pointing to netcdf files.""" tmp_path = tmpdir_factory.mktemp("netcdf_data") items_per_file = {"D": 1, "2D": 2} - daily_xarray_dataset.attrs['items_per_file'] = items_per_file[request.param] + daily_xarray_dataset.attrs["items_per_file"] = items_per_file[request.param] gb = daily_xarray_dataset.resample(time=request.param) _, datasets = zip(*gb) fnames = [f"{n:03d}.nc" for n in range(len(datasets))] @@ -76,6 +76,7 @@ def netcdf_http_server(netcdf_local_paths): @pytest.fixture() def tmp_target(tmpdir_factory): import fsspec + fs = fsspec.get_filesystem_class("file")() path = str(tmpdir_factory.mktemp("target")) return Target(fs, path) @@ -85,9 +86,10 @@ def tmp_target(tmpdir_factory): def tmp_cache(tmpdir_factory): path = str(tmpdir_factory.mktemp("cache")) fs = fsspec.get_filesystem_class("file")() - cache = InputCache(fs, prefix='cache') + cache = InputCache(fs, prefix="cache") return cache + # tests that our fixtures work @@ -107,19 +109,19 @@ def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): def test_target(tmp_target): mapper = tmp_target.get_mapper() - mybytes = b'bar' - mapper['foo'] = b'bar' - with open(tmp_target.path + '/foo') as f: + mybytes = b"bar" + mapper["foo"] = b"bar" + with open(tmp_target.path + "/foo") as f: res = f.read() - assert res == 'bar' + assert res == "bar" def test_cache(tmp_cache): - assert not tmp_cache.exists('foo') - with tmp_cache.open('foo', mode='w') as f: - f.write('bar') - assert tmp_cache.exists('foo') - with tmp_cache.open('foo', mode='r') as f: - assert f.read() == 'bar' - tmp_cache.rm('foo') - assert not tmp_cache.exists('foo') + assert not tmp_cache.exists("foo") + with tmp_cache.open("foo", mode="w") as f: + f.write("bar") + assert tmp_cache.exists("foo") + with tmp_cache.open("foo", mode="r") as f: + assert f.read() == "bar" + tmp_cache.rm("foo") + assert not tmp_cache.exists("foo") diff --git a/tests/test_recipe.py b/tests/test_recipe.py index b91ea2c5..d55b4710 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -4,15 +4,28 @@ from pangeo_forge import recipe -from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache +from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_cache, tmp_target dummy_fnames = ["a.nc", "b.nc", "c.nc"] + + @pytest.mark.parametrize( "file_urls, files_per_chunk, expected_keys, expected_filenames", [ (dummy_fnames, 1, [0, 1, 2], [("a.nc",), ("b.nc",), ("c.nc",)]), - (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",)]) - ] + ( + dummy_fnames, + 2, + [0, 1], + [ + ( + "a.nc", + "b.nc", + ), + ("c.nc",), + ], + ), + ], ) def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames, tmp_target): @@ -47,7 +60,7 @@ def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_c input_urls=netcdf_local_paths, sequence_dim="time", inputs_per_chunk=1, - nitems_per_input=daily_xarray_dataset.attrs['items_per_file'] + nitems_per_input=daily_xarray_dataset.attrs["items_per_file"], ) # this is the cannonical way to manually execute a recipe diff --git a/tests/test_utils.py b/tests/test_utils.py index 03e6be7f..0d9b8ee8 100644 --- a/tests/test_utils.py +++ b/tests/test_utils.py @@ -8,9 +8,29 @@ [ ([1, 2, 3], 1, [(1,), (2,), (3,)]), ([1, 2, 3], 2, [(1, 2), (3,)]), - ([1, 2, 3], 3, [(1, 2, 3,)]), - ([1, 2, 3], 4, [(1, 2, 3,)]) - ] + ( + [1, 2, 3], + 3, + [ + ( + 1, + 2, + 3, + ) + ], + ), + ( + [1, 2, 3], + 4, + [ + ( + 1, + 2, + 3, + ) + ], + ), + ], ) def test_chunked_iterable(iterable, size, expected): actual = list(pangeo_forge.utils.chunked_iterable(iterable, size)) From 79ab04fcdcbda4041cdda9d39043a679014d0dfd Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Wed, 2 Dec 2020 14:00:24 -0500 Subject: [PATCH 14/34] added executors --- pangeo_forge/executors/__init__.py | 0 pangeo_forge/executors/prefect.py | 38 ++++++++++++++++++++++++++++++ pangeo_forge/executors/python.py | 29 +++++++++++++++++++++++ 3 files changed, 67 insertions(+) create mode 100644 pangeo_forge/executors/__init__.py create mode 100644 pangeo_forge/executors/prefect.py create mode 100644 pangeo_forge/executors/python.py diff --git a/pangeo_forge/executors/__init__.py b/pangeo_forge/executors/__init__.py new file mode 100644 index 00000000..e69de29b diff --git a/pangeo_forge/executors/prefect.py b/pangeo_forge/executors/prefect.py new file mode 100644 index 00000000..cea2c367 --- /dev/null +++ b/pangeo_forge/executors/prefect.py @@ -0,0 +1,38 @@ +from typing import Any, Iterable, Tuple + +import prefect + +from ..recipe import DatasetRecipe + +class PrefectExecutor(): + + def prepare_plan(self, r: DatasetRecipe) -> prefect.Flow: + + # wrap our functions as prefect tasks + @prefect.task + def prepare() -> None: + r.prepare() + + @prefect.task + def cache_input(input: Any) -> None: + r.cache_input(input) + + @prefect.task + def store_chunk(input: Any) -> None: + r.store_chunk(input) + + @prefect.task + def finalize() -> None: + r.finalize() + + with prefect.Flow("Pangeo-Forge") as flow: + prepare() + cache_input.map(list(r.iter_inputs())) + store_chunk.map(list(r.iter_chunks())) + finalize() + + return flow + + + def execute_plan(self, plan: prefect.Flow, **kwargs): + return plan.run(**kwargs) diff --git a/pangeo_forge/executors/python.py b/pangeo_forge/executors/python.py new file mode 100644 index 00000000..4be76073 --- /dev/null +++ b/pangeo_forge/executors/python.py @@ -0,0 +1,29 @@ +from functools import partial + +#from ..types import Pipeline, Stage, Task +from ..recipe import DatasetRecipe + +from typing import Callable, Iterable + +Task = Callable[[], None] + +class PythonExecutor: + + def prepare_plan(self, r: DatasetRecipe) -> Task: + tasks = [] + tasks.append(r.prepare) + for input in r.iter_inputs(): + tasks.append(partial(r.cache_input, input)) + for chunk in r.iter_chunks(): + tasks.append(partial(r.store_chunk, chunk)) + tasks.append(r.finalize) + + return partial(_execute_all, tasks) + + def execute_plan(self, plan: Task, **kwargs): + plan() + + +def _execute_all(tasks: Iterable[Task]) -> None: + for task in tasks: + task() From e6b32a98282e823a73c7f4f8b718063a158a1615 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 3 Dec 2020 20:41:35 -0500 Subject: [PATCH 15/34] linting and stuff --- pangeo_forge/executors/prefect.py | 5 ++- pangeo_forge/executors/python.py | 7 ++--- pangeo_forge/recipe.py | 50 ++++++++++++------------------ pangeo_forge/storage.py | 2 +- pangeo_forge/utils.py | 21 +++++++++++++ tests/{fixtures.py => conftest.py} | 39 +---------------------- tests/test_recipe.py | 3 -- 7 files changed, 47 insertions(+), 80 deletions(-) rename tests/{fixtures.py => conftest.py} (69%) diff --git a/pangeo_forge/executors/prefect.py b/pangeo_forge/executors/prefect.py index cea2c367..afe13b8a 100644 --- a/pangeo_forge/executors/prefect.py +++ b/pangeo_forge/executors/prefect.py @@ -1,11 +1,11 @@ -from typing import Any, Iterable, Tuple +from typing import Any import prefect from ..recipe import DatasetRecipe -class PrefectExecutor(): +class PrefectExecutor: def prepare_plan(self, r: DatasetRecipe) -> prefect.Flow: # wrap our functions as prefect tasks @@ -33,6 +33,5 @@ def finalize() -> None: return flow - def execute_plan(self, plan: prefect.Flow, **kwargs): return plan.run(**kwargs) diff --git a/pangeo_forge/executors/python.py b/pangeo_forge/executors/python.py index 4be76073..5e1d1c64 100644 --- a/pangeo_forge/executors/python.py +++ b/pangeo_forge/executors/python.py @@ -1,14 +1,13 @@ from functools import partial +from typing import Callable, Iterable -#from ..types import Pipeline, Stage, Task +# from ..types import Pipeline, Stage, Task from ..recipe import DatasetRecipe -from typing import Callable, Iterable - Task = Callable[[], None] -class PythonExecutor: +class PythonExecutor: def prepare_plan(self, r: DatasetRecipe) -> Task: tasks = [] tasks.append(r.prepare) diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index d265be8e..e651e834 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -4,21 +4,20 @@ import logging from contextlib import contextmanager -from dataclasses import dataclass, field -from typing import Any, Callable, Iterable, Optional +from dataclasses import dataclass +from typing import Callable, Iterable import fsspec -import numpy as np import xarray as xr import zarr from .storage import InputCache, Target -from .utils import chunked_iterable +from .utils import chunked_iterable, fix_scalar_attr_encoding # logger = logging.getLogger(__name__) logger = logging.getLogger("recipe") -### How to manually execute a recipe: ### +# How to manually execute a recipe: ### # # t = PangeoForgeTarget() # r = MyRecipe(target=t, **opts) # 1 @@ -29,8 +28,6 @@ # for chunk_key in r.iter_chunks(): # r.store_chunk(chunk_key) # 5 # r.finalize() # 6 -# -### # 1) Initialize the Recipe object @@ -152,7 +149,7 @@ def open_input(self, fname): logger.info(f"Opening input with Xarray '{fname}'") ds = xr.open_dataset(f, **self.xarray_open_kwargs).load() # do we always want to remove encoding? I think so. - ds = _fix_scalar_attr_encoding(ds) + ds = fix_scalar_attr_encoding(ds) logger.debug(f"{ds}") return ds @@ -196,7 +193,7 @@ def open_target(self): return xr.open_zarr(target_mapper) def initialize_target(self, ds, **expand_dims): - logger.info(f"Creating a new dataset in target") + logger.info("Creating a new dataset in target") target_mapper = self.target.get_mapper() ds.to_zarr(target_mapper, mode="w", compute=False) @@ -222,7 +219,7 @@ class ZarrConsolidatorMixin: def finalize(self): def _finalize(): if self.consolidate_zarr: - logger.info(f"Consolidating Zarr metadata") + logger.info("Consolidating Zarr metadata") target_mapper = self.target.get_mapper() zarr.consolidate_metadata(target_mapper) @@ -231,10 +228,21 @@ def _finalize(): @dataclass class SequenceRecipe(DatasetRecipe): + """There are many inputs (a.k.a. files, granules), arranged in a sequence + along the dimension `sequence_dim`. Each file may contain multiple variables. + """ + input_urls: Iterable[str] + """The inputs used to generate the dataset.""" + sequence_dim: str + """The dimension name along which the inputs will be concatenated.""" + inputs_per_chunk: int = 1 + """The number of inputs to use in each chunk.""" + nitems_per_input: int = 1 + """The length of each input along the `sequence_dim` dimension.""" def __post_init__(self): self._chunks_inputs = { @@ -282,11 +290,9 @@ def iter_chunks(self): def prepare(self): def _prepare(): - target_store = self.target.get_mapper() - try: ds = self.open_target() - logger.info(f"Found an existing dataset in target") + logger.info("Found an existing dataset in target") logger.debug(f"{ds}") except (IOError, zarr.errors.GroupNotFoundError): first_chunk_key = next(self.iter_chunks()) @@ -315,21 +321,3 @@ class StandardSequentialRecipe( # helper utilities - -# only needed because of -# https://github.com/pydata/xarray/issues/4631 -def _fix_scalar_attr_encoding(ds): - def _fixed_attrs(d): - fixed = {} - for k, v in d.items(): - if isinstance(v, np.ndarray) and len(v) == 1: - fixed[k] = v[0] - return fixed - - ds = ds.copy() - ds.attrs.update(_fixed_attrs(ds.attrs)) - ds.encoding.update(_fixed_attrs(ds.encoding)) - for v in ds.variables: - ds[v].attrs.update(_fixed_attrs(ds[v].attrs)) - ds[v].encoding.update(_fixed_attrs(ds[v].encoding)) - return ds diff --git a/pangeo_forge/storage.py b/pangeo_forge/storage.py index b3777097..742bb9f2 100644 --- a/pangeo_forge/storage.py +++ b/pangeo_forge/storage.py @@ -1,5 +1,5 @@ import os -from contextlib import closing, contextmanager +from contextlib import contextmanager from dataclasses import dataclass import fsspec diff --git a/pangeo_forge/utils.py b/pangeo_forge/utils.py index c865e521..1ac84c7b 100644 --- a/pangeo_forge/utils.py +++ b/pangeo_forge/utils.py @@ -1,5 +1,7 @@ import itertools +import numpy as np + # https://alexwlchan.net/2018/12/iterating-in-fixed-size-chunks/ def chunked_iterable(iterable, size): @@ -9,3 +11,22 @@ def chunked_iterable(iterable, size): if not chunk: break yield chunk + + +# only needed because of +# https://github.com/pydata/xarray/issues/4631 +def fix_scalar_attr_encoding(ds): + def _fixed_attrs(d): + fixed = {} + for k, v in d.items(): + if isinstance(v, np.ndarray) and len(v) == 1: + fixed[k] = v[0] + return fixed + + ds = ds.copy() + ds.attrs.update(_fixed_attrs(ds.attrs)) + ds.encoding.update(_fixed_attrs(ds.encoding)) + for v in ds.variables: + ds[v].attrs.update(_fixed_attrs(ds[v].attrs)) + ds[v].encoding.update(_fixed_attrs(ds[v].encoding)) + return ds diff --git a/tests/fixtures.py b/tests/conftest.py similarity index 69% rename from tests/fixtures.py rename to tests/conftest.py index 5ac6738d..c2584809 100644 --- a/tests/fixtures.py +++ b/tests/conftest.py @@ -86,42 +86,5 @@ def tmp_target(tmpdir_factory): def tmp_cache(tmpdir_factory): path = str(tmpdir_factory.mktemp("cache")) fs = fsspec.get_filesystem_class("file")() - cache = InputCache(fs, prefix="cache") + cache = InputCache(fs, prefix=path) return cache - - -# tests that our fixtures work - - -def test_fixture_local_files(daily_xarray_dataset, netcdf_local_paths): - paths = [str(path) for path in netcdf_local_paths] - ds = xr.open_mfdataset(paths, combine="nested", concat_dim="time") - assert ds.identical(daily_xarray_dataset) - - -def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): - url, paths = netcdf_http_server - urls = ["/".join([url, str(path)]) for path in paths] - open_files = [fsspec.open(url).open() for url in urls] - ds = xr.open_mfdataset(open_files, combine="nested", concat_dim="time") - assert ds.identical(daily_xarray_dataset) - - -def test_target(tmp_target): - mapper = tmp_target.get_mapper() - mybytes = b"bar" - mapper["foo"] = b"bar" - with open(tmp_target.path + "/foo") as f: - res = f.read() - assert res == "bar" - - -def test_cache(tmp_cache): - assert not tmp_cache.exists("foo") - with tmp_cache.open("foo", mode="w") as f: - f.write("bar") - assert tmp_cache.exists("foo") - with tmp_cache.open("foo", mode="r") as f: - assert f.read() == "bar" - tmp_cache.rm("foo") - assert not tmp_cache.exists("foo") diff --git a/tests/test_recipe.py b/tests/test_recipe.py index d55b4710..28e58bc3 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -1,11 +1,8 @@ import pytest import xarray as xr -import zarr from pangeo_forge import recipe -from .fixtures import daily_xarray_dataset, netcdf_local_paths, tmp_cache, tmp_target - dummy_fnames = ["a.nc", "b.nc", "c.nc"] From b993dabd5f3db6fbc86826cb18a6372f120e4a20 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 3 Dec 2020 21:19:47 -0500 Subject: [PATCH 16/34] testing executors --- pangeo_forge/executors/__init__.py | 8 ++++++ pangeo_forge/recipe.py | 3 --- setup.cfg | 3 +-- tests/conftest.py | 19 ++++++++++++++ tests/test_executors.py | 14 +++++++++++ tests/test_fixtures.py | 40 ++++++++++++++++++++++++++++++ tests/test_recipe.py | 2 ++ 7 files changed, 84 insertions(+), 5 deletions(-) create mode 100644 tests/test_executors.py create mode 100644 tests/test_fixtures.py diff --git a/pangeo_forge/executors/__init__.py b/pangeo_forge/executors/__init__.py index e69de29b..e0aa6809 100644 --- a/pangeo_forge/executors/__init__.py +++ b/pangeo_forge/executors/__init__.py @@ -0,0 +1,8 @@ +""" +Executors know how to run recipes. +""" + +from .prefect import PrefectExecutor +from .python import PythonExecutor + +__all__ = [PythonExecutor, PrefectExecutor] diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index e651e834..d0a6a6ab 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -318,6 +318,3 @@ class StandardSequentialRecipe( ZarrConsolidatorMixin, ): pass - - -# helper utilities diff --git a/setup.cfg b/setup.cfg index 7468400a..9c7719f5 100644 --- a/setup.cfg +++ b/setup.cfg @@ -11,5 +11,4 @@ combine_as_imports=True line_length=100 [tool:pytest] -log_cli = True -log_level = INFO +log_cli = False diff --git a/tests/conftest.py b/tests/conftest.py index c2584809..6389e837 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -7,6 +7,7 @@ import pytest import xarray as xr +from pangeo_forge import recipe from pangeo_forge.storage import InputCache, Target # where to run the http server @@ -88,3 +89,21 @@ def tmp_cache(tmpdir_factory): fs = fsspec.get_filesystem_class("file")() cache = InputCache(fs, prefix=path) return cache + + +@pytest.fixture +def sequential_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): + r = recipe.StandardSequentialRecipe( + consolidate_zarr=True, + xarray_open_kwargs={}, + xarray_concat_kwargs={}, + require_cache=False, + input_cache=tmp_cache, + target=tmp_target, + chunk_preprocess_funcs=[], + input_urls=netcdf_local_paths, + sequence_dim="time", + inputs_per_chunk=1, + nitems_per_input=daily_xarray_dataset.attrs["items_per_file"], + ) + return r, daily_xarray_dataset, tmp_target diff --git a/tests/test_executors.py b/tests/test_executors.py new file mode 100644 index 00000000..eb90e2a0 --- /dev/null +++ b/tests/test_executors.py @@ -0,0 +1,14 @@ +import pytest +import xarray as xr + +from pangeo_forge.executors import PrefectExecutor, PythonExecutor + + +@pytest.mark.parametrize("Executor", [PythonExecutor, PrefectExecutor]) +def test_recipe_w_executor(Executor, sequential_recipe): + rec, ds_expected, target = sequential_recipe + ex = Executor() + plan = ex.prepare_plan(rec) + ex.execute_plan(plan) + ds_actual = xr.open_zarr(target.get_mapper()).load() + assert ds_actual.identical(ds_expected) diff --git a/tests/test_fixtures.py b/tests/test_fixtures.py new file mode 100644 index 00000000..c867f17b --- /dev/null +++ b/tests/test_fixtures.py @@ -0,0 +1,40 @@ +import fsspec +import xarray as xr + +from pangeo_forge.utils import fix_scalar_attr_encoding + + +def test_fixture_local_files(daily_xarray_dataset, netcdf_local_paths): + paths = [str(path) for path in netcdf_local_paths] + ds = xr.open_mfdataset(paths, combine="nested", concat_dim="time") + assert ds.identical(daily_xarray_dataset) + + +def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): + url, paths = netcdf_http_server + urls = ["/".join([url, str(path)]) for path in paths] + open_files = [fsspec.open(url).open() for url in urls] + ds = xr.open_mfdataset(open_files, combine="nested", concat_dim="time").load() + ds = fix_scalar_attr_encoding(ds) + print(ds) + print(daily_xarray_dataset) + assert ds.identical(daily_xarray_dataset) + + +def test_target(tmp_target): + mapper = tmp_target.get_mapper() + mapper["foo"] = b"bar" + with open(tmp_target.path + "/foo") as f: + res = f.read() + assert res == "bar" + + +def test_cache(tmp_cache): + assert not tmp_cache.exists("foo") + with tmp_cache.open("foo", mode="w") as f: + f.write("bar") + assert tmp_cache.exists("foo") + with tmp_cache.open("foo", mode="r") as f: + assert f.read() == "bar" + tmp_cache.rm("foo") + assert not tmp_cache.exists("foo") diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 28e58bc3..11675e17 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -46,6 +46,8 @@ def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_fil def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): + # the same recipe is created as a fixture in conftest.py + # I left it here explicitly because it makes the test easier to read. r = recipe.StandardSequentialRecipe( consolidate_zarr=True, xarray_open_kwargs={}, From 0e150cb6a5c1291799c7588f69ca209cb1b7b6e9 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 21 Dec 2020 11:45:45 -0500 Subject: [PATCH 17/34] major simplification of recipe class --- pangeo_forge/executors/prefect.py | 15 +- pangeo_forge/executors/python.py | 6 +- pangeo_forge/recipe.py | 302 ++++++++++++------------------ tests/conftest.py | 13 +- tests/test_executors.py | 4 +- tests/test_recipe.py | 14 +- 6 files changed, 140 insertions(+), 214 deletions(-) diff --git a/pangeo_forge/executors/prefect.py b/pangeo_forge/executors/prefect.py index afe13b8a..ce91017e 100644 --- a/pangeo_forge/executors/prefect.py +++ b/pangeo_forge/executors/prefect.py @@ -2,11 +2,9 @@ import prefect -from ..recipe import DatasetRecipe - class PrefectExecutor: - def prepare_plan(self, r: DatasetRecipe) -> prefect.Flow: + def prepare_plan(self, r) -> prefect.Flow: # wrap our functions as prefect tasks @prefect.task @@ -26,10 +24,13 @@ def finalize() -> None: r.finalize() with prefect.Flow("Pangeo-Forge") as flow: - prepare() - cache_input.map(list(r.iter_inputs())) - store_chunk.map(list(r.iter_chunks())) - finalize() + prep_task = prepare() + cache_task = cache_input.map(list(r.iter_inputs())) + cache_task.set_dependencies(upstream_tasks=[prep_task]) + store_task = store_chunk.map(list(r.iter_chunks())) + store_task.set_dependencies(upstream_tasks=[cache_task]) + finalize_task = finalize() + finalize_task.set_dependencies(upstream_tasks=[store_task]) return flow diff --git a/pangeo_forge/executors/python.py b/pangeo_forge/executors/python.py index 5e1d1c64..0661d153 100644 --- a/pangeo_forge/executors/python.py +++ b/pangeo_forge/executors/python.py @@ -1,14 +1,10 @@ from functools import partial from typing import Callable, Iterable -# from ..types import Pipeline, Stage, Task -from ..recipe import DatasetRecipe - Task = Callable[[], None] - class PythonExecutor: - def prepare_plan(self, r: DatasetRecipe) -> Task: + def prepare_plan(self, r): tasks = [] tasks.append(r.prepare) for input in r.iter_inputs(): diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index d0a6a6ab..af389317 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -4,8 +4,8 @@ import logging from contextlib import contextmanager -from dataclasses import dataclass -from typing import Callable, Iterable +from dataclasses import dataclass, field +from typing import Callable, Iterable, Optional import fsspec import xarray as xr @@ -14,8 +14,7 @@ from .storage import InputCache, Target from .utils import chunked_iterable, fix_scalar_attr_encoding -# logger = logging.getLogger(__name__) -logger = logging.getLogger("recipe") +logger = logging.getLogger(__name__) # How to manually execute a recipe: ### # @@ -41,123 +40,159 @@ # Might be coming from the cache or might be read directly. # 6) +@contextmanager +def input_opener(fname, **kwargs): + logger.info(f"Opening input '{fname}'") + with fsspec.open(fname, **kwargs) as f: + yield f + +# Notes about dataclasses: +# - https://www.python.org/dev/peps/pep-0557/#inheritance +# - https://stackoverflow.com/questions/51575931/class-inheritance-in-python-3-7-dataclasses +# This means that, for now, I can't get default arguments to work. @dataclass -class DatasetRecipe: - target: Target - chunk_preprocess_funcs: Iterable[Callable] +class NetCDFtoZarrSequentialRecipe: + """There are many inputs (a.k.a. files, granules), arranged in a sequence + along the dimension `sequence_dim`. Each file may contain multiple variables. + """ + input_urls: Iterable[str] + """The inputs used to generate the dataset.""" - @property - def prepare(self): - def _prepare(): - pass + sequence_dim: str + """The dimension name along which the inputs will be concatenated.""" - return _prepare + inputs_per_chunk: int = 1 + """The number of inputs to use in each chunk.""" - def iter_inputs(self): - return [] - - # need to figure out what's going on with these methods and inheritance - # @property - # def cache_input(self): - # def _cache_input(input_key): - # raise NotImplementedError - # return _cache_input - - # this only gets run when iterating, not preparing! - def preprocess_chunk(self, ds): - for f in self.chunk_preprocess_funcs: - ds = f(ds) - return ds + nitems_per_input: int = 1 + """The length of each input along the `sequence_dim` dimension.""" - def iter_chunks(self): - raise NotImplementedError + target: Optional[Target] = None + """A location in which to put the dataset.""" - # @property - # def store_chunk(self): - # def _store_chunk(chunk_key): - # raise NotImplementedError - # return _store_chunk + input_cache: Optional[InputCache] = None + """The length of each input along the `sequence_dim` dimension.""" - # @property - # def finalize(self): - # - # def _finalize(): - # pass - # return _finalize + consolidate_zarr: bool = True + """Whether to consolidate the resulting Zarr dataset.""" + xarray_open_kwargs: dict = field(default_factory=dict) + """Extra options for opening the inputs with Xarray.""" -# Notes about dataclasses: -# - https://www.python.org/dev/peps/pep-0557/#inheritance -# - https://stackoverflow.com/questions/51575931/class-inheritance-in-python-3-7-dataclasses -# This means that, for now, I can't get default arguments to work. + xarray_concat_kwargs: dict = field(default_factory=dict) + """Extra options to pass to Xarray when concatenating the inputs to form a chunk.""" + def __post_init__(self): + self._chunks_inputs = { + k: v for k, v in enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk)) + } -@dataclass -class FSSpecFileOpenerMixin: - # input_open_kwargs: dict #= field(default_factory=dict) + @property + def prepare(self) -> Callable: + """Prepare target for storing dataset.""" - @contextmanager - def input_opener(self, fname, **kwargs): - logger.info(f"Opening input '{fname}'") - with fsspec.open(fname, **kwargs) as f: - yield f + def _prepare(): + try: + ds = self.open_target() + logger.info("Found an existing dataset in target") + logger.debug(f"{ds}") + except (IOError, zarr.errors.GroupNotFoundError): + first_chunk_key = next(self.iter_chunks()) + for input_url in self.inputs_for_chunk(first_chunk_key): + self.cache_input(input_url) + ds = self.open_chunk(first_chunk_key).chunk() -@dataclass -class InputCachingMixin(FSSpecFileOpenerMixin): - require_cache: bool # = False - input_cache: InputCache + # make sure the concat dim has a valid fill_value to avoid + # overruns when writing chunk + ds[self.sequence_dim].encoding = {"_FillValue": -1} + # actually not necessary if we use decode_times=False + self.initialize_target(ds) + + self.expand_target_dim(self.sequence_dim, self.sequence_len()) + + return _prepare - # returns a function that takes one input, the input_key - # this allows us to parallelize these operations @property - def cache_input(self): + def cache_input(self) -> Callable: + """Cache the input. - opener = super().input_opener + Properties + ---------- + url : URL pointing to the input file. Must be openable by fsspec. + """ def cache_func(fname: str) -> None: logger.info(f"Caching input '{fname}'") - with opener(fname, mode="rb") as source: + with input_opener(fname, mode="rb") as source: with self.input_cache.open(fname, mode="wb") as target: target.write(source.read()) return cache_func + @property + def store_chunk(self) -> Callable: + """Store a chunk in the target. + + Parameters + ---------- + chunk_key : str + The identifier for the chunk + """ + def _store_chunk(chunk_key): + ds_chunk = self.open_chunk(chunk_key) + + def drop_vars(ds): + # writing a region means that all the variables MUST have sequence_dim + to_drop = [v for v in ds.variables if self.sequence_dim not in ds[v].dims] + return ds.drop_vars(to_drop) + + ds_chunk = drop_vars(ds_chunk) + target_mapper = self.target.get_mapper() + write_region = self.region_for_chunk(chunk_key) + logger.info(f"Storing chunk '{chunk_key}' to Zarr region {write_region}") + ds_chunk.to_zarr(target_mapper, region=write_region) + + return _store_chunk + + @property + def finalize(self) -> Callable: + """Finalize writing of dataset.""" + + def _finalize(): + if self.consolidate_zarr: + logger.info("Consolidating Zarr metadata") + target_mapper = self.target.get_mapper() + zarr.consolidate_metadata(target_mapper) + + return _finalize + @contextmanager - def input_opener(self, fname): - if self.input_cache.exists(fname): + def input_opener(self, fname: str): + if self.input_cache is None: + logger.info(f"No cache. Opening input `{fname}` directly.") + # This will bypass the cache. May be slow. + with input_opener(fname, mode="rb") as f: + yield f + elif self.input_cache.exists(fname): logger.info(f"Input '{fname}' found in cache") with self.input_cache.open(fname, mode="rb") as f: yield f - elif self.require_cache: - # this creates an error on prepare because nothing is cached - raise IOError("Input can only be opened from cache. Call .cache_input first.") else: - logger.info(f"Input '{fname}' not found in cache. Opening directly.") - # This will bypass the cache. May be slow. - with super().input_opener(fname, mode="rb") as f: - yield f - + raise ValueError(f"Input '{fname}' has not been cached yet. " + "Call .cache_input() first.") -@dataclass -class XarrayInputOpener: - xarray_open_kwargs: dict - - def open_input(self, fname): + def open_input(self, fname: str): with self.input_opener(fname) as f: logger.info(f"Opening input with Xarray '{fname}'") - ds = xr.open_dataset(f, **self.xarray_open_kwargs).load() - # do we always want to remove encoding? I think so. + ds = xr.open_dataset(f, **self.xarray_open_kwargs) + # explicitly load into memory + ds = ds.load() ds = fix_scalar_attr_encoding(ds) logger.debug(f"{ds}") return ds - -@dataclass -class XarrayConcatChunkOpener(XarrayInputOpener): - xarray_concat_kwargs: dict - def open_chunk(self, chunk_key): logger.info(f"Concatenating inputs for chunk '{chunk_key}'") inputs = self.inputs_for_chunk(chunk_key) @@ -166,28 +201,9 @@ def open_chunk(self, chunk_key): ds = xr.concat(dsets, self.sequence_dim, **self.xarray_concat_kwargs) logger.debug(f"{ds}") - # do we really want to just delete all encoding? - # for v in ds.variables: - # ds[v].encoding = {} - # TODO: maybe do some chunking here? return ds - -@dataclass -class ZarrXarrayWriterMixin: - @property - def store_chunk(self) -> Callable: - def _store_chunk(chunk_key): - ds_chunk = self.open_chunk(chunk_key) - ds_chunk = self.preprocess_chunk(ds_chunk) - target_mapper = self.target.get_mapper() - write_region = self.region_for_chunk(chunk_key) - logger.info(f"Storing chunk '{chunk_key}' to Zarr region {write_region}") - ds_chunk.to_zarr(target_mapper, region=write_region) - - return _store_chunk - def open_target(self): target_mapper = self.target.get_mapper() return xr.open_zarr(target_mapper) @@ -210,52 +226,6 @@ def expand_target_dim(self, dim, dimsize): shape[axis] = dimsize arr.resize(shape) - -@dataclass -class ZarrConsolidatorMixin: - consolidate_zarr: bool # = True - - @property - def finalize(self): - def _finalize(): - if self.consolidate_zarr: - logger.info("Consolidating Zarr metadata") - target_mapper = self.target.get_mapper() - zarr.consolidate_metadata(target_mapper) - - return _finalize - - -@dataclass -class SequenceRecipe(DatasetRecipe): - """There are many inputs (a.k.a. files, granules), arranged in a sequence - along the dimension `sequence_dim`. Each file may contain multiple variables. - """ - - input_urls: Iterable[str] - """The inputs used to generate the dataset.""" - - sequence_dim: str - """The dimension name along which the inputs will be concatenated.""" - - inputs_per_chunk: int = 1 - """The number of inputs to use in each chunk.""" - - nitems_per_input: int = 1 - """The length of each input along the `sequence_dim` dimension.""" - - def __post_init__(self): - self._chunks_inputs = { - k: v for k, v in enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk)) - } - - def drop_vars(ds): - # writing a region means that all the variables MUST have sequence_dim - to_drop = [v for v in ds.variables if self.sequence_dim not in ds[v].dims] - return ds.drop_vars(to_drop) - - self.chunk_preprocess_funcs.append(drop_vars) - def inputs_for_chunk(self, chunk_key): return self._chunks_inputs[chunk_key] @@ -272,7 +242,8 @@ def region_for_chunk(self, chunk_key): # specifies where in the overall array to put this chunk's data stride = self.nitems_per_input * self.inputs_per_chunk start = chunk_key * stride - return {self.sequence_dim: slice(start, start + self.nitems_for_chunk(chunk_key))} + region_slice = slice(start, start + self.nitems_for_chunk(chunk_key)) + return {self.sequence_dim: region_slice} def sequence_len(self): # tells the total size of dataset along the sequence dimension @@ -285,36 +256,3 @@ def sequence_chunks(self): def iter_chunks(self): for k in self._chunks_inputs: yield k - - @property - def prepare(self): - def _prepare(): - - try: - ds = self.open_target() - logger.info("Found an existing dataset in target") - logger.debug(f"{ds}") - except (IOError, zarr.errors.GroupNotFoundError): - first_chunk_key = next(self.iter_chunks()) - ds = self.open_chunk(first_chunk_key).chunk() - - # make sure the concat dim has a valid fill_value to avoid - # overruns when writing chunk - ds[self.sequence_dim].encoding = {"_FillValue": -1} - # actually not necessary if we use decode_times=False - self.initialize_target(ds) - - self.expand_target_dim(self.sequence_dim, self.sequence_len()) - - return _prepare - - -@dataclass -class StandardSequentialRecipe( - SequenceRecipe, - InputCachingMixin, - XarrayConcatChunkOpener, - ZarrXarrayWriterMixin, - ZarrConsolidatorMixin, -): - pass diff --git a/tests/conftest.py b/tests/conftest.py index 6389e837..6386481a 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -92,18 +92,13 @@ def tmp_cache(tmpdir_factory): @pytest.fixture -def sequential_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): - r = recipe.StandardSequentialRecipe( - consolidate_zarr=True, - xarray_open_kwargs={}, - xarray_concat_kwargs={}, - require_cache=False, - input_cache=tmp_cache, - target=tmp_target, - chunk_preprocess_funcs=[], +def netCDFtoZarr_sequential_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): + r = recipe.NetCDFtoZarrSequentialRecipe( input_urls=netcdf_local_paths, sequence_dim="time", inputs_per_chunk=1, nitems_per_input=daily_xarray_dataset.attrs["items_per_file"], + target=tmp_target, + input_cache=tmp_cache, ) return r, daily_xarray_dataset, tmp_target diff --git a/tests/test_executors.py b/tests/test_executors.py index eb90e2a0..551abaa5 100644 --- a/tests/test_executors.py +++ b/tests/test_executors.py @@ -5,8 +5,8 @@ @pytest.mark.parametrize("Executor", [PythonExecutor, PrefectExecutor]) -def test_recipe_w_executor(Executor, sequential_recipe): - rec, ds_expected, target = sequential_recipe +def test_recipe_w_executor(Executor, netCDFtoZarr_sequential_recipe): + rec, ds_expected, target = netCDFtoZarr_sequential_recipe ex = Executor() plan = ex.prepare_plan(rec) ex.execute_plan(plan) diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 11675e17..c146565e 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -6,6 +6,7 @@ dummy_fnames = ["a.nc", "b.nc", "c.nc"] +@pytest.mark.skip(reason="Removed this class for now") @pytest.mark.parametrize( "file_urls, files_per_chunk, expected_keys, expected_filenames", [ @@ -44,22 +45,17 @@ def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_fil assert fnames == expected -def test_full_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): +def test_NetCDFtoZarrSequentialRecipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): # the same recipe is created as a fixture in conftest.py # I left it here explicitly because it makes the test easier to read. - r = recipe.StandardSequentialRecipe( - consolidate_zarr=True, - xarray_open_kwargs={}, - xarray_concat_kwargs={}, - require_cache=False, - input_cache=tmp_cache, - target=tmp_target, - chunk_preprocess_funcs=[], + r = recipe.NetCDFtoZarrSequentialRecipe( input_urls=netcdf_local_paths, sequence_dim="time", inputs_per_chunk=1, nitems_per_input=daily_xarray_dataset.attrs["items_per_file"], + target=tmp_target, + input_cache=tmp_cache, ) # this is the cannonical way to manually execute a recipe From 1ec63eb1513b46b6cf649711151e919d820409fc Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 21 Dec 2020 11:49:35 -0500 Subject: [PATCH 18/34] fix precommit again --- pangeo_forge/executors/python.py | 1 + pangeo_forge/recipe.py | 10 ++++++++-- tests/test_recipe.py | 17 ++++------------- 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/pangeo_forge/executors/python.py b/pangeo_forge/executors/python.py index 0661d153..8e6625e8 100644 --- a/pangeo_forge/executors/python.py +++ b/pangeo_forge/executors/python.py @@ -3,6 +3,7 @@ Task = Callable[[], None] + class PythonExecutor: def prepare_plan(self, r): tasks = [] diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index af389317..2af0ebfe 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -40,22 +40,26 @@ # Might be coming from the cache or might be read directly. # 6) + @contextmanager def input_opener(fname, **kwargs): logger.info(f"Opening input '{fname}'") with fsspec.open(fname, **kwargs) as f: yield f + # Notes about dataclasses: # - https://www.python.org/dev/peps/pep-0557/#inheritance # - https://stackoverflow.com/questions/51575931/class-inheritance-in-python-3-7-dataclasses # This means that, for now, I can't get default arguments to work. + @dataclass class NetCDFtoZarrSequentialRecipe: """There are many inputs (a.k.a. files, granules), arranged in a sequence along the dimension `sequence_dim`. Each file may contain multiple variables. """ + input_urls: Iterable[str] """The inputs used to generate the dataset.""" @@ -140,6 +144,7 @@ def store_chunk(self) -> Callable: chunk_key : str The identifier for the chunk """ + def _store_chunk(chunk_key): ds_chunk = self.open_chunk(chunk_key) @@ -180,8 +185,9 @@ def input_opener(self, fname: str): with self.input_cache.open(fname, mode="rb") as f: yield f else: - raise ValueError(f"Input '{fname}' has not been cached yet. " - "Call .cache_input() first.") + raise ValueError( + f"Input '{fname}' has not been cached yet. " "Call .cache_input() first." + ) def open_input(self, fname: str): with self.input_opener(fname) as f: diff --git a/tests/test_recipe.py b/tests/test_recipe.py index c146565e..9b4c5461 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -11,18 +11,7 @@ "file_urls, files_per_chunk, expected_keys, expected_filenames", [ (dummy_fnames, 1, [0, 1, 2], [("a.nc",), ("b.nc",), ("c.nc",)]), - ( - dummy_fnames, - 2, - [0, 1], - [ - ( - "a.nc", - "b.nc", - ), - ("c.nc",), - ], - ), + (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",)]), ], ) def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames, tmp_target): @@ -45,7 +34,9 @@ def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_fil assert fnames == expected -def test_NetCDFtoZarrSequentialRecipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): +def test_NetCDFtoZarrSequentialRecipe( + daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache +): # the same recipe is created as a fixture in conftest.py # I left it here explicitly because it makes the test easier to read. From a4bf88a820f1f2444d6aa0bb898b665a0caa4666 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 21 Dec 2020 20:50:05 -0500 Subject: [PATCH 19/34] finally --- tests/test_recipe.py | 2 +- tests/test_utils.py | 24 ++---------------------- 2 files changed, 3 insertions(+), 23 deletions(-) diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 9b4c5461..2a6a412d 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -11,7 +11,7 @@ "file_urls, files_per_chunk, expected_keys, expected_filenames", [ (dummy_fnames, 1, [0, 1, 2], [("a.nc",), ("b.nc",), ("c.nc",)]), - (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",)]), + (dummy_fnames, 2, [0, 1], [("a.nc", "b.nc",), ("c.nc",),],), # noqa: E231 ], ) def test_sequence_recipe(file_urls, files_per_chunk, expected_keys, expected_filenames, tmp_target): diff --git a/tests/test_utils.py b/tests/test_utils.py index 0d9b8ee8..d486b7af 100644 --- a/tests/test_utils.py +++ b/tests/test_utils.py @@ -8,28 +8,8 @@ [ ([1, 2, 3], 1, [(1,), (2,), (3,)]), ([1, 2, 3], 2, [(1, 2), (3,)]), - ( - [1, 2, 3], - 3, - [ - ( - 1, - 2, - 3, - ) - ], - ), - ( - [1, 2, 3], - 4, - [ - ( - 1, - 2, - 3, - ) - ], - ), + ([1, 2, 3], 3, [(1, 2, 3,)],), + ([1, 2, 3], 4, [(1, 2, 3,)],), ], ) def test_chunked_iterable(iterable, size, expected): From dbf6b132bbd066ff4afacd59943d391fe34aded6 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 18 Jan 2021 12:39:19 -0500 Subject: [PATCH 20/34] cleanup --- pangeo_forge/executors/__init__.py | 8 ------- pangeo_forge/executors/prefect.py | 38 ------------------------------ pangeo_forge/executors/python.py | 25 -------------------- pangeo_forge/recipe.py | 9 +++++++ setup.cfg | 2 +- tests/test_executors.py | 14 +++++++---- 6 files changed, 20 insertions(+), 76 deletions(-) delete mode 100644 pangeo_forge/executors/__init__.py delete mode 100644 pangeo_forge/executors/prefect.py delete mode 100644 pangeo_forge/executors/python.py diff --git a/pangeo_forge/executors/__init__.py b/pangeo_forge/executors/__init__.py deleted file mode 100644 index e0aa6809..00000000 --- a/pangeo_forge/executors/__init__.py +++ /dev/null @@ -1,8 +0,0 @@ -""" -Executors know how to run recipes. -""" - -from .prefect import PrefectExecutor -from .python import PythonExecutor - -__all__ = [PythonExecutor, PrefectExecutor] diff --git a/pangeo_forge/executors/prefect.py b/pangeo_forge/executors/prefect.py deleted file mode 100644 index ce91017e..00000000 --- a/pangeo_forge/executors/prefect.py +++ /dev/null @@ -1,38 +0,0 @@ -from typing import Any - -import prefect - - -class PrefectExecutor: - def prepare_plan(self, r) -> prefect.Flow: - - # wrap our functions as prefect tasks - @prefect.task - def prepare() -> None: - r.prepare() - - @prefect.task - def cache_input(input: Any) -> None: - r.cache_input(input) - - @prefect.task - def store_chunk(input: Any) -> None: - r.store_chunk(input) - - @prefect.task - def finalize() -> None: - r.finalize() - - with prefect.Flow("Pangeo-Forge") as flow: - prep_task = prepare() - cache_task = cache_input.map(list(r.iter_inputs())) - cache_task.set_dependencies(upstream_tasks=[prep_task]) - store_task = store_chunk.map(list(r.iter_chunks())) - store_task.set_dependencies(upstream_tasks=[cache_task]) - finalize_task = finalize() - finalize_task.set_dependencies(upstream_tasks=[store_task]) - - return flow - - def execute_plan(self, plan: prefect.Flow, **kwargs): - return plan.run(**kwargs) diff --git a/pangeo_forge/executors/python.py b/pangeo_forge/executors/python.py deleted file mode 100644 index 8e6625e8..00000000 --- a/pangeo_forge/executors/python.py +++ /dev/null @@ -1,25 +0,0 @@ -from functools import partial -from typing import Callable, Iterable - -Task = Callable[[], None] - - -class PythonExecutor: - def prepare_plan(self, r): - tasks = [] - tasks.append(r.prepare) - for input in r.iter_inputs(): - tasks.append(partial(r.cache_input, input)) - for chunk in r.iter_chunks(): - tasks.append(partial(r.store_chunk, chunk)) - tasks.append(r.finalize) - - return partial(_execute_all, tasks) - - def execute_plan(self, plan: Task, **kwargs): - plan() - - -def _execute_all(tasks: Iterable[Task]) -> None: - for task in tasks: - task() diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index 2af0ebfe..5c9b322a 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -10,6 +10,7 @@ import fsspec import xarray as xr import zarr +from rechunker.types import MultiStagePipeline, ParallelPipelines, Stage from .storage import InputCache, Target from .utils import chunked_iterable, fix_scalar_attr_encoding @@ -92,6 +93,14 @@ def __post_init__(self): k: v for k, v in enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk)) } + def to_pipelines(self) -> ParallelPipelines: + pipeline = [] # type: MultiStagePipeline + pipeline.append(Stage(self.prepare)) + pipeline.append(Stage(self.cache_input, list(self.iter_inputs()))) + pipeline.append(Stage(self.store_chunk, list(self.iter_chunks()))) + pipeline.append(Stage(self.finalize)) + return [pipeline] # type: ParallelPipelines + @property def prepare(self) -> Callable: """Prepare target for storing dataset.""" diff --git a/setup.cfg b/setup.cfg index 9c7719f5..d2063367 100644 --- a/setup.cfg +++ b/setup.cfg @@ -3,7 +3,7 @@ max-line-length = 100 [isort] known_first_party=pangeo_forge -known_third_party=click,fsspec,numpy,pandas,pkg_resources,prefect,pytest,setuptools,sphinx_pangeo_theme,xarray,zarr +known_third_party=click,fsspec,numpy,pandas,pkg_resources,prefect,pytest,rechunker,setuptools,sphinx_pangeo_theme,xarray,zarr multi_line_output=3 include_trailing_comma=True force_grid_wrap=0 diff --git a/tests/test_executors.py b/tests/test_executors.py index 551abaa5..b21ad56a 100644 --- a/tests/test_executors.py +++ b/tests/test_executors.py @@ -1,14 +1,20 @@ import pytest import xarray as xr +from rechunker.executors import ( + DaskPipelineExecutor, + PrefectPipelineExecutor, + PythonPipelineExecutor, +) -from pangeo_forge.executors import PrefectExecutor, PythonExecutor - -@pytest.mark.parametrize("Executor", [PythonExecutor, PrefectExecutor]) +@pytest.mark.parametrize( + "Executor", [PythonPipelineExecutor, DaskPipelineExecutor, PrefectPipelineExecutor] +) def test_recipe_w_executor(Executor, netCDFtoZarr_sequential_recipe): rec, ds_expected, target = netCDFtoZarr_sequential_recipe + pipeline = rec.to_pipelines() ex = Executor() - plan = ex.prepare_plan(rec) + plan = ex.pipelines_to_plan(pipeline) ex.execute_plan(plan) ds_actual = xr.open_zarr(target.get_mapper()).load() assert ds_actual.identical(ds_expected) From c2927be41f69f63ec4b52992d66fe2dc02f0f600 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 18 Jan 2021 12:41:20 -0500 Subject: [PATCH 21/34] add rechunker to CI --- ci/py3.7.yml | 2 ++ ci/py3.8.yml | 2 ++ 2 files changed, 4 insertions(+) diff --git a/ci/py3.7.yml b/ci/py3.7.yml index c310ff4c..807f4192 100644 --- a/ci/py3.7.yml +++ b/ci/py3.7.yml @@ -27,3 +27,5 @@ dependencies: - toolz - xarray>=0.16.2 - zarr>=2.6.0 + - pip: + - git+https://github.com/rabernat/rechunker.git@refactor-executors diff --git a/ci/py3.8.yml b/ci/py3.8.yml index 739ce1ef..bbdd3a6e 100644 --- a/ci/py3.8.yml +++ b/ci/py3.8.yml @@ -27,3 +27,5 @@ dependencies: - toolz - xarray>=0.16.2 - zarr>=2.6.0 + - pip: + - git+https://github.com/rabernat/rechunker.git@refactor-executors From 9a1d11f41a7be9b508496ed6d70c185acdfc431d Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 18 Jan 2021 12:42:46 -0500 Subject: [PATCH 22/34] add rechunker to requirements.txt --- requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/requirements.txt b/requirements.txt index 11a36cbe..83520897 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,6 +2,7 @@ setuptools click dask distributed +rechunker xarray >= 0.16.2 zarr >= 2.6.0 fsspec[http] From 1879acb2a2a6ecf928c6fba9264f736f5c833e1b Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 18 Jan 2021 16:55:03 -0500 Subject: [PATCH 23/34] create ABC for Recipe --- docs/{design.md => concepts.md} | 13 ++++++- docs/index.md | 5 +-- pangeo_forge/recipe.py | 61 +++++++++++++++++++++++++++------ 3 files changed, 65 insertions(+), 14 deletions(-) rename docs/{design.md => concepts.md} (93%) diff --git a/docs/design.md b/docs/concepts.md similarity index 93% rename from docs/design.md rename to docs/concepts.md index fa81f088..65da4a82 100644 --- a/docs/design.md +++ b/docs/concepts.md @@ -1,8 +1,19 @@ -# Design +# Concepts pangeo-forge is modeled after [conda-forge], a community-led collection of recipes for building conda packages. +## Recipes + +The most important concept in Pangeo Forge is a ``Recipe``. +A recipe defines how to transform data in one format / location into another format / location. + + +## Storage + + + + ## Components The high-level components of pangeo-forge are: diff --git a/docs/index.md b/docs/index.md index 14b7ad31..6b629e03 100644 --- a/docs/index.md +++ b/docs/index.md @@ -4,6 +4,7 @@ :maxdepth: 2 :caption: Contents -contribute -design +concepts +contribte + ``` diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index 5c9b322a..adee9124 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -3,9 +3,10 @@ """ import logging +from abc import ABC, abstractmethod from contextlib import contextmanager from dataclasses import dataclass, field -from typing import Callable, Iterable, Optional +from typing import Callable, Hashable, Iterable, NoReturn, Optional import fsspec import xarray as xr @@ -49,14 +50,60 @@ def input_opener(fname, **kwargs): yield f +class BaseRecipe(ABC): + @property + @abstractmethod + def prepare(self) -> Callable[[], NoReturn]: + pass + + @abstractmethod + def iter_inputs(self) -> Iterable[Hashable]: + pass + + @property + @abstractmethod + def cache_input(self) -> Callable[[Hashable], NoReturn]: + pass + + @abstractmethod + def iter_chunks(self) -> Iterable[Hashable]: + pass + + @property + @abstractmethod + def store_chunk(self) -> Callable[[Hashable], NoReturn]: + pass + + @property + @abstractmethod + def finalize(self) -> Callable[[], NoReturn]: + pass + + def to_pipelines(self) -> ParallelPipelines: + """Translate recipe to pipelines + + Returns + ------- + pipeline : ParallelPipelines + """ + + pipeline = [] # type: MultiStagePipeline + pipeline.append(Stage(self.prepare)) + pipeline.append(Stage(self.cache_input, list(self.iter_inputs()))) + pipeline.append(Stage(self.store_chunk, list(self.iter_chunks()))) + pipeline.append(Stage(self.finalize)) + pipelines = [] # type: ParallelPipelines + pipelines.append(pipeline) + return pipelines + + # Notes about dataclasses: # - https://www.python.org/dev/peps/pep-0557/#inheritance # - https://stackoverflow.com/questions/51575931/class-inheritance-in-python-3-7-dataclasses -# This means that, for now, I can't get default arguments to work. @dataclass -class NetCDFtoZarrSequentialRecipe: +class NetCDFtoZarrSequentialRecipe(BaseRecipe): """There are many inputs (a.k.a. files, granules), arranged in a sequence along the dimension `sequence_dim`. Each file may contain multiple variables. """ @@ -93,14 +140,6 @@ def __post_init__(self): k: v for k, v in enumerate(chunked_iterable(self.input_urls, self.inputs_per_chunk)) } - def to_pipelines(self) -> ParallelPipelines: - pipeline = [] # type: MultiStagePipeline - pipeline.append(Stage(self.prepare)) - pipeline.append(Stage(self.cache_input, list(self.iter_inputs()))) - pipeline.append(Stage(self.store_chunk, list(self.iter_chunks()))) - pipeline.append(Stage(self.finalize)) - return [pipeline] # type: ParallelPipelines - @property def prepare(self) -> Callable: """Prepare target for storing dataset.""" From 0fa41ee1390616f6d2889712b1646b70b62f6084 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Mon, 18 Jan 2021 22:49:56 -0500 Subject: [PATCH 24/34] start working on docs --- docs/bakeries.md | 1 + docs/concepts.md | 75 ------------------------------------------ docs/conf.py | 4 +++ docs/execution.md | 1 + docs/index.md | 38 +++++++++++++++++++-- docs/recipes.md | 15 +++++++++ pangeo_forge/recipe.py | 51 +++++++++++++++------------- 7 files changed, 85 insertions(+), 100 deletions(-) create mode 100644 docs/bakeries.md delete mode 100644 docs/concepts.md create mode 100644 docs/execution.md create mode 100644 docs/recipes.md diff --git a/docs/bakeries.md b/docs/bakeries.md new file mode 100644 index 00000000..efdfd4d7 --- /dev/null +++ b/docs/bakeries.md @@ -0,0 +1 @@ +# Bakeries diff --git a/docs/concepts.md b/docs/concepts.md deleted file mode 100644 index 65da4a82..00000000 --- a/docs/concepts.md +++ /dev/null @@ -1,75 +0,0 @@ -# Concepts - -pangeo-forge is modeled after [conda-forge], a community-led collection of recipes -for building conda packages. - -## Recipes - -The most important concept in Pangeo Forge is a ``Recipe``. -A recipe defines how to transform data in one format / location into another format / location. - - -## Storage - - - - -## Components - -The high-level components of pangeo-forge are: - -* https://github.com/pangeo-forge/staged-recipes: The birthplace for new recipes. - Anyone interested in adding a dataset to pangeo-forge can submit a new recipe - through a pull request, using the examples there as a starting point. -* Recipes: Metadata describing a dataset and code for transforming it from - raw input to analysis-ready dataset. -* pangeo-forge: A Python library containing useful pipeline helpers and the - [pangeo-forge command-line-interface][cli] for validating pipelines. -* https://github.com/pangeo-forge/docker-images: A repository for building docker - images for pangeo-forge. - -## Pipeline Structure - -pangeo-forge uses [Prefect] for - -1. Pipeline definitions: the code that expresses the transformation from raw to analysis-ready data. -2. Orchestration: everything involved with taking a pipeline definition and actually running it. - -This imposes a few constraints which we build upon. - -All recipe definitions must be named `recipe/pipeline.py` and must contain an instances -of a `prefect.Flow` at the top-level of the module. - -Additionally, pangeo-forge imposes additional strucutre. All pipelines modules should include a `Pipeline` -class that inherits from `pangeo_forge.AbstractPipeline`. So most pipeline modules will have - - -``` -class Pipeline(pangeo_forge.AbstractPipeline): - @property - def flow(self): - with prefect.Flow(self.name) as flow: - ... - return flow - - -pipeline = Pipeline() -flow = pipeline.flow -``` - -## Lifecycle of a recipe - -A maintainer contributes a recipe to [staged-recipes] through a pull request. We -use GitHub Actions to perform some initial validation. The actions are at https://github.com/pangeo-forge/staged-recipes/blob/master/.github/workflows/main.yaml and the logs are available at https://github.com/pangeo-forge/staged-recipes/actions. - -When a PR is merged into `staged-recipes` the [Create Repository](https://github.com/pangeo-forge/staged-recipes/blob/master/.github/workflows/create-repository.yaml) action is run, which - -1. Creates a new git repository with the contents of `staged-recipes/examples/` - inserted, along with a few other files provided by pangeo-forge, including the `register_pipeline` action definition. -2. Pushes that new repository to `https://pangeo-forge/`, for example https://github.com/pangeo-forge/example-pipeline/. -3. Executes the [Register Pipeline](https://github.com/pangeo-forge/staged-recipes/blob/master/.github/workflows/scripts/register_pipeline.yaml) action, which registers the new pipeline with Prefect and executes it. - -[conda-forge]: https://conda-forge.github.io -[cli]: https://github.com/pangeo-forge/pangeo-forge/blob/master/pangeo_forge/cli.py -[Prefect]: https://docs.prefect.io -[staged-recipes]: https://github.com/pangeo-forge/staged-recipes/ diff --git a/docs/conf.py b/docs/conf.py index c46ec4db..2dfe4699 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -11,6 +11,9 @@ extensions = [ "myst_parser", + "sphinx.ext.autodoc", + # "numpydoc", + "sphinx_autodoc_typehints", ] templates_path = ["_templates"] @@ -21,5 +24,6 @@ html_theme = "pangeo" html_static_path = ["_static"] +html_sidebars = {"index": [], "**": ["localtoc.html"]} myst_heading_anchors = 2 diff --git a/docs/execution.md b/docs/execution.md new file mode 100644 index 00000000..7bb5b258 --- /dev/null +++ b/docs/execution.md @@ -0,0 +1 @@ +# Recipe Execution diff --git a/docs/index.md b/docs/index.md index 6b629e03..2718b2bd 100644 --- a/docs/index.md +++ b/docs/index.md @@ -1,10 +1,42 @@ -# pangeo-forge +# Pangeo Forge + +Pangeo Forge is an open source tool for data Extraction, Transformation, and Loading (ETL). +The goal of Pangeo Forge is to make it easy to extract data from traditional data +repositories and deposit in cloud object storage in analysis-ready, cloud-optimize (ARCO) format. + +Pangeo Forge is inspired by [Conda Forge](https://conda-forge.org/), a +community-led collection of recipes for building conda packages. +We hope that Pangeo Forge can play the same role for datasets. + +## Recipes + +The most important concept in Pangeo Forge is a ``recipe``. +A recipe defines how to transform data in one format / location into another format / location. +The primary way people contribute to Pangeo Forge is by writing / maintaining recipes. +Recipes developed by the community are stored in GitHub repositories. +For information about how to write a recipe, see {doc}`recipes`. + +## Recipe Execution + +There are several different ways to execute recipes. +See {doc}`execution` for details. + +## Bakeries + +Bakeries are cloud-based environments for executing recipes. +Each Bakery is coupled to one or more cloud storage buckets where the ARCO data is stored. +Bakeries use [Prefect](https://prefect.io/) to orchestrate the various steps +of the recipe. +For more information, see {doc}`bakeries`. + ```{toctree} :maxdepth: 2 :caption: Contents -concepts -contribte +recipes +execution +bakeries +contribute ``` diff --git a/docs/recipes.md b/docs/recipes.md new file mode 100644 index 00000000..f053c335 --- /dev/null +++ b/docs/recipes.md @@ -0,0 +1,15 @@ +# Recipes + +## The Base Recipe Class + +```{eval-rst} +.. autoclass:: pangeo_forge.recipe.BaseRecipe + :members: +``` + +## Specific Recipe Classes + +```{eval-rst} +.. autoclass:: pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe + :show-inheritance: +``` diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index adee9124..e42fbb7d 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -51,40 +51,53 @@ def input_opener(fname, **kwargs): class BaseRecipe(ABC): + """Base recipe class from which all other Recipes inherit. + """ + @property @abstractmethod def prepare(self) -> Callable[[], NoReturn]: + """Prepare the recipe for execution by initializing the target. + Attribute that returns a callable function. + """ pass @abstractmethod def iter_inputs(self) -> Iterable[Hashable]: + """Iterate over all inputs.""" pass @property @abstractmethod def cache_input(self) -> Callable[[Hashable], NoReturn]: + """Copy an input from its source location to the cache. + Attribute that returns a callable function. + """ pass @abstractmethod def iter_chunks(self) -> Iterable[Hashable]: + """Iterate over all target chunks.""" pass @property @abstractmethod def store_chunk(self) -> Callable[[Hashable], NoReturn]: + """Store a chunk of data in the target. + Attribute that returns a callable function. + """ pass @property @abstractmethod def finalize(self) -> Callable[[], NoReturn]: + """Final step to finish the recipe after data has been written. + Attribute that returns a callable function. + """ pass def to_pipelines(self) -> ParallelPipelines: - """Translate recipe to pipelines - - Returns - ------- - pipeline : ParallelPipelines + """Translate recipe to pipeline for execution. """ pipeline = [] # type: MultiStagePipeline @@ -106,34 +119,28 @@ def to_pipelines(self) -> ParallelPipelines: class NetCDFtoZarrSequentialRecipe(BaseRecipe): """There are many inputs (a.k.a. files, granules), arranged in a sequence along the dimension `sequence_dim`. Each file may contain multiple variables. + + :param input_urls: The inputs used to generate the dataset. + :param sequence_dim: The dimension name along which the inputs will be concatenated. + :param inputs_per_chunk: The number of inputs to use in each chunk. + :param nitems_per_input: The length of each input along the `sequence_dim` dimension. + :param target: A location in which to put the dataset. Can also be assigned at run time. + :param input_cache: A location in which to cache temporary data. + :param consolidate_zarr: Whether to consolidate the resulting Zarr dataset. + :param xarray_open_kwargs: Extra options for opening the inputs with Xarray. + :param xarray_concat_kwargs: Extra options to pass to Xarray when concatenating + the inputs to form a chunk. """ input_urls: Iterable[str] - """The inputs used to generate the dataset.""" - sequence_dim: str - """The dimension name along which the inputs will be concatenated.""" - inputs_per_chunk: int = 1 - """The number of inputs to use in each chunk.""" - nitems_per_input: int = 1 - """The length of each input along the `sequence_dim` dimension.""" - target: Optional[Target] = None - """A location in which to put the dataset.""" - input_cache: Optional[InputCache] = None - """The length of each input along the `sequence_dim` dimension.""" - consolidate_zarr: bool = True - """Whether to consolidate the resulting Zarr dataset.""" - xarray_open_kwargs: dict = field(default_factory=dict) - """Extra options for opening the inputs with Xarray.""" - xarray_concat_kwargs: dict = field(default_factory=dict) - """Extra options to pass to Xarray when concatenating the inputs to form a chunk.""" def __post_init__(self): self._chunks_inputs = { From 4ee634a4806b569310d43f87d04f40a1a4222d27 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Tue, 19 Jan 2021 10:39:46 -0500 Subject: [PATCH 25/34] writing more docs --- docs/_static/pangeo-forge-logo-blue.png | Bin 0 -> 17602 bytes docs/api.md | 26 +++++++++++++ docs/conf.py | 13 ++++--- docs/index.md | 1 + docs/recipes.md | 48 ++++++++++++++++++++++-- pangeo_forge/recipe.py | 20 +--------- pangeo_forge/storage.py | 33 +++++++--------- setup.cfg | 2 +- 8 files changed, 96 insertions(+), 47 deletions(-) create mode 100644 docs/_static/pangeo-forge-logo-blue.png create mode 100644 docs/api.md diff --git a/docs/_static/pangeo-forge-logo-blue.png b/docs/_static/pangeo-forge-logo-blue.png new file mode 100644 index 0000000000000000000000000000000000000000..467758a8305bbc71aac9b174d10be530bc8ed8c1 GIT binary patch literal 17602 zcmdSB^;?wR7dA==DAGtt2@EAQLpLhj0!sG~j&wHz(kbhz}_12W{#HJ4$jt@ z2jVZ$&|aY_%DnyHnYFj*;cM`B_VIYw4>UWJLsat%jrZyMSC$|_*eCY&d@TK!q5%i* zlymvr6W&c%y5*|9n)zNk^8IwXuxRA?eO?*1x!9AZSW2&a5+v-9T4POcI9v1+Ld8u5 z`Gk(cY2`}t-Gynt-kNgdP2*RL|NnlqeR(E|P8`oYXz0lqOB+@94RRq|MeKNkX)5yc zfT2tz7!srVGw^0!5GSkJ6_wD`%P-q!Mit|P4A{oRLnFmT6cMk>L59=}FF%@(mW$ti z^Xp{cdaGfUFl_Y!ueV2jRVI>eE-r?a@x|x->k&0=&RDD3O4lqthbIiNNGEJ&H@}5F z%$9V^$y>gx*?82SXVo@)Y=$njV|HaW;dtIT%*hZFmnI8)$RNsd;TM9gQlNGWX^UEw z9c)KA*3KrI72xxMFBmzkgG6YBS&Aq_u}zt$9&l_@h~gv=Ct@%85-MHoHX)*7hkbKPR4Tm-=4B`)jk}1DgAHGI;FemHDL;w z>7%UAz!)Mj*#$)aB6*NVQAT6!or0{`GX_a^C;YC*v+yXSBC}pUvNpB-ZHHfJ-WXam zseYB947;8*J;1;&9C?lFPV6RNa?BjYbdnC|?(-)ZpfRl)9-W7XM-H^r3O&Y?MH%?U z@>Hg4KwgB!q0!@fx6TWbpnM2$f)9<&oBVm>+fV#j%^q*RBmvyhIX~M6U8glQ0%M`D)FXI--^Y#(+yOhoTKjZ91o*H1h+e)&XXD6?+06>AW~vJ2T{7W~ zc7s?^IC+o_go>{X(1xc!kL&H}c;ZXGwJu`pOoyyCv4*J{rQ9TPYMLd)DRDgD8q&GG z%=Ub7#p^aXY}HzvZX%9OYJi{fxFY{7oz%1Db16N{t)#YqYHJtrdwt@g3EE686R6!` zcg)^lPrju5T6J#UWFT-x9o#ygl~p^tueCT6cnn)h3?C>k;kO+&=zaOGoqHC@bedCPuHLwdGaO!Nd*vdUyfN5credV~K5fdLo zZRziySGfcYZE)dPlenjVZ=;ac3@6q%|oG^_^I{g{#7iZ{hl2bR*I{|p zp6nE4Tpf>ns%4eB`Ct@iD_VJ{vcd-pSKyR%Hf=E@UIaW{6r6S|piT^6AQtxpYqpwh zh3SoIi7GZ;N)XgPAyNAMW^yXV;d9X#{%96d5>B>^T12M7ku}UeWi}*7yGqgYLz=<` zrB7Y)Bq4+36N?kWJd#MUrQFV|ho4zWijN_0i;u~jSTCKGQtDhz%^%bRvxE-$Ri@(F zkCf?E?oQvbGL&eUG0uKa;ZJ!`OBC$Bjd^445Y-Cp=xq9{6*RQw_7fG?JL9`_^qGxT zHutG$M46*X>-^^8h~bG(*wJn}A1mSE#hJZV&9kpB@VapVc9Q0q>mgFe$KWM%!OfJo z&5{gNy{!SBUSSmXif^l}&S7ffJ=%l+)@+nOoT=x_Zow)Xw5sn8^Mr8`5@l>FiTF52{VMi0^9 z8_BAd*H!58xy#wQL5Gs23K@`&u5wXsJs^^_BplFekfHf`v!`-!EVd!E>YC{2p7ZQ4 zeYY4J*W(K}0jcMrWOtbc&P2Kj79)q=;^;U39GFZ6a^`StICx$w%}*Mhvf%YAOta8i7nKvasrQ_fD9(@e57KXYc824zmJFPAs9(C#7I?uoW@iYdQ

Kyh&O|n-nl)4? zccr6kiImZ-6fgLpy1PVW8)t(-NH(q`#_t(^;)ISx^9vXavNn$8Y6+_ zDfQS_T^z4u#tVJGt70u29mTEkQLF<8CW-(jHf2gY)jJox29Hq&AIw@s^ODIr;9T_W z?Al!Ef2uWXUO(r4P_P*tF*rU(q&5+U(raX{8MRHH#k9&q4X!c~nEQOPTkMSN9i`(I zpAH*JadAA7FDczqXtJ~$bYLr`4Et+9=i*5{Ub#CE*~>+@u8{H{IDUh($&GdMqc2`6 z^HNay#08(z*n&wG4IrpdB8YH^@Tzf!me~^}Qr4>=y7uE03!!93>3am_J=k{H~E}u>I>HYf6ADJ*6 zjKjq^cSQB7P3tS6e7^c)VgQAB&sWZ|A2f_$}@pXx>w25a$5#@W8N~M{eSFciN zVRgN}X|u`KdX3ZvJI;wKnlxK6raQk?eSHJU6 zI8XkgXW0Lwqd0qBVc)JUhmo_yw%a)lDUPUHmq30k(=cN!slHI8{{rfe@A(?4tmx4G zhRLmg-%Mqv8~qtmINhIOuAOCD5G_h9Taq-!`og{b*o-crPTeb3L#K^I*~3Gk-mD3K zI!n>+ynjL)F1CHm+4gp~ef`K=e(?(M&5qHR!fLqmp&&;91O-Cn4k0Qhr7U9-=hKd! z82_n04)%$@J%Ewm++NliP)quFUQROUWzLGRtoT7|Vo`&V8|Vy|rAOa$b)+$;?l5eY zWf$|-ODc_j)%<|pj_Kq(MYeXtazCha2_Co~zNN8COO*!z>T~pYyJd?&bzud^?HJL@ z*xAjB<6Co`qDeZ$%z%is{d@V=uvcf&g!{CHr!H1HDKy8Ej!xfM(Yz&u%b6n-)BE+j zZ=;0Wac4LnDA%r_P%V>l_8l9#u-|xUwd_yo<2H%>eZ$eKP*X~wYhbg(xEfN}= z?^i^HB25FL6^x5qTU?Lmh*d8_7l~Vv$X5ERk8{MWRR@%K9Dy*HT$)Pl>HtE>_))3Q zfnyX~8RdR@=54R;tgMt+S8Ijim|F?)33}i9bobn)^xQfpmb9j)WjRkU;zOiwziiFB zkO+_istd{>Iq&o8r&QDK*i6fYK&;U-&{CQbhdVmcSY)EG)0NIP^J;uI9w*^fe>03D z=HIBt*ts79Xq+X{8v50lt{wCkd=$~$)k~M36+iy*izxnV-0$7NsQ0}~Cc%2GVdY+1 zg3?1PZaT|Al1}A@nyFXXL0`fmHAwMtFJ<;mhExh`Y5n(Xw+*gSzVv! zj=j4kkTT>Cuwt$Rp$$Co++O;7$gEH_+tK1DSzu>K{sq#|J&)=OXnT)Kwwr`8DK9EEXjT`U5@8A$+}t8k)<(do@MZ&^27qK2rSTy%{5xYKWqPMY)ToIc&P? z*nI8ay`cy8{58hRxbLo%MUL9qk5Y-VswAO9X=3Np74I}nv;Dd!T4};Wl2ljtD}O7V zt!VSLoVccZ=&Q2*HE?QftrHqScSq4zSpD2F!8q3sx+Gm|_7p4y^@~s_u6A9a{)`Fr(Q+>#Kj%U%o!nW))-ShR(8+R7Mkj= z`>nYt-D`}U$}+AA0fCmP&41Ka?;CWI&N`k@9^}Uj5mrW>vaMLUR6`>K3|I2~yrUIq z3I$(1h#i@+st$bVejJ{sVqs*YDga`T?G(B&r)VSt>AGus%x%q%c2%5rq+S1mW2_}V z-|L&B7217-ourH}#m{y}lBxf|#PL~$z+-=xqrJ~{7L6hI-`~#gH{{_U+Rtv}%(G8L zvs|!>gb0o(nLdUpuvLF(H2DOn8g>yEx`R3T2}f(l^uU3YQ;P=dHg4t|TqbkJ8e7(ZRCC=?8J$Z9A~|*R*6;7gsw8P*Ss2%NfK}r^ z|F6`)9$6Y)JSiNebLaZaf7Je{k&!ukl?Letb*ZaQnGfKiW1>^m+TeAhJED}A>G1$& z{>j;(rL%a--UJvNrU5Fy*lyxurNWx1L$^riX}SQsteCRId5}7!AMi+&w*teE@>?pJ zOZuF3i$u4&^%4c1ylfL@EzUFt0Ec37c(AIbSIXb}r_^9sU-;_za!ifa7tbyiEX^?u z*8xciR}on*&XA(ie@$o$A*OSRmrpire_AOUAL#BD>-MZeS{R$ufndA~edXuG$9wi| z@;-0szQBR%gTIQd8`ac^xwwSEt{&n0xFNLlp>EOhLX$sKhMV1pj({kMzaBClNH0C- z{HEcFDIYszV7MUkC);+j{z}uidKa^kg$lw?eo4z>y02c3e>n%^?5IYsOT-R#Gf{d&79rc& z8P#{(A-kCY+BEI~vtQQBHu`0${rW*uo)B;Wa@TqI=^SC}^DAren;mcG&qjkmN!oYh z4mk8T$e9?)g*$Jak!yl>c8j%c#U53<`OQk^WV@u4Lo~kM(n);j|MG|qvOeS0YwO>Z ziO8o3vaeOVnYz(;epC{Q;-i)M#PE=62CW2FFmr@NZ2Pip3J@26`Ok;J(H7;h9q2aY zJ>N}$FdmGKR7&|+3>SP4yY5{%b~Ux}zAU!2nKD1pOcU-|U&gLb6MMIPH&At9ODOVY zZD?3D4g!kePh#p6u|9YMKExM|Ih(!2s*=4o;A*ntN$~(5p>`3UHP#9ZJ*mlPp5wS7hGN zBN7(B?cGmOkowRc?dFl@x8K-X4L!c2b9QjJ~qJ(yF4@8>D**7e3$%erNFM^JmlEH8udt5HfL{8SE=5fnM}eNXyL%KhARZ z?|z(9*X<5o7GG>kt$MR3bl~j+Ha8-1qcjfa!}N4`X}j#(S5Ga;Jc7J{BoA*iWwA_m zp({&-;etjgICiFAcu0klzI%8dv&^N#p&G>TZv%ijfjF3Bi{JN1Z75xgWebF+tN#J-9*2M$=`drV5+Tr|=L~SD%n5 zdKd9#(%+w4yAKHu{g}K8iIylqJWce^L^0|8IJcfG^^o8i$*+hR&1IP0UKYkQ@+5M* zn1tsqN}8D5gJho&mH)Eg!`2hN&a5?v5@}Uu7*adV3rz)Tb-gXwbBs)k z68ut*#~*gI;uCO3gA2J(Q&&ztjZj6&C?{^b&1&kq%PDGTqj4oq5LNY0-g7yIj_vCF zSNf2@$u?dY^?uBOB8N3P!{%3!dWu_6DVv?A)LUB5hC*Js*)PM;Mb`e(n9qCeY}f81 zdK*d2KRUFukK=j_b6ZEBtWpQ5y&L-`#*$$4lcU>Iasv{{Z1I3nmgFFpulCHXbGUly zg0gUL6=zQ7=AOo+4J>-^W5D%QyY|1yV*`OXLJG4+9 zfP9^^m6pJDaXG!A#E<@Xogm@6-Y_<%SwWN$u;AgEok!C@ml%Wj<<~5ExA(lxA$2BJ zv-`8c=C2+(JJvsPPNN;?X(@IMmsH+-98pxb`4^7pA#cI4b^b3%9KuhD{8h*W+VM25 zcM*ugFMjM*4?3K&YtB?h?@UENy;JW^T$F*u*52%dK`ux=muH1sqOVyop+<*;j9za6 z@rL)PZ=idYK!tO67(8T?^08rUI9=!vdVzoM7rvgGoJ`w@(t2{O=%IJ+Z@cvlJp3d@ zdK&PG8ai6Uu8dy!^(Z0TQ}!=3jC*s-PYGMU;2eG;6eT{E1BO zaXu-9r`}UK_qe<5)HD0=cd%-NM+xHTM818n>k&BoUU5^}W-8rkEFGdK zSz~)dY4(%HtMK>CPjR^!Y0jJHCe9*r31WnSO+b?1@|o477SVSmee?c3fC*oA4dbS@ zX77Fh87iI1@rUSpf{6Q1HNV6oRu_S)=l1TC9igatTTDVM!;9KgxpBCJO$R*SW@p!F z5Q}oh?+*u+PXIN{Ml!_m<90R-B%%yRfGZ+}F?Rogws&RN?*0SSx+MwfWtk8Az~QGe zuH1&F!#KC{;;IKfmBsFBRt?^8#sp-B2j8>zrF@hK6?m@b(H`;lnVkLZ-E!)WxkLKA zdX$Zf7>ns3#;ZhXGXV@K-o9p+9i8Xp`xnto928JhU#0z=5zxv{5t zm=Dqis1OvFAu_1LYt;)YZnW)$Owvw1tuu*q?O}CCG*PT&JTt{P^K8qf?G^+p|5vV` z7=pL_Sx%N7kHS1H93m?rRHfRuhpdWiemiWruf6Z}mnkCiXJEc*=nBzxbr5ogm&S!4Fq|3@c4X$@aB-ScIOK2f1#4{LRgprhIA=6;v7xF@ zptp%^AVTzgQ+M;dQUv+JQPjk8GkRrr-2c6SgrzJ3?z36jN=Y$`J);BjQuVRZ9sY>lTgrA2u{KZPQ1eW^*=_ayrbx+s|^t0V3Rr6YF& zHqd-Ibor01UzCi6QN)h($bUr#eUkwO6x$xS#H_~<)#9UQu)_B~fcQOfaMUgk-{YbcwfCAqBDP)KZj-cNS6(HI`0#S_gD zSa{rrv={$fwc7=U#&&JbUq|W?_e+wpe+ftZxMlV?Wwl?=$r~riEdjH?ueP%Zw=+@x zH1jF4FIUMkcnFs?9%H31Vt-Bs*+&sUKhm20uCNRA=+-8ZSl4seuC%+P<8(v#{nH&k zBb3Fg>!yc=@w(SeLdNw}NwmJKi$#cLDrY+j#K&(V^2mio&g@hDOg=n1Tjlr)CVY=$ z4dx;yJ~a1f7_Z)DE)Wwc5_P&VH1)EoNl*crF^`dfQkaK|b!Jy-kQb_Hauz+upP&T9 z{bYi4OgB6FqJm|hrCO;;6!(!7Lg;>2rRMFST*!Jh!f>Un(9<|}+~HO4y88D|3l=Jc z!c6@U;b%Q|_Jw}q1Fp8lAu->5gGU%vrV+G=*UO2CK!r=wQx1I^Jr^gmcx)v2os67n z;YQ*3Q+%3?_zfAESn~OX#0z|m?U{B+wx977rj*2;!u3Vu1|1noNGavjtM}+xIiJ^$ zR#so+I^y<1)zQ|ka;=h-)+f(v%i$58>wzCKTR;RnMtBQ2J@-TEU%ml#;WUCj zn37LqG+|pwXGNY}FO*nvqO}YYfH?~BYs+{OQ^$<8yGfMW=#V&wr+q%1 z4z=+|TZkL_vL)~#?UyS_qGcu4P2t!G`?hrdRpOS**oQ31d)|bs@QqwgEzI&=<9xgI zm$sZKpHHz@;DIkqza91~4gu+1r3m@e6Gu!(UqIUR_x#>8erjndqfcpR*v2^Yly76p zYb(6cV~ys=BdTz(AV++8v(pQpN+9t2783A?(T-;L_4E3#uV!Z=PQ1+P>J#8N1s+o| z(NBxDu_==r5kke{_uW`@uOR8?5Pm}@;IV^jbYJpD?5#{C>Egm*NmBKK!w9XX=exOp zWdjk$G>Z#EhZ?fZ*q++-$8k|YvtN4zOv!)|3Wiw0s%*8|E8zrcK z+D-r}XKG^2|*BwOr=L)F=QTCAd@R`+t5%m!&s-w}CEUTzec zmVk*Xu@h!A>S@w4G?f7`I1s*3Pgm+sgQWjbmC>{?O06$95UJfE6LmjSW9`8u@rk=b z55LLCI>UXf(!)a+=7l|W*h9K41S3(?&|o%7E{Jn~-Z zqI{gkN&y|K2oqoNN5ssB0_Q=7@hlUZ^Pp8sLM6^d+7$6C({*ninXjWgTH{1I<)&rV z9@MiCPmkf_r>^GB{I3Ux89!`=VvFX{j*J~8iapN^GK2;7A zmS(BHiGJui3VRYK6tDi)buliEn)vjm@RDTKw-GA96xBQtXgBJT@ehe=UF=i9sc2r~ zDyPwJOZon>)M;K*a<&1#5l8n)>z0o*V^|3|-ZoRgJbZH;QT5?v#{2Mg(mu%6L}COU zRTgk8a5Zi#7Wvy-Gg1Z(5UPN3@j<}h$KE@Y5Q+Z=EH^I+O$YsXz__NJq4*jlwp!-9 zWhW>k)<0S{79|JeuUwWHA1K`x{(92ie$?kU==bKgO^EDRQO8!o^Y4kegQsSiG^l6i z``rQTth`4Xior^ed2F_m81GS%;+d2aO`nsmx*ABQkyeACd|72(*=9?4ITnISyPu>5vryfps!x5X-q3zh0}38c^7)#gWprcRC2dSlx2 z|D9}(K`epC&Ag<;|>G$#@w#2rnv z8ZUM1Hdf%;q;@JzIgWui3S|{uc=*!_bb!p)dn*5fC(W0o&(7J+Mv|W4NuJAe-QQY) zM2LAr3HY5I|9ZbPQM749y*jeV!<_+;l#vR^zi~TFSD+x4fmr2|BFGuFPy$17w#;%h z6K{N`!bw??*C_!(Yhy}EdCOSD5zP_lO2eCf;%;D&v1lsLkg!t2(LE{HAABk3`GPx( zvk-{>bw~a`OnNUI?21>DV`%P%MGI8fr%7oM%L<(5eiN4)GBqk85ifr9gD#}6UySf1 zhpK4=ztuwWot-6y2Y8kh}n<-NKI7 z9pocS`wH0kmJmueG{>Luo;CXSo6lp^vyIB*q5CwxFwDVi{OPh+P97W6Ac84Ub~d2F z*5DZ6k-+wzo4GglzJLLf(tAz@TRk&?R1r4qON?gDWk=)d#fM|8D}VOh=EtB3bwvsK z4q&mMFyQ^n|GokXyKoHrfyV5p_8FeUDTwjD4wiXV<~EIfQLZaJ-fX8kty*`j zy@Euj2THWFvL*xphuA?dprwG zUIrPcKJpQh=p3*(8lb}fI`(lS22(Vq^;D^AVyjaPY&{>Iu(ERw{-pIh^RYR)UV=T_ zWE|s;;ELhMT#kga9}|e}VqF;{CdG3>5T|WCK+EWh)&k9r*m)J#~ z+|e~+da`pwL~Kw2(P4k4k~sl_%E$=3u>qlZTxzi9Vd{T&T+g=ae+jbl$!u=fi$iRV zIL0V60C8a9cgib#aB=B#xgp7P;!U;R#&RT$_;7u>^-yjhWtYH^H*OcaB>g6H>`EYk zeMDG*AHrI!#dI^Vlv4PC@+OuhYqAGK*^+C#tfg_HLEnXn`biJ_1 zg<|vm)jC87XE2BMMA8YZ~zE_WUhAEV>3mKs@EUDEQWtl zQ#uVj(^FTevEJ?Gao0)ukY6AJvIJB=HjQnF)ry8EcGFK=XCsU{s}5GG4Q?c&oomhw zturE67035wkJ=7@foG?~zNSig^xE<7sQOG=6sbEU*x%E9g@xpCaPg`93yd87*-O#<<6WQ za?!ZlZ0%WtsE7~4tKokyDstLGdJZZW6IlKYc+HUgf<@JUfJkgY&Eky{8hga@%c+Bd z&&+BZM^T)`O#hsC_|+zZDF^r>yi$K4P(V!NVjqciN5)zoEpzR zSebLtW=mp;XZ>}R!A-y=C5B62^$72Hlzyv@iS5&l#7TaVn0!?L+wH8#@h14x|8Fv7 zV6qMZbRG={YwwMG}~WRbc`VYfQ3H_x?S{c<-0FJ`dz9~ zpLEpSn#jt+3xP^X_@n5e;$TeQhqR3*Px-RaVty}$9@c-8@zGqt5u0K@yi3H>39I51 zhg+^lodD}VgTTW?T~SU(7+_gzMi4OJ)k|>RKxx?|ZKCh=`M@;A750(FN))0^@3%tX z$`+?Pu>&_#HRHNfGlx<@;5s%lOUF)pLmcpz+sSV_GmHpYQ+gb=-EDJXFOEfBa>e2P zHv*1kycC3zW?+Zgv+dwW>GVM*-XC5%d^~p>UULL20{Wyl4AnW{w21#O($&jHo z<2A6q3e$aP%UBhvsI_+F*=pgXibW5V#M1kC4Cs1FxpG7GL1#>le#^q{#$W-$2!zVz zx8fIx7ys5Jvr&G7Zh}qzwWm*Sb&ZIW`z5mR6uA=o9G-7b#wAxK5F67XV*s6LWS*i! z#xzqRP2YtoEQ9+X!BNt}vJ73!4DVz^q>dZDl?1t6I#|rt(9UdVhaNs@$GGG9IuT43 z|E?IA10TCFqerH9RKnQSbV3B_H0HyqDgGbQtIDfr{e4oXYE11OtjNuTn(I~{Ha{)5 z`ygm;XJpSiRbKEwe1pf4KU^hwMy3BQ*Npj$el|4_Nyye%P;@*gUDj;Pg)=1tn%|%z z=@uK{O-)wt|Ci`Hk~@llnbMWh_Ld0GHftN-H4GE%b4w`2n>yhp_!Rc7&w?J zJGZ0h4uITKiJQF&V#TwVDeMf<*wULhUZB%{bd#{VolmswJX9>ZnM)a6ka14KCEeZ+ zMUPV0o0(k~R_R|cXEA{UYFNkox?RrszCXC}E0u_W%j#kK_5q^?dhJRMjPB2YEddf9y$Y2ifQ6E{un$ z^_Z<;L4{Y>s=eA2^~<=1rkb;Djhcg`#axOhw<96ojo`P@V(=HpY>0T}p=C5%URcDGR|Ru4hGvX ze2Yt2abmZT{q$MCs``U|rX_Tk?>kN1L!VRAFNRW|?xS^Tn+ioRSK_}B8^NGqbOL{R z1||we3sRtQ_S{>2yeRwhCYb$@dN^%t{p6z7%4T)4IlJs;fJHQ;VA1G?V5Oh4h&h4u zLx7R_gXkM*UiwFkk%IN&cpKvW3t0Z$nf;yf8L5Aj#Y*wh>=Mk zb|dR z@09`v6kl*DLa^0ZT{#nZ#3U=De%%;6cDGMvb93dy0GhJwddvDM!3NJX+M%0H&&_e1 zA*aoTjLRpnUHOFOh;@a|RJxC3cd0b&wj-=(HQXBAC7?85y~XAeS6LgJ1322e_s#t; z2q&6k7_YhE)?XtmA{Bi}h8vLee|`SDUxFvFD;cgxnzDeiduk@~A``ypt-t_ulp*Qu zAD0lAyH&ze_dWH>Ft#JOb5q!jUv>DQAPJCoPRPks{3Qf=7W-YKjDo0l-E*U-1&=cD z3>6O!d0(twF~- z+)wT*r!HLu2b2-lpcIA5fxgR8YxS>bVJB3k4muQW{K)@W+?autU_CS!o{Vsj#tQ)+ zcRLs&$V1nXvpgKtu{g7cWB+^sH|y<#dVy-WeGmLY@3EvS|_9_7IB5&+M>1Y)SdZ9Z|G2{dHz*pX+5q-AR{& zAohbk^ZBox1~pIvtf}gk28jcC(>?n|k(7PDlR+vg+ToKWMf0HAcb7OMV#5#dY7aSf zCG!JY4TNTAdeT30bd^4&hqtPEsK|&yoiCkCp?te^OS2zQTnNjv*n$m#OLUX1oqqV^ zXVU5m&y`Ap20~)5Y#_d(;Yvl`bG07L4=$>@--9AZ6MZsU0<;2$Y42qWq?0Be8?v=3 zlx#L{Ir~R*h2qh+VgY^NiGVlrqL4!r)H5y3C8ZN^X>)~d4f<$hJ9`lVga?J|?>UdA zD@*^W4-hEU>K8zkvN>2Doz(IP^wLvdrl?Dp+1yZ!efcV)p`{FNK`~xIL#ALZq3OhU zBDx467KvoI`ui&@r&9ODYL=m zGflIW+~Fb8#1Fom7VtQqP+KVmW|{Vj>PfkkZ(VNK+g-U{Zc{o&EZqK_Zf#)NdLbhN z!tssiDeLxy`wkpBsG#e?^CRx}dik5UAKvEGWutTOkKwNv_YZ%X7X$_3 zY(Y+Ma%r|5-mnhPLka%SL<0CdPv0@I_jm?42WGXK|CA&x#bXF|oc5PpFJa$EPi85K}o@1w)%Lz70UqUj>SA_!Q- zwt7BQ(fQ&&^Iw}S1MVQ?vRX|fMH^Pq7~N|I)Z#%YUkT9)5DY75Q~ZH;U8|OcxAber z$($YE2EOIU{$mFBWCC{u%<&IQ_>Kz{FsisX& z)Ak^1YMgN^GdM6V{{S)}p{uWd<7@73DCerbqwl##&&m|;GXtzqw)K0Kb%2K~{G#Ov zlpCA-NuPIgBl>`}t_yuHX0EBC<)1K4S#Jq|8=(s8T@Jkb$B5R440%l~)idq`{bN$6 zCiLz)MD;_FdoXzw9D9>{=SR*7*Js%EGQtpL&^pl@_>uBfz05_*4Kp`DNvM?S52q9@ zav|Vv5CVaBUPCxQ@3fFv(&ncedPB5L24smhzhASM>(RTul&InS-hOkFG@s4qffs)? zUF}N~ewCv$P1jRx0tGt0IR$_8g#$f(gVaT?EipsIV0P_Mz$FCUxpA-OYJF}USgmn; z_vO2YncQ{Z<@Np_fBTfE!3(kccM4Ih;ru{;Z4W^qoU})sg*W!2?gm1 z;(gMZccL^Km)`3;NjUPQ-v%!x3a5QGymbTmjNi%f%X|f=?glnn?`kk@>8rtP^ zwDo$v_Ennhrv@@Eud^0obpy>G*RVs6aG+KG)zOm-JT~3v|8qgj&&_KDrtmPj-L=*m zkVmSZdEzw$aPj#zmvI(k9c7=rdAj$GCdV8QbUOsxbaxZoGqVhKc`y;Su7EFZp8l5S zZCq6>yT|S%V@C6mk7{HJ~jJ=n;Tybm806Zt$^uKm%5`5^HR)03qMEwTV2^{;#l}{cnFB)|_DSY?)FK zU%l{@x!Ym6p%GGQv#Ft1B!gqfRN-{%@VdXs6$#Qnegv>xNV%ISgt@jpG()0^2*Ey` zNB_oA>@mLt{M`gA)i5`Gv>W}ahA^S;X7h)ZLs{=fBEmQRbyY?~9n3uCffmUx-q5sr zt`Je!B5z`Xe=>W+uA{$~r<+oRSnfU)g_9kuPyuB0Z3gC^7G@(b(BNDYTlv+e5tzlu zJEcwL4kx4SJ3`b|1V$9DB?1{J3pe@=HyW%5*U1 zZSkvwQb3edK2E=Z>7h&?ywE7V5rY>qXk}J6esP&%6?+%6={Uc6 z`B%@;BBuBBOtC_B0%GybTp)ZMj!M{Ts1!#i|JDo8J%wrs=FzOpU{ba z)x+H#^~);MbmM~RF=A6!bJl$WE194OU#l908{I}Ayp-!dNV8&QV*WQrOT^&-!r2B@ zFCNYq`Dl8-vn#v8ZAfRSci!9}F4n@UvWIZB-TGXWi+(r5|Hm{dDlpYIJUJ5&U&N)R zc7u$Jg@hBScIm5Yq;LD$dee22A^ILvH!0TKUN}}1n~YGh`tX5x?N+yarOjsf0pEyk zXx>LW-Rs89Z+rYu*zlA)I-5}C_?z{@FT9n*Y^i*wmdc2s4witlV4|wZoJeoRw93>0hdfIKV;iuf3 zHf~5M_OzE$$z4-eqvm(C`iV2?3;`GZxxBXTB8I!h8{ylfhH^1DU}cn%;aSKEq=!P5NJXbA-=9gwOA{g>SZF zBhDoIVv*kWJ^K&cE_B4VU=7mlfx~>W)=hEV&XY1i>dnht#tYBk>UES;95LD@Iap#} zw1pT7k9&6ikJz%;rXJq~WP0B8yLhLq1>N{Q@I*aLm#<@yc8JDWs4m|h7rJ!Q1vz?_ zVsa0*HEaQY>pO;fp4`9;#n69 zOvFH-+xE364oP_GM8Ans>+{)8oI)?%x#E)Odvz4+jVG?nv4cZQPh{CU$qyQ!U4WS9 zO^@7tl4zI^^qAs;ib2c}ddEVy?<$hcb+um_s_)B!z5#mwLbi$V8|7nSx!UJpmVMs- zhxY4P@{%xGfyek{&ty{C?>+4sCH-D!6nxYC&fQYumG5{`sjKh=M6tI|J+DA?oRPc<`9iE{W9V@A{Wj)!n z{r67__p3&hao%%=!3?*>@?wTxrv2Pxd;v}2MxeJV==H{xlrIsv@?v#14{K)ZF|@aH zOGZG=7ac-!pZ(-eo>@BLaUAabnt#rZ9Ub_o8`tVX7P4k%O64peRAJ;2<-Abw=EzWWf52;5YFD=qArKnxzBovpFU3{v4uuTGl#eVo>hY)#erZf9t zhVzm&;r1VFIV~(0lrWZj>UQ&|dcV9hbg7t5m9Y3AoSb zr=sZbOKJvV%K1)CiwDEHPBGJ`FqiQDU*EU&MuOdzWr1smg$rYz3v5g7r-6Ucigb6! zIHYRRQF(ks=)j=5b5*EP&eENSV6&iUh9Dwfe*l7TBT^WxH%H%Kd`+6*EoA^{25zBo z+RE`c-`U|w6L9$?P2}-)Rc<$!V9TDu3={7nNx4{Kki(SzT*Cg+udmZrtkf^k6-aHK z3}jF{?&+&T{>SDhEP|18kh31k+3Abt456y*7Ql!&hHp^uIzfvYEqkUTeRM&hr`P1O zb7I1kcf=`Eg~Gn(21o7#rMHVlyQK#}a1Qx>wtJ&FHvD)9H~NVGe%;3~UL3d(_Jzzj z^=B=rsq6x_;HMi06;a!>XeDR)IeOX?-^Hy%LgVJUpXeT7vYs z;C(t_2vq=X-ZHku4PYO>-E;e63>Lq?NG3S4JcL~xJ19!~4GIN5o--r_LAA2xO{aYh zr~H>(!mFAVT=2S$haob!(SUpQOZn)~eOogU*t1{)xo=j`FCQRyAara-(jZRCHYvBA z%}t&VcoSN1e{+3kQZ%ve<`DOQiDZsNM%K;`|0hg5pt^JlUP9?aym&>%GG@SaO4I4g z&V2tc1h9M(?fJeH5L%KoKnHasE(r|?DD*mYy^(a_hAYxP{zBWzq^yLMr}^UsY?*M< zlA%JbY`7$W`kJ3?Y_BMXL?02maK_#1w#3z3T@w~Z<7gXg4gF&q!&96{97!Z_7ng~r z#Ffo1EmAy)q#l_2;8`ORIO!Dg7s>zpoI#jyd44KQm)(@f8vH;2${ar zURZcDdq)r5kI8{jwPT9E<8`nSd|-`U_sdZzSl0qysI4)ul?$Wop^v{Y-`0kk@~5Cb zoSn0zgpJu)vj(NFy9i3wn5HSidOz&^=pc3#On?LI(zfw2v5Ug{zLB=4VB)mczqY2l z7AvZQN3Ugr=q@GfMWYQcYj26R_y2Hq@Ro4whE3e?J-grvMN35FV71bwy<;(!LfakK zST($;&LjO_1nkX&Tq1YHh5fv6Px*lAqoY)|alg7?vH8=G#>X0=y71ZvZ?gT2D!PcX z^g(?@(PCz6!&o_<320p|mDXE5tK#3{AFGPe^>rFiq+y^j>7f?u@rH!hK_xRLlFPX| zp48urSYJ@xp66-(`U3We-6nMRdE$f1WwLDgCz;bC_}uP9 z$fi%otd>X4*%1!p)wzF0WOO^+Ocx0Y>tu8bDF7p@0Vt+r>q;xVd0N|jlD2mpy3Pjn z4rca8yA}P_GEoWh>28XHZ{Hm-doB+OC9Y9QzP;a9$IcG_Uv!OebQX6(?7{;Qib|BA z++^H=A8C$6oc&jSqj3bw^o5c;UTlUjyX_}R=|-&^26WCxbf%96Lg>HO-(0?BWlSIh zl7UvM>;0DS+SCLC7jk2Ss5LDeOF!Y1?{cm%UIAOeotL1pgAWyS7(3^r2oAUHZ2SoX zF05NJGAALpq{m&i@%vLE8`5+Sdd`~+01dszQ6e-HjW5FN50?yfw(`AQ=J<0nKTkjN zk`Br9kC*5J5|{X{lAb?9}oaCw;X@l;Duy*I?^yO|!I%+G~K%tn;$=Tdv`-6k=1$F3!l zT>p4iWBRB80fbuIJ!Z7E`*mKDewi*!QN;~Be11I8r~ksjYyhfjpk#Dj41|+Xm=YT& zh8aR)2*3~LC4Rk2oGZy-l;_}jq)rvm>{vfjQ4H1raK8CRn zG?2|%>Oa@;UlAD`i(CEY`EW_;gX^5%upyXW@6qAV-`F;~OE>^N{D(G14;=CE-k>+NBx~7(qNeS^W^bx%4LlWePF`CnD6dpUL zdbyVp>e)l{BEwpTF6>2gikqoprF?C!>qkK7t4c}Bw*kvy-Z5#f2 zXa2a&RpOQ7f9R&|%yh|3_poeMbtarPl6|}__kSj4C+nA;0BnrCPqPlNQ!7>b9(K=T zS#-VSp?f!NEvKZO=1*DRvt)|%%A3>PeY=n|$$I6^sXJ56KmVwC@9i$T*56LG^4jLF zlJ>jx&OW=hWSzh6>LBAw*NT_!J9FXB);oJI%`UqC?&L(CexMnhS7dxP?L2M1&+Fve zC3pND-#$C<=*_;`VE4&YdMB@`9R8nk@7o^#PYP$Bb=vH@(=QqLYfdv~N!Iz3qM0Vg zf=-{CvdhqPcF-m>(`HS{SecN0YNd83+f_DuR?dCm+Qv5j$-CCLDONA8{Fr(5=e$_W zwX!Qyle5g0TrrJ2z`jtMPpC9w@U1oVl{{E-d ko%yHCfhU5{80hHQGjLSB4s(@A>;hTk>FVdQ&MBb@0MQy3l>h($ literal 0 HcmV?d00001 diff --git a/docs/api.md b/docs/api.md new file mode 100644 index 00000000..f363d279 --- /dev/null +++ b/docs/api.md @@ -0,0 +1,26 @@ +# API Reference + + +## Storage + +```{eval-rst} +.. autoclass:: pangeo_forge.storage.Target + :members: +``` + +```{eval-rst} +.. autoclass:: pangeo_forge.storage.InputCache + :members: +``` + +## Recipes + +```{eval-rst} +.. autoclass:: pangeo_forge.recipe.BaseRecipe + :members: +``` + +```{eval-rst} +.. autoclass:: pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe + :show-inheritance: +``` diff --git a/docs/conf.py b/docs/conf.py index 2dfe4699..a0e1904c 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -1,8 +1,9 @@ -import sphinx_pangeo_theme # noqa +# import sphinx_pangeo_theme # noqa +import sphinx_book_theme # noqa # -- Project information ----------------------------------------------------- -project = "pangeo-forge" +project = "Pangeo Forge" copyright = "2020, Pangeo Community" author = "Pangeo Community" @@ -22,8 +23,10 @@ # -- Options for HTML output ------------------------------------------------- -html_theme = "pangeo" -html_static_path = ["_static"] -html_sidebars = {"index": [], "**": ["localtoc.html"]} +html_theme = "sphinx_book_theme" +html_logo = "_static/pangeo-forge-logo-blue.png" +# html_theme = "pangeo" +# html_static_path = ["_static"] +# html_sidebars = {"index": [], "**": ["localtoc.html"]} myst_heading_anchors = 2 diff --git a/docs/index.md b/docs/index.md index 2718b2bd..f35e3bfb 100644 --- a/docs/index.md +++ b/docs/index.md @@ -38,5 +38,6 @@ recipes execution bakeries contribute +api ``` diff --git a/docs/recipes.md b/docs/recipes.md index f053c335..a45789fc 100644 --- a/docs/recipes.md +++ b/docs/recipes.md @@ -1,15 +1,57 @@ # Recipes +A recipe defines how to transform data in one format / location into another format / location. +The primary way people contribute to Pangeo Forge is by writing / maintaining recipes. + +```{warning} +The Recipe API is still in flux and may change. Make sure the version of the documentation +you are reading matches your installed version of pangeo_forge. +``` + +## Storage + +Recipes need a place to store data. +The location where the final dataset produced by the recipe is stored is called the +``Target``. Pangeo forge has a special class for this: {class}`pangeo_forge.storage.Target` + +Creating a Target requires two arguments: +- The ``fs`` argument is an [fsspec](https://filesystem-spec.readthedocs.io/en/latest/) + filesystem. Fsspec supports many different types of storage via its + [built in](https://filesystem-spec.readthedocs.io/en/latest/api.html#built-in-implementations) + and [third party](https://filesystem-spec.readthedocs.io/en/latest/api.html#other-known-implementations) + implementations. +- The `path` argument specifies the specific path where the data should be stored. + +For example, creating a storage target for AWS S3 might look like this: +```{code-block} python +import s3fs +fs = s3fs.S3FileSystem(key="MY_AWS_KEY", secret="MY_AWS_SECRET") +target_path = "pangeo-forge-bucket/my-dataset-v1.zarr" +target = Target(fs=fs, path=target_path) +``` + +Temporary data can be stored in an {class}`pangeo_forge.storage.InputCache` object. +``InputCache`` is similar to ``Target``, but instead of specifying a ``path``, +you specify ``prefix``. + + ## The Base Recipe Class -```{eval-rst} -.. autoclass:: pangeo_forge.recipe.BaseRecipe - :members: +A recipe is initialized from a recipe class. +```{code-block} python +recipe = Recipe(option1='foo', option2=) ``` +All recipes follow the same basic steps. + + + + + ## Specific Recipe Classes ```{eval-rst} .. autoclass:: pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe :show-inheritance: + :noindex: ``` diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index e42fbb7d..9d07c79f 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -119,6 +119,7 @@ def to_pipelines(self) -> ParallelPipelines: class NetCDFtoZarrSequentialRecipe(BaseRecipe): """There are many inputs (a.k.a. files, granules), arranged in a sequence along the dimension `sequence_dim`. Each file may contain multiple variables. + This class uses Xarray to read and write data. :param input_urls: The inputs used to generate the dataset. :param sequence_dim: The dimension name along which the inputs will be concatenated. @@ -149,8 +150,6 @@ def __post_init__(self): @property def prepare(self) -> Callable: - """Prepare target for storing dataset.""" - def _prepare(): try: @@ -175,13 +174,6 @@ def _prepare(): @property def cache_input(self) -> Callable: - """Cache the input. - - Properties - ---------- - url : URL pointing to the input file. Must be openable by fsspec. - """ - def cache_func(fname: str) -> None: logger.info(f"Caching input '{fname}'") with input_opener(fname, mode="rb") as source: @@ -192,14 +184,6 @@ def cache_func(fname: str) -> None: @property def store_chunk(self) -> Callable: - """Store a chunk in the target. - - Parameters - ---------- - chunk_key : str - The identifier for the chunk - """ - def _store_chunk(chunk_key): ds_chunk = self.open_chunk(chunk_key) @@ -218,8 +202,6 @@ def drop_vars(ds): @property def finalize(self) -> Callable: - """Finalize writing of dataset.""" - def _finalize(): if self.consolidate_zarr: logger.info("Consolidating Zarr metadata") diff --git a/pangeo_forge/storage.py b/pangeo_forge/storage.py index 742bb9f2..acc5ad33 100644 --- a/pangeo_forge/storage.py +++ b/pangeo_forge/storage.py @@ -1,6 +1,7 @@ import os from contextlib import contextmanager from dataclasses import dataclass +from typing import BinaryIO, NoReturn import fsspec @@ -8,20 +9,16 @@ @dataclass class Target: """Representation of a storage target for Pangeo Forge. - Attributes - ---------- - fs : FileSystemSpec.AbtractFileSystem - The filesystem we are writing to. Should be instantiated outside this - class. - path : str - The path where the target data will be saved. + + :param fs: The filesystem object we are writing to. + :param path: The path where the target data will be saved. """ fs: fsspec.AbstractFileSystem path: str - def get_mapper(self): - # don't want to use this because we want to use a fancier Zarr FSStore + def get_mapper(self) -> fsspec.mapping.FSMap: + """Get a mutable mapping object suitable for storing Zarr data.""" return self.fs.get_mapper(self.path) @@ -34,13 +31,8 @@ class InputCache: """Representation of an intermediate storage location where remote files Can be cached locally. - Attributes - ---------- - fs : FileSystemSpec.AbtractFileSystem - The filesystem we are writing to. Should be instantiated outside this - class. - prefix : str - A path prepended to all paths. + :param fs: The filesystem we are writing to. + :param prefix: A path prepended to all paths. """ fs: fsspec.AbstractFileSystem @@ -49,14 +41,17 @@ class InputCache: def _full_path(self, path): return os.path.join(self.prefix, _hash_path(path)) - def exists(self, path): + def exists(self, path) -> bool: + """Check that the file is in the cache.""" return self.fs.exists(self._full_path(path)) - def rm(self, path): + def rm(self, path) -> NoReturn: + """Remove file from the cache.""" self.fs.rm(self._full_path(path)) @contextmanager - def open(self, path, **kwargs): + def open(self, path, **kwargs) -> BinaryIO: + """Open file with a context manager.""" with self.fs.open(self._full_path(path), **kwargs) as f: yield f diff --git a/setup.cfg b/setup.cfg index d2063367..6a8506d2 100644 --- a/setup.cfg +++ b/setup.cfg @@ -3,7 +3,7 @@ max-line-length = 100 [isort] known_first_party=pangeo_forge -known_third_party=click,fsspec,numpy,pandas,pkg_resources,prefect,pytest,rechunker,setuptools,sphinx_pangeo_theme,xarray,zarr +known_third_party=click,fsspec,numpy,pandas,pkg_resources,prefect,pytest,rechunker,setuptools,sphinx_book_theme,xarray,zarr multi_line_output=3 include_trailing_comma=True force_grid_wrap=0 From fa20daf64b5ccbccd32371d0dd6755a58a1f7b55 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 00:42:18 -0500 Subject: [PATCH 26/34] add tutorial to docs --- docs/conf.py | 4 +- docs/index.md | 1 + docs/recipe_tutorial.ipynb | 215 +++++++++++++++++++++++++++++++++++++ pangeo_forge/recipe.py | 11 +- 4 files changed, 226 insertions(+), 5 deletions(-) create mode 100644 docs/recipe_tutorial.ipynb diff --git a/docs/conf.py b/docs/conf.py index a0e1904c..35c8d26d 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -11,14 +11,14 @@ # -- General configuration --------------------------------------------------- extensions = [ - "myst_parser", + "myst_nb", "sphinx.ext.autodoc", # "numpydoc", "sphinx_autodoc_typehints", ] templates_path = ["_templates"] -exclude_patterns = [] +exclude_patterns = ["_build", "**.ipynb_checkpoints"] master_doc = "index" # -- Options for HTML output ------------------------------------------------- diff --git a/docs/index.md b/docs/index.md index f35e3bfb..319199e4 100644 --- a/docs/index.md +++ b/docs/index.md @@ -35,6 +35,7 @@ For more information, see {doc}`bakeries`. :caption: Contents recipes +recipe_tutorial execution bakeries contribute diff --git a/docs/recipe_tutorial.ipynb b/docs/recipe_tutorial.ipynb new file mode 100644 index 00000000..16cc5820 --- /dev/null +++ b/docs/recipe_tutorial.ipynb @@ -0,0 +1,215 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Recipe Tutorial\n", + "\n", + "This tutorial describes how to create a recipe from scratch.\n" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 1: Get to know your source data\n", + "\n", + "If you are developing a new recipe, you are probably starting from an existing\n", + "datset. The first step is to just get to know the dataset. For this tutorial,\n", + "our example will be the _NOAA Optimum Interpolation Sea Surface Temperature\n", + "(OISST) v2.1_. The authoritative website describing the data is\n", + ".\n", + "This website contains links to the actual data files on the\n", + "[data access](https://www.ncdc.noaa.gov/oisst/data-access) page. We will use the\n", + "_AVHRR-Only_ version of the data and follow the corresponding link to the\n", + "[Gridded netCDF Data](https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/).\n", + "Browsing through the directories, we can see that there is one file per day. The\n", + "very first day of the dataset is stored at the following URL:\n", + "\n", + "```text\n", + "https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810901.nc\n", + "```\n", + "\n", + "From this example, we can work out the pattern of the file naming conventions.\n", + "But first, let's just download one of the files and open it up.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "! curl -O https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810901.nc " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import xarray as xr\n", + "\n", + "ds = xr.open_dataset(\"oisst-avhrr-v02r01.19810901.nc\")\n", + "ds" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "We can see there are four data variables, all with dimension\n", + "`(time, zlev, lat, lon)`. There is a _dimension coordinate_ for each dimension,\n", + "and no _non-dimension coordinates_. Each file in the sequence presumably has the\n", + "same `zlev`, `lat`, and `lon`, but we expect `time` to be different in each one.\n", + "\n", + "Let's also check the total size of the dataset in the file.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "print(f\"File size is {ds.nbytes/1e6} MB\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "The file size is important because it will help us define the _chunk size_\n", + "Pangeo Forge will use to build up the target dataset.\n" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 2: Pick a Recipe class\n", + "\n", + "For our first recipe, we will want to use a pre-defined Recipe class from Pangeo\n", + "Forge.\n", + "\n", + "By examining the {doc}`recipes` documentation page, we see that our scenario is\n", + "a good case for the {class}`pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe`\n", + "class. Let's examine its documentation string in our notebook.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "from pangeo_forge.recipe import NetCDFtoZarrSequentialRecipe\n", + "NetCDFtoZarrSequentialRecipe?" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 3: Define Recipe parameters\n", + "\n", + "Our chosen class has only two required parameters: `input_urls` and\n", + "`sequence_dim`.\n", + "\n", + "`input_urls` is a list of URLs pointing to the data. To populate this, we need\n", + "to explicitly create this list based on what we know about the file naming\n", + "conventions. Let's look again at the first URL\n", + "\n", + "```text\n", + "https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810901.nc\n", + "```\n", + "\n", + "From this we deduce the following format string.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "input_url_pattern = (\n", + " \"https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation\"\n", + " \"/v2.1/access/avhrr/{yyyymm}/oisst-avhrr-v02r01.{yyyymmdd}.nc\"\n", + ")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Now we need a sequence of datetimes. Pandas is the easiest way to get this. At\n", + "the time of writing, the latest available data is from 2021-01-05.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import pandas as pd\n", + "\n", + "dates = pd.date_range(\"1981-09-01\", \"2021-01-05\", freq=\"D\")\n", + "input_urls = [\n", + " input_url_pattern.format(\n", + " yyyymm=day.strftime(\"%Y%m\"), yyyymmdd=day.strftime(\"%Y%m%d\")\n", + " )\n", + " for day in dates\n", + "]\n", + "print(f\"Found {len(input_urls)} files!\")\n", + "input_urls[-1]" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "That's a lot of files!\n", + "\n", + "The other remaining parameter is `sequence_dim`. It's just `\"time\"`. We can now\n", + "instantiate our recipe.\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "recipe = NetCDFtoZarrSequentialRecipe(\n", + " input_urls=input_urls, sequence_dim=\"time\"\n", + ")\n", + "recipe" + ] + } + ], + "metadata": { + "kernelspec": { + "display_name": "Python 3", + "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.8.6" + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index 9d07c79f..b962ce1e 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -117,8 +117,13 @@ def to_pipelines(self) -> ParallelPipelines: @dataclass class NetCDFtoZarrSequentialRecipe(BaseRecipe): - """There are many inputs (a.k.a. files, granules), arranged in a sequence - along the dimension `sequence_dim`. Each file may contain multiple variables. + """This class represents a dataset composed of many individual NetCDF files. + The files are arraged in a sequence along a single dimension, called the + `sequence_dim`. Each file may contain multiple variables. + + The dataset is assembled by concatenating all of these files along `sequence_dim`. + The target is written in Zarr format. + This class uses Xarray to read and write data. :param input_urls: The inputs used to generate the dataset. @@ -133,7 +138,7 @@ class NetCDFtoZarrSequentialRecipe(BaseRecipe): the inputs to form a chunk. """ - input_urls: Iterable[str] + input_urls: Iterable[str] = field(repr=False) sequence_dim: str inputs_per_chunk: int = 1 nitems_per_input: int = 1 From 8d66eb948e7456333c94d8e70bb76231bfae814a Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 11:47:06 -0500 Subject: [PATCH 27/34] refactored storage targets --- pangeo_forge/recipe.py | 31 +++++++++-------- pangeo_forge/storage.py | 75 ++++++++++++++++++++++++++++++----------- tests/conftest.py | 6 ++-- tests/test_fixtures.py | 2 +- 4 files changed, 76 insertions(+), 38 deletions(-) diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index b962ce1e..a2ed5aa3 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -13,7 +13,7 @@ import zarr from rechunker.types import MultiStagePipeline, ParallelPipelines, Stage -from .storage import InputCache, Target +from .storage import AbstractTarget from .utils import chunked_iterable, fix_scalar_attr_encoding logger = logging.getLogger(__name__) @@ -132,6 +132,9 @@ class NetCDFtoZarrSequentialRecipe(BaseRecipe): :param nitems_per_input: The length of each input along the `sequence_dim` dimension. :param target: A location in which to put the dataset. Can also be assigned at run time. :param input_cache: A location in which to cache temporary data. + :param require_cache: Whether to allow opening inputs directly which have not + yet been cached. This could lead to very slow behavior if the inputs + live on a slow network. :param consolidate_zarr: Whether to consolidate the resulting Zarr dataset. :param xarray_open_kwargs: Extra options for opening the inputs with Xarray. :param xarray_concat_kwargs: Extra options to pass to Xarray when concatenating @@ -142,8 +145,9 @@ class NetCDFtoZarrSequentialRecipe(BaseRecipe): sequence_dim: str inputs_per_chunk: int = 1 nitems_per_input: int = 1 - target: Optional[Target] = None - input_cache: Optional[InputCache] = None + target: Optional[AbstractTarget] = None + input_cache: Optional[AbstractTarget] = None + require_cache: bool = True consolidate_zarr: bool = True xarray_open_kwargs: dict = field(default_factory=dict) xarray_concat_kwargs: dict = field(default_factory=dict) @@ -217,19 +221,18 @@ def _finalize(): @contextmanager def input_opener(self, fname: str): - if self.input_cache is None: - logger.info(f"No cache. Opening input `{fname}` directly.") - # This will bypass the cache. May be slow. - with input_opener(fname, mode="rb") as f: - yield f - elif self.input_cache.exists(fname): - logger.info(f"Input '{fname}' found in cache") + try: with self.input_cache.open(fname, mode="rb") as f: + logger.info(f"Opening '{fname}' from cache") yield f - else: - raise ValueError( - f"Input '{fname}' has not been cached yet. " "Call .cache_input() first." - ) + except IOError: # TODO figure out the excpetion to catch + if self.require_cache: + raise + else: + logger.info(f"No cache found. Opening input `{fname}` directly.") + # This will bypass the cache. May be slow. + with input_opener(fname, mode="rb") as f: + yield f def open_input(self, fname: str): with self.input_opener(fname) as f: diff --git a/pangeo_forge/storage.py b/pangeo_forge/storage.py index acc5ad33..87fce491 100644 --- a/pangeo_forge/storage.py +++ b/pangeo_forge/storage.py @@ -1,4 +1,7 @@ import os +import re +import unicodedata +from abc import ABC, abstractmethod from contextlib import contextmanager from dataclasses import dataclass from typing import BinaryIO, NoReturn @@ -6,20 +9,25 @@ import fsspec -@dataclass -class Target: - """Representation of a storage target for Pangeo Forge. +class AbstractTarget(ABC): + @abstractmethod + def get_mapper(self): + pass - :param fs: The filesystem object we are writing to. - :param path: The path where the target data will be saved. - """ + @abstractmethod + def exists(self, path) -> bool: + """Check that the file exists.""" + pass - fs: fsspec.AbstractFileSystem - path: str + @abstractmethod + def rm(self, path) -> NoReturn: + """Remove file.""" + pass - def get_mapper(self) -> fsspec.mapping.FSMap: - """Get a mutable mapping object suitable for storing Zarr data.""" - return self.fs.get_mapper(self.path) + @contextmanager + def open(self, path, **kwargs) -> BinaryIO: + """Open file with a context manager.""" + pass def _hash_path(path: str) -> str: @@ -27,19 +35,22 @@ def _hash_path(path: str) -> str: @dataclass -class InputCache: - """Representation of an intermediate storage location where remote files - Can be cached locally. +class FSSpecTarget(AbstractTarget): + """Representation of a storage target for Pangeo Forge. - :param fs: The filesystem we are writing to. - :param prefix: A path prepended to all paths. + :param fs: The filesystem object we are writing to. + :param root_path: The path under which the target data will be stored. """ fs: fsspec.AbstractFileSystem - prefix: str = "" + root_path: str = "" + + def get_mapper(self) -> fsspec.mapping.FSMap: + """Get a mutable mapping object suitable for storing Zarr data.""" + return self.fs.get_mapper(self.root_path) def _full_path(self, path): - return os.path.join(self.prefix, _hash_path(path)) + return os.path.join(self.root_path, path) def exists(self, path) -> bool: """Check that the file is in the cache.""" @@ -56,5 +67,29 @@ def open(self, path, **kwargs) -> BinaryIO: yield f def __post_init__(self): - if not self.fs.isdir(self.prefix): - self.fs.mkdir(self.prefix) + if not self.fs.isdir(self.root_path): + self.fs.mkdir(self.root_path) + + +class FlatFSSpecTarget(FSSpecTarget): + """A target that sanitizes all the path names so that everthing is stored + in a single directory. + + Designed to be used as a cache for inputs. + """ + + def _full_path(self, path): + slug = _slugify(path) + prefix = prefix = hex(hash(path))[2:10] + new_path = "-".join([prefix, slug]) + return os.path.join(self.root_path, new_path) + + +def _slugify(value): + # Adopted from + # https://github.com/django/django/blob/master/django/utils/text.py + # https://stackoverflow.com/questions/295135/turn-a-string-into-a-valid-filename + value = str(value) + value = unicodedata.normalize("NFKD", value).encode("ascii", "ignore").decode("ascii") + value = re.sub(r"[^\w\s-]+", "_", value.lower()) + return re.sub(r"[-\s]+", "-", value).strip("-_") diff --git a/tests/conftest.py b/tests/conftest.py index 6386481a..1525eb57 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -8,7 +8,7 @@ import xarray as xr from pangeo_forge import recipe -from pangeo_forge.storage import InputCache, Target +from pangeo_forge.storage import FlatFSSpecTarget, FSSpecTarget # where to run the http server _PORT = "8080" @@ -80,14 +80,14 @@ def tmp_target(tmpdir_factory): fs = fsspec.get_filesystem_class("file")() path = str(tmpdir_factory.mktemp("target")) - return Target(fs, path) + return FSSpecTarget(fs, path) @pytest.fixture() def tmp_cache(tmpdir_factory): path = str(tmpdir_factory.mktemp("cache")) fs = fsspec.get_filesystem_class("file")() - cache = InputCache(fs, prefix=path) + cache = FlatFSSpecTarget(fs, path) return cache diff --git a/tests/test_fixtures.py b/tests/test_fixtures.py index c867f17b..4e5e34fa 100644 --- a/tests/test_fixtures.py +++ b/tests/test_fixtures.py @@ -24,7 +24,7 @@ def test_fixture_http_files(daily_xarray_dataset, netcdf_http_server): def test_target(tmp_target): mapper = tmp_target.get_mapper() mapper["foo"] = b"bar" - with open(tmp_target.path + "/foo") as f: + with open(tmp_target.root_path + "/foo") as f: res = f.read() assert res == "bar" From 86f6f929d79a659acbee4993b9afe9641660108e Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 12:10:08 -0500 Subject: [PATCH 28/34] better target testing --- .pre-commit-config.yaml | 6 +----- pangeo_forge/recipe.py | 4 ++-- pangeo_forge/storage.py | 26 ++++++++++++++++++++++++++ tests/conftest.py | 7 ++++++- tests/test_fixtures.py | 20 ++++++++++++++++++++ 5 files changed, 55 insertions(+), 8 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 067b22f0..65a3907e 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -26,16 +26,12 @@ repos: rev: v2.2.0 hooks: - id: seed-isort-config + - repo: https://github.com/pre-commit/mirrors-isort rev: v5.2.0 hooks: - id: isort -- repo: https://github.com/deathbeds/prenotebook - rev: f5bdb72a400f1a56fe88109936c83aa12cc349fa - hooks: - - id: prenotebook - - repo: https://github.com/myint/rstcheck rev: master hooks: diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index a2ed5aa3..c22c12fa 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -165,7 +165,7 @@ def _prepare(): ds = self.open_target() logger.info("Found an existing dataset in target") logger.debug(f"{ds}") - except (IOError, zarr.errors.GroupNotFoundError): + except (FileNotFoundError, IOError, zarr.errors.GroupNotFoundError): first_chunk_key = next(self.iter_chunks()) for input_url in self.inputs_for_chunk(first_chunk_key): self.cache_input(input_url) @@ -225,7 +225,7 @@ def input_opener(self, fname: str): with self.input_cache.open(fname, mode="rb") as f: logger.info(f"Opening '{fname}' from cache") yield f - except IOError: # TODO figure out the excpetion to catch + except (IOError, FileNotFoundError): if self.require_cache: raise else: diff --git a/pangeo_forge/storage.py b/pangeo_forge/storage.py index 87fce491..44fc8e15 100644 --- a/pangeo_forge/storage.py +++ b/pangeo_forge/storage.py @@ -93,3 +93,29 @@ def _slugify(value): value = unicodedata.normalize("NFKD", value).encode("ascii", "ignore").decode("ascii") value = re.sub(r"[^\w\s-]+", "_", value.lower()) return re.sub(r"[-\s]+", "-", value).strip("-_") + + +class UninitializedTarget(AbstractTarget): + def get_mapper(self): + raise UninitializedTargetError + + def exists(self, path) -> bool: + raise UninitializedTargetError + + def rm(self, path) -> NoReturn: + raise UninitializedTargetError + + def open(self, path, **kwargs) -> BinaryIO: + raise UninitializedTargetError + + +class TargetError(Exception): + """Base class for exceptions in this module.""" + + pass + + +class UninitializedTargetError(TargetError): + """Operation on an uninitialized Target.""" + + pass diff --git a/tests/conftest.py b/tests/conftest.py index 1525eb57..37b4eaa4 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -8,7 +8,7 @@ import xarray as xr from pangeo_forge import recipe -from pangeo_forge.storage import FlatFSSpecTarget, FSSpecTarget +from pangeo_forge.storage import FlatFSSpecTarget, FSSpecTarget, UninitializedTarget # where to run the http server _PORT = "8080" @@ -91,6 +91,11 @@ def tmp_cache(tmpdir_factory): return cache +@pytest.fixture() +def uninitialized_target(): + return UninitializedTarget() + + @pytest.fixture def netCDFtoZarr_sequential_recipe(daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache): r = recipe.NetCDFtoZarrSequentialRecipe( diff --git a/tests/test_fixtures.py b/tests/test_fixtures.py index 4e5e34fa..cfc64fd1 100644 --- a/tests/test_fixtures.py +++ b/tests/test_fixtures.py @@ -1,6 +1,8 @@ import fsspec +import pytest import xarray as xr +from pangeo_forge.storage import UninitializedTargetError from pangeo_forge.utils import fix_scalar_attr_encoding @@ -27,6 +29,24 @@ def test_target(tmp_target): with open(tmp_target.root_path + "/foo") as f: res = f.read() assert res == "bar" + with pytest.raises(FileNotFoundError): + tmp_target.rm("baz") + with pytest.raises(FileNotFoundError): + with tmp_target.open("baz"): + pass + + +def test_uninitialized_target(uninitialized_target): + target = uninitialized_target + with pytest.raises(UninitializedTargetError): + target.get_mapper() + with pytest.raises(UninitializedTargetError): + target.exists("foo") + with pytest.raises(UninitializedTargetError): + target.rm("foo") + with pytest.raises(UninitializedTargetError): + with target.open("foo"): + pass def test_cache(tmp_cache): From 049e692cde62ddfb4cde1dcf5261f615806f76f3 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 14:36:15 -0500 Subject: [PATCH 29/34] change cannonical recipe execution order --- pangeo_forge/recipe.py | 43 ++++++++++++++++++++++++----------------- pangeo_forge/storage.py | 6 ++++++ tests/conftest.py | 4 ++-- tests/test_recipe.py | 21 ++++++++++++++++++-- 4 files changed, 52 insertions(+), 22 deletions(-) diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index c22c12fa..ebe82dda 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -13,7 +13,7 @@ import zarr from rechunker.types import MultiStagePipeline, ParallelPipelines, Stage -from .storage import AbstractTarget +from .storage import AbstractTarget, UninitializedTarget from .utils import chunked_iterable, fix_scalar_attr_encoding logger = logging.getLogger(__name__) @@ -23,12 +23,12 @@ # t = PangeoForgeTarget() # r = MyRecipe(target=t, **opts) # 1 # # manual execution of recipe -# r.prepare() # 3 # for input_key in r.iter_inputs(): # r.cache_input(input_key) # 4 +# r.prepare_target() # 3 # for chunk_key in r.iter_chunks(): # r.store_chunk(chunk_key) # 5 -# r.finalize() # 6 +# r.finalize_target() # 6 # 1) Initialize the Recipe object @@ -56,7 +56,7 @@ class BaseRecipe(ABC): @property @abstractmethod - def prepare(self) -> Callable[[], NoReturn]: + def prepare_target(self) -> Callable[[], NoReturn]: """Prepare the recipe for execution by initializing the target. Attribute that returns a callable function. """ @@ -90,7 +90,7 @@ def store_chunk(self) -> Callable[[Hashable], NoReturn]: @property @abstractmethod - def finalize(self) -> Callable[[], NoReturn]: + def finalize_target(self) -> Callable[[], NoReturn]: """Final step to finish the recipe after data has been written. Attribute that returns a callable function. """ @@ -101,10 +101,10 @@ def to_pipelines(self) -> ParallelPipelines: """ pipeline = [] # type: MultiStagePipeline - pipeline.append(Stage(self.prepare)) pipeline.append(Stage(self.cache_input, list(self.iter_inputs()))) + pipeline.append(Stage(self.prepare_target)) pipeline.append(Stage(self.store_chunk, list(self.iter_chunks()))) - pipeline.append(Stage(self.finalize)) + pipeline.append(Stage(self.finalize_target)) pipelines = [] # type: ParallelPipelines pipelines.append(pipeline) return pipelines @@ -145,8 +145,8 @@ class NetCDFtoZarrSequentialRecipe(BaseRecipe): sequence_dim: str inputs_per_chunk: int = 1 nitems_per_input: int = 1 - target: Optional[AbstractTarget] = None - input_cache: Optional[AbstractTarget] = None + target: Optional[AbstractTarget] = field(default_factory=UninitializedTarget) + input_cache: Optional[AbstractTarget] = field(default_factory=UninitializedTarget) require_cache: bool = True consolidate_zarr: bool = True xarray_open_kwargs: dict = field(default_factory=dict) @@ -158,8 +158,8 @@ def __post_init__(self): } @property - def prepare(self) -> Callable: - def _prepare(): + def prepare_target(self) -> Callable: + def _prepare_target(): try: ds = self.open_target() @@ -167,8 +167,8 @@ def _prepare(): logger.debug(f"{ds}") except (FileNotFoundError, IOError, zarr.errors.GroupNotFoundError): first_chunk_key = next(self.iter_chunks()) - for input_url in self.inputs_for_chunk(first_chunk_key): - self.cache_input(input_url) + # for input_url in self.inputs_for_chunk(first_chunk_key): + # self.cache_input(input_url) ds = self.open_chunk(first_chunk_key).chunk() # make sure the concat dim has a valid fill_value to avoid @@ -179,7 +179,7 @@ def _prepare(): self.expand_target_dim(self.sequence_dim, self.sequence_len()) - return _prepare + return _prepare_target @property def cache_input(self) -> Callable: @@ -210,14 +210,14 @@ def drop_vars(ds): return _store_chunk @property - def finalize(self) -> Callable: - def _finalize(): + def finalize_target(self) -> Callable: + def _finalize_target(): if self.consolidate_zarr: logger.info("Consolidating Zarr metadata") target_mapper = self.target.get_mapper() zarr.consolidate_metadata(target_mapper) - return _finalize + return _finalize_target @contextmanager def input_opener(self, fname: str): @@ -227,7 +227,11 @@ def input_opener(self, fname: str): yield f except (IOError, FileNotFoundError): if self.require_cache: - raise + raise FileNotFoundError( + f"You are trying to open input {fname}, but the file is " + "not cached yet. First call `cache_input` or set " + "`require_cache=False`." + ) else: logger.info(f"No cache found. Opening input `{fname}` directly.") # This will bypass the cache. May be slow. @@ -249,7 +253,10 @@ def open_chunk(self, chunk_key): inputs = self.inputs_for_chunk(chunk_key) dsets = [self.open_input(i) for i in inputs] # CONCAT DELETES ENCODING!!! + # OR NO IT DOESN'T! Not in the latest version of xarray? ds = xr.concat(dsets, self.sequence_dim, **self.xarray_concat_kwargs) + for var in ds.variables: + ds[var].encoding = {} logger.debug(f"{ds}") # TODO: maybe do some chunking here? diff --git a/pangeo_forge/storage.py b/pangeo_forge/storage.py index 44fc8e15..43b11a2d 100644 --- a/pangeo_forge/storage.py +++ b/pangeo_forge/storage.py @@ -85,6 +85,12 @@ def _full_path(self, path): return os.path.join(self.root_path, new_path) +class CacheFSSpecTarget(FlatFSSpecTarget): + """Alias for FlatFSSpecTarget""" + + pass + + def _slugify(value): # Adopted from # https://github.com/django/django/blob/master/django/utils/text.py diff --git a/tests/conftest.py b/tests/conftest.py index 37b4eaa4..af505ade 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -8,7 +8,7 @@ import xarray as xr from pangeo_forge import recipe -from pangeo_forge.storage import FlatFSSpecTarget, FSSpecTarget, UninitializedTarget +from pangeo_forge.storage import CacheFSSpecTarget, FSSpecTarget, UninitializedTarget # where to run the http server _PORT = "8080" @@ -87,7 +87,7 @@ def tmp_target(tmpdir_factory): def tmp_cache(tmpdir_factory): path = str(tmpdir_factory.mktemp("cache")) fs = fsspec.get_filesystem_class("file")() - cache = FlatFSSpecTarget(fs, path) + cache = CacheFSSpecTarget(fs, path) return cache diff --git a/tests/test_recipe.py b/tests/test_recipe.py index 2a6a412d..05fc2ae5 100644 --- a/tests/test_recipe.py +++ b/tests/test_recipe.py @@ -2,6 +2,7 @@ import xarray as xr from pangeo_forge import recipe +from pangeo_forge.storage import UninitializedTargetError dummy_fnames = ["a.nc", "b.nc", "c.nc"] @@ -50,13 +51,29 @@ def test_NetCDFtoZarrSequentialRecipe( ) # this is the cannonical way to manually execute a recipe - r.prepare() for input_key in r.iter_inputs(): r.cache_input(input_key) + r.prepare_target() for chunk_key in r.iter_chunks(): r.store_chunk(chunk_key) - r.finalize() + r.finalize_target() ds_target = xr.open_zarr(tmp_target.get_mapper(), consolidated=True).load() ds_expected = daily_xarray_dataset.compute() assert ds_target.identical(ds_expected) + + +def test_NetCDFtoZarrSequentialRecipeNoTarget( + daily_xarray_dataset, netcdf_local_paths, tmp_target, tmp_cache +): + + r = recipe.NetCDFtoZarrSequentialRecipe( + input_urls=netcdf_local_paths, + sequence_dim="time", + inputs_per_chunk=1, + nitems_per_input=daily_xarray_dataset.attrs["items_per_file"], + ) + + # this is the cannonical way to manually execute a recipe + with pytest.raises(UninitializedTargetError): + r.cache_input(next(r.iter_inputs())) From 49519ef2a5fcb7b1dfe08b7ed61ed66a56917ff3 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 15:38:12 -0500 Subject: [PATCH 30/34] big update --- docs/api.md | 4 +- docs/conf.py | 12 +- docs/recipe_tutorial.ipynb | 3458 +++++++++++++++++++++++++++++++++++- docs/recipes.md | 14 +- pangeo_forge/recipe.py | 12 +- 5 files changed, 3467 insertions(+), 33 deletions(-) diff --git a/docs/api.md b/docs/api.md index f363d279..667283b2 100644 --- a/docs/api.md +++ b/docs/api.md @@ -4,12 +4,12 @@ ## Storage ```{eval-rst} -.. autoclass:: pangeo_forge.storage.Target +.. autoclass:: pangeo_forge.storage.FSSpecTarget :members: ``` ```{eval-rst} -.. autoclass:: pangeo_forge.storage.InputCache +.. autoclass:: pangeo_forge.storage.CacheFSSpecTarget :members: ``` diff --git a/docs/conf.py b/docs/conf.py index 35c8d26d..8ff69e2c 100644 --- a/docs/conf.py +++ b/docs/conf.py @@ -15,18 +15,22 @@ "sphinx.ext.autodoc", # "numpydoc", "sphinx_autodoc_typehints", + "sphinx_copybutton", ] templates_path = ["_templates"] exclude_patterns = ["_build", "**.ipynb_checkpoints"] master_doc = "index" +# we always have to manually run the notebooks because they are slow / expensive +jupyter_execute_notebooks = "off" + # -- Options for HTML output ------------------------------------------------- html_theme = "sphinx_book_theme" html_logo = "_static/pangeo-forge-logo-blue.png" -# html_theme = "pangeo" -# html_static_path = ["_static"] -# html_sidebars = {"index": [], "**": ["localtoc.html"]} - +html_static_path = ["_static"] myst_heading_anchors = 2 +html_css_files = [ + "custom.css", +] diff --git a/docs/recipe_tutorial.ipynb b/docs/recipe_tutorial.ipynb index 16cc5820..3e7b2cfe 100644 --- a/docs/recipe_tutorial.ipynb +++ b/docs/recipe_tutorial.ipynb @@ -46,9 +46,474 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 2, "metadata": {}, - "outputs": [], + "outputs": [ + { + "data": { + "text/html": [ + "

\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 720, lon: 1440, time: 1, zlev: 1)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n",
+       "  * lon      (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n",
+       "  * time     (time) datetime64[ns] 1981-09-01T12:00:00\n",
+       "  * zlev     (zlev) float32 0.0\n",
+       "Data variables:\n",
+       "    anom     (time, zlev, lat, lon) float32 ...\n",
+       "    err      (time, zlev, lat, lon) float32 ...\n",
+       "    ice      (time, zlev, lat, lon) float32 ...\n",
+       "    sst      (time, zlev, lat, lon) float32 ...\n",
+       "Attributes:\n",
+       "    title:                      NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...\n",
+       "    source:                     ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n",
+       "    id:                         oisst-avhrr-v02r01.19810901.nc\n",
+       "    naming_authority:           gov.noaa.ncei\n",
+       "    summary:                    NOAAs 1/4-degree Daily Optimum Interpolation ...\n",
+       "    cdm_data_type:              Grid\n",
+       "    history:                    Final file created using preliminary as first...\n",
+       "    date_modified:              2020-05-08T19:05:13Z\n",
+       "    date_created:               2020-05-08T19:05:13Z\n",
+       "    product_version:            Version v02r01\n",
+       "    processing_level:           NOAA Level 4\n",
+       "    institution:                NOAA/National Centers for Environmental Infor...\n",
+       "    creator_url:                https://www.ncei.noaa.gov/\n",
+       "    creator_email:              oisst-help@noaa.gov\n",
+       "    keywords:                   Earth Science > Oceans > Ocean Temperature > ...\n",
+       "    keywords_vocabulary:        Global Change Master Directory (GCMD) Earth S...\n",
+       "    platform:                   Ships, buoys, Argo floats, MetOp-A, MetOp-B\n",
+       "    platform_vocabulary:        Global Change Master Directory (GCMD) Platfor...\n",
+       "    instrument:                 Earth Remote Sensing Instruments > Passive Re...\n",
+       "    instrument_vocabulary:      Global Change Master Directory (GCMD) Instrum...\n",
+       "    standard_name_vocabulary:   CF Standard Name Table (v40, 25 January 2017)\n",
+       "    geospatial_lat_min:         -90.0\n",
+       "    geospatial_lat_max:         90.0\n",
+       "    geospatial_lon_min:         0.0\n",
+       "    geospatial_lon_max:         360.0\n",
+       "    geospatial_lat_units:       degrees_north\n",
+       "    geospatial_lat_resolution:  0.25\n",
+       "    geospatial_lon_units:       degrees_east\n",
+       "    geospatial_lon_resolution:  0.25\n",
+       "    time_coverage_start:        1981-09-01T00:00:00Z\n",
+       "    time_coverage_end:          1981-09-01T23:59:59Z\n",
+       "    metadata_link:              https://doi.org/10.25921/RE9P-PT57\n",
+       "    ncei_template_version:      NCEI_NetCDF_Grid_Template_v2.0\n",
+       "    comment:                    Data was converted from NetCDF-3 to NetCDF-4 ...\n",
+       "    sensor:                     Thermometer, AVHRR\n",
+       "    Conventions:                CF-1.6, ACDD-1.3\n",
+       "    references:                 Reynolds, et al.(2007) Daily High-Resolution-...
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 720, lon: 1440, time: 1, zlev: 1)\n", + "Coordinates:\n", + " * lat (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n", + " * lon (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n", + " * time (time) datetime64[ns] 1981-09-01T12:00:00\n", + " * zlev (zlev) float32 0.0\n", + "Data variables:\n", + " anom (time, zlev, lat, lon) float32 ...\n", + " err (time, zlev, lat, lon) float32 ...\n", + " ice (time, zlev, lat, lon) float32 ...\n", + " sst (time, zlev, lat, lon) float32 ...\n", + "Attributes:\n", + " title: NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...\n", + " source: ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n", + " id: oisst-avhrr-v02r01.19810901.nc\n", + " naming_authority: gov.noaa.ncei\n", + " summary: NOAAs 1/4-degree Daily Optimum Interpolation ...\n", + " cdm_data_type: Grid\n", + " history: Final file created using preliminary as first...\n", + " date_modified: 2020-05-08T19:05:13Z\n", + " date_created: 2020-05-08T19:05:13Z\n", + " product_version: Version v02r01\n", + " processing_level: NOAA Level 4\n", + " institution: NOAA/National Centers for Environmental Infor...\n", + " creator_url: https://www.ncei.noaa.gov/\n", + " creator_email: oisst-help@noaa.gov\n", + " keywords: Earth Science > Oceans > Ocean Temperature > ...\n", + " keywords_vocabulary: Global Change Master Directory (GCMD) Earth S...\n", + " platform: Ships, buoys, Argo floats, MetOp-A, MetOp-B\n", + " platform_vocabulary: Global Change Master Directory (GCMD) Platfor...\n", + " instrument: Earth Remote Sensing Instruments > Passive Re...\n", + " instrument_vocabulary: Global Change Master Directory (GCMD) Instrum...\n", + " standard_name_vocabulary: CF Standard Name Table (v40, 25 January 2017)\n", + " geospatial_lat_min: -90.0\n", + " geospatial_lat_max: 90.0\n", + " geospatial_lon_min: 0.0\n", + " geospatial_lon_max: 360.0\n", + " geospatial_lat_units: degrees_north\n", + " geospatial_lat_resolution: 0.25\n", + " geospatial_lon_units: degrees_east\n", + " geospatial_lon_resolution: 0.25\n", + " time_coverage_start: 1981-09-01T00:00:00Z\n", + " time_coverage_end: 1981-09-01T23:59:59Z\n", + " metadata_link: https://doi.org/10.25921/RE9P-PT57\n", + " ncei_template_version: NCEI_NetCDF_Grid_Template_v2.0\n", + " comment: Data was converted from NetCDF-3 to NetCDF-4 ...\n", + " sensor: Thermometer, AVHRR\n", + " Conventions: CF-1.6, ACDD-1.3\n", + " references: Reynolds, et al.(2007) Daily High-Resolution-..." + ] + }, + "execution_count": 2, + "metadata": {}, + "output_type": "execute_result" + } + ], "source": [ "import xarray as xr\n", "\n", @@ -70,9 +535,17 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 3, "metadata": {}, - "outputs": [], + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "File size is 16.597452 MB\n" + ] + } + ], "source": [ "print(f\"File size is {ds.nbytes/1e6} MB\")" ] @@ -101,9 +574,60 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 4, "metadata": {}, - "outputs": [], + "outputs": [ + { + "data": { + "text/plain": [ + "\u001b[0;31mInit signature:\u001b[0m\n", + "\u001b[0mNetCDFtoZarrSequentialRecipe\u001b[0m\u001b[0;34m(\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0minput_urls\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mIterable\u001b[0m\u001b[0;34m[\u001b[0m\u001b[0mstr\u001b[0m\u001b[0;34m]\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0msequence_dim\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mstr\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0minputs_per_chunk\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mint\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;36m1\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mnitems_per_input\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mint\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;36m1\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mtarget\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mUnion\u001b[0m\u001b[0;34m[\u001b[0m\u001b[0mpangeo_forge\u001b[0m\u001b[0;34m.\u001b[0m\u001b[0mstorage\u001b[0m\u001b[0;34m.\u001b[0m\u001b[0mAbstractTarget\u001b[0m\u001b[0;34m,\u001b[0m \u001b[0mNoneType\u001b[0m\u001b[0;34m]\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;34m<\u001b[0m\u001b[0mfactory\u001b[0m\u001b[0;34m>\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0minput_cache\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mUnion\u001b[0m\u001b[0;34m[\u001b[0m\u001b[0mpangeo_forge\u001b[0m\u001b[0;34m.\u001b[0m\u001b[0mstorage\u001b[0m\u001b[0;34m.\u001b[0m\u001b[0mAbstractTarget\u001b[0m\u001b[0;34m,\u001b[0m \u001b[0mNoneType\u001b[0m\u001b[0;34m]\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;34m<\u001b[0m\u001b[0mfactory\u001b[0m\u001b[0;34m>\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mrequire_cache\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mbool\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;32mTrue\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mconsolidate_zarr\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mbool\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;32mTrue\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mxarray_open_kwargs\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mdict\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;34m<\u001b[0m\u001b[0mfactory\u001b[0m\u001b[0;34m>\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mxarray_concat_kwargs\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mdict\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;34m<\u001b[0m\u001b[0mfactory\u001b[0m\u001b[0;34m>\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m \u001b[0mdelete_input_encoding\u001b[0m\u001b[0;34m:\u001b[0m \u001b[0mbool\u001b[0m \u001b[0;34m=\u001b[0m \u001b[0;32mTrue\u001b[0m\u001b[0;34m,\u001b[0m\u001b[0;34m\u001b[0m\n", + "\u001b[0;34m\u001b[0m\u001b[0;34m)\u001b[0m \u001b[0;34m->\u001b[0m \u001b[0;32mNone\u001b[0m\u001b[0;34m\u001b[0m\u001b[0;34m\u001b[0m\u001b[0m\n", + "\u001b[0;31mDocstring:\u001b[0m \n", + "This class represents a dataset composed of many individual NetCDF files.\n", + "The files are arraged in a sequence along a single dimension, called the\n", + "`sequence_dim`. Each file may contain multiple variables.\n", + "\n", + "The dataset is assembled by concatenating all of these files along `sequence_dim`.\n", + "The target is written in Zarr format.\n", + "\n", + "This class uses Xarray to read and write data.\n", + "\n", + ":param input_urls: The inputs used to generate the dataset.\n", + ":param sequence_dim: The dimension name along which the inputs will be concatenated.\n", + ":param inputs_per_chunk: The number of inputs to use in each chunk.\n", + ":param nitems_per_input: The length of each input along the `sequence_dim` dimension.\n", + ":param target: A location in which to put the dataset. Can also be assigned at run time.\n", + ":param input_cache: A location in which to cache temporary data.\n", + ":param require_cache: Whether to allow opening inputs directly which have not\n", + " yet been cached. This could lead to very slow behavior if the inputs\n", + " live on a slow network.\n", + ":param consolidate_zarr: Whether to consolidate the resulting Zarr dataset.\n", + ":param xarray_open_kwargs: Extra options for opening the inputs with Xarray.\n", + ":param xarray_concat_kwargs: Extra options to pass to Xarray when concatenating\n", + " the inputs to form a chunk.\n", + ":param delete_input_encoding: Whether to remove Xarray encoding from variables\n", + " in the input dataset\n", + "\u001b[0;31mFile:\u001b[0m ~/pangeo-forge/pangeo-forge/pangeo_forge/recipe.py\n", + "\u001b[0;31mType:\u001b[0m ABCMeta\n", + "\u001b[0;31mSubclasses:\u001b[0m \n" + ] + }, + "metadata": {}, + "output_type": "display_data" + } + ], "source": [ "from pangeo_forge.recipe import NetCDFtoZarrSequentialRecipe\n", "NetCDFtoZarrSequentialRecipe?" @@ -131,7 +655,7 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 5, "metadata": {}, "outputs": [], "source": [ @@ -151,9 +675,27 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 6, "metadata": {}, - "outputs": [], + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Found 14372 files!\n" + ] + }, + { + "data": { + "text/plain": [ + "'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202101/oisst-avhrr-v02r01.20210105.nc'" + ] + }, + "execution_count": 6, + "metadata": {}, + "output_type": "execute_result" + } + ], "source": [ "import pandas as pd\n", "\n", @@ -174,21 +716,2909 @@ "source": [ "That's a lot of files!\n", "\n", - "The other remaining parameter is `sequence_dim`. It's just `\"time\"`. We can now\n", - "instantiate our recipe.\n" + "The other remaining parameter is `sequence_dim`. It's just `\"time\"`. This is enough to initialize the recipe." ] }, { "cell_type": "code", - "execution_count": null, + "execution_count": 7, "metadata": {}, - "outputs": [], + "outputs": [ + { + "data": { + "text/plain": [ + "NetCDFtoZarrSequentialRecipe(sequence_dim='time', inputs_per_chunk=1, nitems_per_input=1, target=, input_cache=, require_cache=True, consolidate_zarr=True, xarray_open_kwargs={}, xarray_concat_kwargs={}, delete_input_encoding=True)" + ] + }, + "execution_count": 7, + "metadata": {}, + "output_type": "execute_result" + } + ], "source": [ "recipe = NetCDFtoZarrSequentialRecipe(\n", - " input_urls=input_urls, sequence_dim=\"time\"\n", + " input_urls=input_urls,\n", + " sequence_dim=\"time\"\n", ")\n", "recipe" ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "However, now let's think about the Zarr chunks that this recipe will produce.\n", + "Each target chunk corresponds to one input. So each variable chunk will only be a few MB.\n", + "That is too small. Let's increase `inputs_per_chunk` to 20.\n", + "This means that we will need to be able to hold 20 files like the one we examined above in memory at once.\n", + "That's `16MB * 20 = 320MB`. Not a problem!" + ] + }, + { + "cell_type": "code", + "execution_count": 8, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "NetCDFtoZarrSequentialRecipe(sequence_dim='time', inputs_per_chunk=20, nitems_per_input=1, target=, input_cache=, require_cache=True, consolidate_zarr=True, xarray_open_kwargs={}, xarray_concat_kwargs={}, delete_input_encoding=True)" + ] + }, + "execution_count": 8, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "recipe = NetCDFtoZarrSequentialRecipe(\n", + " input_urls=input_urls,\n", + " sequence_dim=\"time\",\n", + " inputs_per_chunk=20\n", + ")\n", + "recipe" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 4: Play with the recipe\n", + "\n", + "Now we will just explore our recipe a bit to check whether things make sense.\n", + "\n", + "We can see how many inputs the recipe has like this:" + ] + }, + { + "cell_type": "code", + "execution_count": 9, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "14372" + ] + }, + "execution_count": 9, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "all_inputs = list(recipe.iter_inputs())\n", + "len(all_inputs)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "And how many chunks:" + ] + }, + { + "cell_type": "code", + "execution_count": 10, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "719" + ] + }, + "execution_count": 10, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "all_chunks = list(recipe.iter_chunks())\n", + "len(all_chunks)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "And we can see their dependencies as follows:" + ] + }, + { + "cell_type": "code", + "execution_count": 11, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "first chunk ID: 0\n" + ] + }, + { + "data": { + "text/plain": [ + "('https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810901.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810902.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810903.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810904.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810905.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810906.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810907.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810908.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810909.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810910.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810911.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810912.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810913.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810914.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810915.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810916.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810917.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810918.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810919.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810920.nc')" + ] + }, + "execution_count": 11, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "print(f'first chunk ID: {all_chunks[0]}')\n", + "recipe.inputs_for_chunk(all_chunks[0])" + ] + }, + { + "cell_type": "code", + "execution_count": 12, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "last chunk ID: 718\n" + ] + }, + { + "data": { + "text/plain": [ + "('https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201225.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201226.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201227.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201228.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201229.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201230.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202012/oisst-avhrr-v02r01.20201231.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202101/oisst-avhrr-v02r01.20210101.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202101/oisst-avhrr-v02r01.20210102.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202101/oisst-avhrr-v02r01.20210103.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202101/oisst-avhrr-v02r01.20210104.nc',\n", + " 'https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/202101/oisst-avhrr-v02r01.20210105.nc')" + ] + }, + "execution_count": 12, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "print(f'last chunk ID: {all_chunks[-1]}')\n", + "recipe.inputs_for_chunk(all_chunks[-1])" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Those are all the files that will go into the first and last chunk. \n", + "\n", + "We can now try to load the first chunk. This will raise an exception because we have not initialized any targets." + ] + }, + { + "cell_type": "code", + "execution_count": 13, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "\n" + ] + } + ], + "source": [ + "try:\n", + " recipe.open_chunk(all_chunks[0])\n", + "except Exception as e:\n", + " print(type(e))" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 5: Create storage targets\n", + "\n", + "We in order to run our recipe, we need to define to places to store data:\n", + "- The Input Cache, where we will temporarily store the files we have downloaded\n", + "- The Target, where the final Zarr dataset will live\n" + ] + }, + { + "cell_type": "code", + "execution_count": 14, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "NetCDFtoZarrSequentialRecipe(sequence_dim='time', inputs_per_chunk=20, nitems_per_input=1, target=FSSpecTarget(fs=, root_path='/var/folders/n8/63q49ms55wxcj_gfbtykwp5r0000gn/T/tmppoaidedy'), input_cache=CacheFSSpecTarget(fs=, root_path='/var/folders/n8/63q49ms55wxcj_gfbtykwp5r0000gn/T/tmp61q8ifva'), require_cache=True, consolidate_zarr=True, xarray_open_kwargs={}, xarray_concat_kwargs={}, delete_input_encoding=True)" + ] + }, + "execution_count": 14, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "import tempfile\n", + "from fsspec.implementations.local import LocalFileSystem\n", + "from pangeo_forge.storage import FSSpecTarget, CacheFSSpecTarget\n", + "\n", + "fs_local = LocalFileSystem()\n", + "\n", + "cache_dir = tempfile.TemporaryDirectory()\n", + "cache_target = CacheFSSpecTarget(fs_local, cache_dir.name)\n", + "\n", + "target_dir = tempfile.TemporaryDirectory()\n", + "target = FSSpecTarget(fs_local, target_dir.name)\n", + "\n", + "recipe.input_cache = cache_target\n", + "recipe.target = target\n", + "recipe" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Now we try to load the chunk." + ] + }, + { + "cell_type": "code", + "execution_count": 15, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "You are trying to open input https://www.ncei.noaa.gov/data/sea-surface-temperature-optimum-interpolation/v2.1/access/avhrr/198109/oisst-avhrr-v02r01.19810901.nc, but the file is not cached yet. First call `cache_input` or set `require_cache=False`.\n" + ] + } + ], + "source": [ + "try:\n", + " recipe.open_chunk(all_chunks[0])\n", + "except Exception as e:\n", + " print(e)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "It still didn't work! That's because we have not cached the inputs yet." + ] + }, + { + "cell_type": "code", + "execution_count": 16, + "metadata": {}, + "outputs": [], + "source": [ + "for input_file in recipe.inputs_for_chunk(all_chunks[0]):\n", + " recipe.cache_input(input_file)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 6: Examine some chunks\n", + "\n", + "Now we can finally open the first chunk!" + ] + }, + { + "cell_type": "code", + "execution_count": 17, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 720, lon: 1440, time: 20, zlev: 1)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n",
+       "  * lon      (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n",
+       "  * time     (time) datetime64[ns] 1981-09-01T12:00:00 ... 1981-09-20T12:00:00\n",
+       "  * zlev     (zlev) float32 0.0\n",
+       "Data variables:\n",
+       "    anom     (time, zlev, lat, lon) float32 nan nan nan nan ... 0.41 0.41 0.41\n",
+       "    err      (time, zlev, lat, lon) float32 nan nan nan nan ... 0.3 0.3 0.3 0.3\n",
+       "    ice      (time, zlev, lat, lon) float32 nan nan nan nan ... 0.9 0.9 0.9 0.9\n",
+       "    sst      (time, zlev, lat, lon) float32 nan nan nan ... -1.39 -1.39 -1.39\n",
+       "Attributes:\n",
+       "    title:                      NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...\n",
+       "    source:                     ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n",
+       "    id:                         oisst-avhrr-v02r01.19810901.nc\n",
+       "    naming_authority:           gov.noaa.ncei\n",
+       "    summary:                    NOAAs 1/4-degree Daily Optimum Interpolation ...\n",
+       "    cdm_data_type:              Grid\n",
+       "    history:                    Final file created using preliminary as first...\n",
+       "    date_modified:              2020-05-08T19:05:13Z\n",
+       "    date_created:               2020-05-08T19:05:13Z\n",
+       "    product_version:            Version v02r01\n",
+       "    processing_level:           NOAA Level 4\n",
+       "    institution:                NOAA/National Centers for Environmental Infor...\n",
+       "    creator_url:                https://www.ncei.noaa.gov/\n",
+       "    creator_email:              oisst-help@noaa.gov\n",
+       "    keywords:                   Earth Science > Oceans > Ocean Temperature > ...\n",
+       "    keywords_vocabulary:        Global Change Master Directory (GCMD) Earth S...\n",
+       "    platform:                   Ships, buoys, Argo floats, MetOp-A, MetOp-B\n",
+       "    platform_vocabulary:        Global Change Master Directory (GCMD) Platfor...\n",
+       "    instrument:                 Earth Remote Sensing Instruments > Passive Re...\n",
+       "    instrument_vocabulary:      Global Change Master Directory (GCMD) Instrum...\n",
+       "    standard_name_vocabulary:   CF Standard Name Table (v40, 25 January 2017)\n",
+       "    geospatial_lat_min:         -90.0\n",
+       "    geospatial_lat_max:         90.0\n",
+       "    geospatial_lon_min:         0.0\n",
+       "    geospatial_lon_max:         360.0\n",
+       "    geospatial_lat_units:       degrees_north\n",
+       "    geospatial_lat_resolution:  0.25\n",
+       "    geospatial_lon_units:       degrees_east\n",
+       "    geospatial_lon_resolution:  0.25\n",
+       "    time_coverage_start:        1981-09-01T00:00:00Z\n",
+       "    time_coverage_end:          1981-09-01T23:59:59Z\n",
+       "    metadata_link:              https://doi.org/10.25921/RE9P-PT57\n",
+       "    ncei_template_version:      NCEI_NetCDF_Grid_Template_v2.0\n",
+       "    comment:                    Data was converted from NetCDF-3 to NetCDF-4 ...\n",
+       "    sensor:                     Thermometer, AVHRR\n",
+       "    Conventions:                CF-1.6, ACDD-1.3\n",
+       "    references:                 Reynolds, et al.(2007) Daily High-Resolution-...
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 720, lon: 1440, time: 20, zlev: 1)\n", + "Coordinates:\n", + " * lat (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n", + " * lon (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n", + " * time (time) datetime64[ns] 1981-09-01T12:00:00 ... 1981-09-20T12:00:00\n", + " * zlev (zlev) float32 0.0\n", + "Data variables:\n", + " anom (time, zlev, lat, lon) float32 nan nan nan nan ... 0.41 0.41 0.41\n", + " err (time, zlev, lat, lon) float32 nan nan nan nan ... 0.3 0.3 0.3 0.3\n", + " ice (time, zlev, lat, lon) float32 nan nan nan nan ... 0.9 0.9 0.9 0.9\n", + " sst (time, zlev, lat, lon) float32 nan nan nan ... -1.39 -1.39 -1.39\n", + "Attributes:\n", + " title: NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...\n", + " source: ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n", + " id: oisst-avhrr-v02r01.19810901.nc\n", + " naming_authority: gov.noaa.ncei\n", + " summary: NOAAs 1/4-degree Daily Optimum Interpolation ...\n", + " cdm_data_type: Grid\n", + " history: Final file created using preliminary as first...\n", + " date_modified: 2020-05-08T19:05:13Z\n", + " date_created: 2020-05-08T19:05:13Z\n", + " product_version: Version v02r01\n", + " processing_level: NOAA Level 4\n", + " institution: NOAA/National Centers for Environmental Infor...\n", + " creator_url: https://www.ncei.noaa.gov/\n", + " creator_email: oisst-help@noaa.gov\n", + " keywords: Earth Science > Oceans > Ocean Temperature > ...\n", + " keywords_vocabulary: Global Change Master Directory (GCMD) Earth S...\n", + " platform: Ships, buoys, Argo floats, MetOp-A, MetOp-B\n", + " platform_vocabulary: Global Change Master Directory (GCMD) Platfor...\n", + " instrument: Earth Remote Sensing Instruments > Passive Re...\n", + " instrument_vocabulary: Global Change Master Directory (GCMD) Instrum...\n", + " standard_name_vocabulary: CF Standard Name Table (v40, 25 January 2017)\n", + " geospatial_lat_min: -90.0\n", + " geospatial_lat_max: 90.0\n", + " geospatial_lon_min: 0.0\n", + " geospatial_lon_max: 360.0\n", + " geospatial_lat_units: degrees_north\n", + " geospatial_lat_resolution: 0.25\n", + " geospatial_lon_units: degrees_east\n", + " geospatial_lon_resolution: 0.25\n", + " time_coverage_start: 1981-09-01T00:00:00Z\n", + " time_coverage_end: 1981-09-01T23:59:59Z\n", + " metadata_link: https://doi.org/10.25921/RE9P-PT57\n", + " ncei_template_version: NCEI_NetCDF_Grid_Template_v2.0\n", + " comment: Data was converted from NetCDF-3 to NetCDF-4 ...\n", + " sensor: Thermometer, AVHRR\n", + " Conventions: CF-1.6, ACDD-1.3\n", + " references: Reynolds, et al.(2007) Daily High-Resolution-..." + ] + }, + "execution_count": 17, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "ds_chunk = recipe.open_chunk(all_chunks[0])\n", + "ds_chunk" + ] + }, + { + "cell_type": "code", + "execution_count": 18, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "Total chunk size: 331.784804 MB\n" + ] + } + ], + "source": [ + "print(f'Total chunk size: {ds_chunk.nbytes / 1e6} MB')" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "👀 Inspect the Xarray HTML repr carefully by clicking on the buttons to expand the different sections.\n", + "- ✅ Is the shape of the variable what we expect\n", + "- ✅ Is `time` in going the right order\n", + "- ✅ Do the variable attributes make sense?\n", + "\n", + "\n", + "Now let's visualize some data make sure things look" + ] + }, + { + "cell_type": "code", + "execution_count": 19, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "" + ] + }, + "execution_count": 19, + "metadata": {}, + "output_type": "execute_result" + }, + { + "data": { + "image/png": "iVBORw0KGgoAAAANSUhEUgAAAZYAAAEXCAYAAACOFGLrAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjMuMiwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy8vihELAAAACXBIWXMAAAsTAAALEwEAmpwYAAD/rklEQVR4nOy9eZxkWVUn/j33LRGRkZXVVdV7021DS6/Y0Gw9zS4Cg6iIyqCgjuDCDCMuoCguP0HUAdHBwVF77LEFZcANRkURZBdEZG22ptnX3quqq3KJjIj33r3n98e55777XkZmRlZG1kaezyc+EfHW++5776zfcw4xM3Zpl3Zpl3Zpl2ZF5kQPYJd2aZd2aZdOL9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLBER0UVEtEJEyYkeyy7t0k4QEV1MRExE6Ykeyy6dvvQNLViI6CtE9Dj9z8xfY+Z5ZrYnclwbERE9l4g+TERjInr1hPU/TkRf8ALyLUR0frSuQ0T/m4juIqJ7iOgfiOiCaP1vENEniagiohdPMZaLiehdRLRKRJ+J55KEfoWIvkZES0T0l0S0sMnxnkFEXyWiARH9HRHtj9Y9jYj+zZ/r3dHyR/prXfH7cfR/xSsLE/f1+19KRH9PRAf9nPwzEV22yTifR0R3EtEiEf0pEXWidfuJ6G/9WL5KRM/Y5Fjf5udu1c/lN7Xm8LeJ6LD/vJyIaKPjnYq00XxO2PYBRPQRP18fIaIHHMeh7tKU9A0tWE5Ruh3AbwL40/YKIno0gP8O4LsB7AfwZQB/EW3yMwCuA3A1gPMBHAXwv6L1XwDwCwDeNOVY/gLATQAOAPgVAK8norP8uv8M4IcBPNyfq9c6V3vsVwH4Y7/POQBWAfxRtMk9AP4ngJfF+zHze70yMA/gKr/4DF3GzF9bb1/dFsAbAVzmz/tBAH+/wTj/I4AXAvg2ABcDuA+AX482+UMAhT/WDwK43l/bpGOdCeD/Afj/IPfrwwD+Ktrk2QCeAuD+kHv2nQD+y3pjOxVpivmMt80h9+b/AtgH4M8A/L1fvksnEzHzN+QHwGsAOABDACsQhnoxAAaQ+m3eDWHi/+a3+QcIE30tgCUAHwJwcXTMywG8DcLIPgvgaTs4/t8E8OrWst8F8IfR//P99Vzi/18P4OXR+u8A8NkJx/6/AF68yfkvBTAGsCda9l4A/9X/fj2AF0TrHgZgBGBuneP9dwCvi/5fAmHQe1rb/TiAd69zjMb9m7B+3X2jbfb7YxxYZ/3rAPz36P+3AbjT/+77MV/aes5ets6xng3g36L/ff88Xu7//xuAZ0frfwzAv0/5fHzcP7P6YQCPmfCM7wVwI4A7ANzmn6sEQAeieNwvOuZZfnxnz/A5Xnc+J2z7BD9GipZ9DcATZzWe3c9sPt+wFgsz/zDkofwuFs325ets+gMQLfoCCLN7P4BXQRjQLQBeBABE1IcIldcBOBvA0wH80Qba6h8R0dF1Pp84xssi/4n/A8D9/PeNAB5OROcT0RxEo37zMZ7rKgBfYublaNnHUVsNk8bSAXDfDY73cf3DzF+EZ9LHOL5jpUdBGNthACCiRxDR0Wh9Y5z+9zlEdAAyVsvMn2utD8+Av7+PmHQsZh4A+GK0/aRzTXye2sTM9+fakns+RNH56IRN/wxABeCbAVwDYd4/zsxjiDX19GjbpwH4F2a+u30QnacNPo9o77PBNep8Ttr2E+wliqdPYMo52aXjR9+wgmUL9Cpm/iIzL0KY8BeZ+e3MXAH4G8jLCIib4ivM/Cpmrpj5owDeAOCpkw7KzP+Nmc9Y53P1MY71nwA8jYiuJqIegF+DaKdzfv3nIML0NojFdQWAlxzjueYBLLaWLQLY43+/GcCP+zjMXgC/6JfPYTJtdrwdJyK6F8SV9Xxdxsz/ysxnRJu1x6m/90xYp+vDNfj7+6/rHKu9/aRzzW8lzuIZ+m8CeDIzL7XWnQPg2wH8LDMPvMD4PYgyBYiSFAuWZ/hla0jnaYPPv07ab51rBCbf9xP+jOzSdLQrWDanu6Lfwwn/5/3vbwJwbaylQSyCc4/LKAEw8zsgFtQbAHwVwFcALAO41W9yPYAuxJ3Xh2ikU1ksRHRzFBB/JMS90g7GL/jzARID+guIO/FmAO/yy29tBdxv9ss3O96Oko8NvRXAHzHzX2ywaXuc+nt5wjpdv941bLb9pHOttDT2dYmILgTw1wB+pGVFKX0TgAzAHdEz+8cQixsA3gmgR0TXelDBAwD87TTn3gJtNJ+bbavbH5dnZJemp290wTLL0s5fh7gJYi1tnpmfM2ljEnTWyjqfmyftMw0x8x8y832Z+WyIgEkBfMqvvj8kLnOPd3X8LwAP9UHkzY57FdcB8fdChMV9iCjWFu/vl4OZHTO/iJkvZuZ7+eW3AbiNo4A7M6sb42a/v87PfSCus0kMcaZERPsgQuWNzPxbm2zeGKf/fZd3nX0OQEpE922tX+9+tq+5D3G33jxp/SbHapC3WP8OwP9k5vWUh69D4mRnRs/sgt4TZnYQwfR0iLXyjy3XZ3y+R27wPKsyMok2ms9J217dstiuxpRzskvHj77RBctdEBTKLOgfAVxKRD9MRJn/PISIrpi0MTP/14i5tj/r+oyJKCWiLiTAmhBRl3xOgv99Pw9TvQjADQBeycxH/O4fAvCfiWgvEWUA/huA25n5kN8/88c2EAbZpXVyerwG/DEAL/LbfQ/kJX+DP9Z+IrrEj+VKAK8A8BLPrCbRawF8l2dQfYiL7v8pIyOixI8tBWD8ObP15qk1Z+vuSwKB/mcA72PmF05xuD8H8GNEdKUXSL8K4NV+TgYQK/AlRNQnoodDEHqvWedYfwvgfkT0fX58vwaJIXwmOtfziegCEtj4z+m5/NjfTevDwv8UwGc2iB2Cme+ACNT/QUQLRGT8PXt0tNnrAHw/xPqe6Abzx3rvBs+zKiOTaN35nEDvBmAB/DQJdP65fvk71xvXLp0gOpaI/+nygbz0X4OgX34ek1FhPx5t30BiAXgcgC9E/y+DQHUPAjgMeeAfMOMxv9iPMf682K87AxLMHAC4E8BLASTRvopou9tf878CeGi0/tUTjv3MDcZysZ+jISQ4/Lho3aV+2SrELff8Ka7tGf5+DCCw0v3RumdOGNurJ4xnDSpso30B/Ij/P0ATRXWRX/9IiPspPt7zIUrJEgTI0YnW7YdYCgN/Lc9o7bsC4JGtZ+gzfg7fjSbKkAC8HIIyvMf/jhFRXwTw+HXmkv3cx9f0yPYcQVBh10PcpYsQ+PgPtI71BX/+fIfew43m880Afjn6fw2Aj/j5+iiAa3aSR+x+ju1D/mbt0i7t0ilEHmjwN8x83Ykeyy7tUpt2Bcsu7dIu7dIuzZS+0WMsu7RLu7RLuzRj2hUsu7RLu7RLuzRT2hUsu7RLu7RLuzRTOm1KZx8480y+6KKLTvQwdmmXdukUoI/ddNMhZj5r8y3XpwupxyOsh55v0iEU/8zMT9zO+U4lOm0Ey0UXXYR3v/d9J3oYu7RLu3QK0Bnzc1/d7jFGcPg+nDfVtn+Mr26ahHw60WkjWAiAmbKCktsFwu3SLp20NO17vB4dr/ebACTTjvUbjOecNoJlK7TRg7srdE4tmoYJzeqeHq/nRs/zjfgsbleozOoY0xAByKc92UnbOnBn6LQWLHrLt/J+GvrGfKFPRZr2nY63O9Z7u9m5ZiUM4vO0z3m6P5ft653m9p7IKRGL5bRr6DkTOq0Ey3q3eNLyjR7Ik0W4bCd5Na7Tp8fZQrX1bdOJOOc0dCz3disa8HYEzDTC63g+lxs9fzt5X/XIZpNzOOZN3+1JyuXMRk5bcIV9g9FpJViA6R5GYO3DNYv3db0X8Vhewu1WRJi0/yzHt9kx11uv52Lmxu9paL1xrjf69Y66XQYdn2+jcwDTn2cr1tdGx9yqMDjW52y9+7pdagsVPSxF52O/0LTu/Hrv9nrLtku7Fsv6dBoJFlrzMLaJWR5YN+FlIjSZxFaZz0YvaMxEj/VYs2ZQk863VeZwLEwp3mer+291Htv3dBbUPvtOnGOnqH2fZ1nOadIzNM3xJ93P+D2OBQp8YWzS9LtwHc39Jr3fO0FbCt5/g9FpJFg2pmN51k6US4yIZvrST3vOaSiek/Y+J2Pdue0w/lhIbzQ72xUu68UWJh3zWK2VY9nuWGhHlQWanM9N1Hy/YwGzmQdje0S7Fss6dMIFCxE9D8CPQ96jTwJ4FqR97V9BSnx/BcDTuO4pss6B1rdUYlJtRp/DeJdJL/Qk4bJdTX0n6FiRMOu91JOOJ5BuQkIyj2uuiKgxV9u95kmadcyIHMs422eZZFXExJh8X/WaJ83Iegwqdr+0x7GZUrKZ4GofcxZCZSOKj79/zxzuWV7F/j3NTtJHV1abYzwG6+eM+fW6U29Mw9EIxBxcYc1xrN1eXWU7xfsJQDaDg/tePO+BNLZLAbyemV9ERPuxVT54ktAJFSxEdAGAnwZwJTMPieivIf22rwTwDmZ+GRG9EMALUfdMP8ZzNV1h23kctutKmGSRbKb9b2Q9HYtQ2apLSc5D4SU1aLoUOWwznZW3HlOPxxfiFJidBRe7QjcSnhu5U6c/13TbbTVGdKzUFhyb0aRtzpifw3A0ArMIVctr1wPA4mAY7pkuW1kdHuPIhYaluMISI3Ob+Puk7rK2wNlpY4JmF7wfA3gsM6/4BnT/SkRvBvC9mDEfPF50wi0WyBh6RFRCLJXbAfwSgMf49X8GaYC04YSSNJeRP3GTwsh8rhlkvXqS9tNmHqqhN7aBMLxjoYSa57WOGwx5PSY6DWxWBcY0x5uWGDJ3jnlNwFTXTxrXei699YSKLjckY540v22BGO8DbB6kje9j2yJoCxWaMHZCfe8a7v8J28U0yUJejzYDBACT7+26Fpgfb0JAFxWYDEarA5C+J+zQmd+77niGo1E9Nm4y7ISA+bnemn329tcum7TdVmj/njmMVgco/SzO9boAgOXVIazjBqOfpBxMup/bpVm4wlhu4or/m/kPQxoRPsYvn4oPnix0QgULM99GRL8L6bQ3BPBWZn4rEZ3D0jYVzHwHEZ093QGdvCwuykZSU0UDf+wAVwG2ApIUMCk4ydb4b5lMvYwdkkhYsUnhQCDwGsaU+LeZ/BhoQidephTkqsaOluRZcry+NREzkYnadiRUYjpjfg6Lg2FYN4kprcewYzIE2BZj1m9eZ59Jlti0+RnWrbUsJgkq8i44dYtt5VVvb6vCMwx7gpJCzgIkIpYMNQRMbQ2153Njprbe2q24vybd95hGqwOAIc9e9GyvRyurwzUMWi2EXm97QuJYqTvXR9f/XhwMkScEApAaQuoVtsoxHPOOMzfC7IL3vgX4RwB8M4A/ZOYPHDMfPAnoRLvC9kGk8r0hrXL/hoh+aAv7PxvAswHgwgsvrJk4kbz4xQBUFYAtQeyQnndflHd+ETDSxp2NAVzVYC5VPo/ElbJeNVMkMJHgYaod+3FchgEUVhgnwSAliBBjBkwCNiJQqBrXJ/TLY/eMMpNY+3Let7xeDCNm4G1mtDgYBs1ydTgKFpiNXEwbCRkdV3y9k2gSxDbez/jxqxBiXis4YkrMlBaPvxnqlokF7GbvfZtxNqwQohqBFC0DJUH7Ja/Ch/+Ggvas7qLN9OS21TWJtgrJNlQLBkC0++5cH4AXMP557k4QEKPVAdi/IwCangAyx2ypz5LuWV4NSlxKEKXSOcCkSP1yxxGyLPCG2RV0F8Ey9VycSUQfjv7fwMw36B9mtgAeQERnAPhbIrrfzAZ6AuhEu8IeB+DLzHwQAIjo/wF4GIC7iOg8L6XPg/RoX0P+xtwAAA964DXhzaNyCDM4DLIVYAzYpAA7VLfeDCKD9NwrAADlXV8Gd/pinQAAGREqzoKMCJTKMVJD0KgMMcO62rfMzA1GlyeEFN5ysrZ+oK0DWRFYqjEyGbDpgKlmX465oQWZwChozToQTRUwVoa1HPm4MxJGnBjaUCuO91dKDDXcCs7HW7i1fS0gSZg/WxgykRuQ1p1LdYdtdF3tc+lv3c/ydEJGTxOjiBoun9ja9ZbsJBeq7pd4K0bnJLYQY6Hdth43Ythtt+Kk47QFNUMsTMfAaBjFNzZgrqvDUXgWxQIAKidzubffX3e/40mrQ3HNMTMKC2SG5LmKtiFmsH++AMBQAgMG2aLp0dgmbcFiOcTMD95sI2Y+SkTvBvBETMkHT0Y60YLlawD+AxHNQVxh3wbgwwAGAH4EwMv8999veiRbITl6K6gcg1wVhAlVY8CkYhmkOUAG5d1fQXb2xQAAKlYDk3Rz++Sh81YEszAj61iERTUGsUNejoAkBWc92KQDAEjsWATHeCTCLO2CM9EGnXetkLMgW4KTHJx2AiNxThiaAcPAhRdf16v7aT0GBdTMZFLwPGa6GvBkAHmiGrc/bsRU19O0GRoTap6/LWxkciFMmRGuSc9BzoLYwbBDalI4SkQQRAxyIyHZOI1HquncxMvjOdD94/lou/ZUuFA0Jyr2nVq63jxtx2Jk2w3GTGuF+DTr2vBuFZLxPSAwjDF6qMY9Sg0Qm2KsFlbLDSaWTAKQganG8u5UBdJqjOzcSyZc2YmjPBGlKigtANiIIgjmsNwQ5L2tRqDxQHaOrLHtENEWaoVteBw6C0DphUoPonD/NoA3Yqt88CShEx1j+QARvR7ARwFUAG6CWCDzAP6aiH4MInz+06YHIxIBklRgdgjckx2oHMrDZAxcdy+YTO0SYwbnPWH2SSbWhBcAqY+DUDkCZ11xazkHGCPHtSUSb3lQOQQVohVytkdccbYAACTMoGpUa4quAo1XYJIUrrsXDsCokpc8SwwyiFVjjNweAxdemvUC8pO03VhLbySPcR1IXVkdiisBNYNsxw7UrRMzUo73YQemJIzBsJV5bDGvEFcKQka0f7lWwLXcTpsJyfXK1myUGxEfT481act2kLoxLhW5LbdZTERN5NxGRuEkAZonom23VwUFwRCMVZetAZkk3C+1mjQWxKBgVYYYoHfHjlcWASAE7w2LwAczqBwF91exeAj53rryu1q/e7YZkD8WIhLGVbuqYwFbKzmmGAC2CO8lyIC7e0TpnMU4MLMYy3kA/szHWQyAv2bmfySi92OrfPAkoRNtsYCZXwTgRa3FY4j1soUDOcAzcs464eXgJAeRD1YOjoKSHMh6sr6U7clV4LQDtgVgq4arCoAImnJUn4eNP1/NKDnJwfPeVWBL0HiljqUkmeya+Ol2tb+XbIHclsjZweV9EFtZ72wdv9GgMTxjiYLVShsxydiaIEPodbth3XpInfHyUQE1AAADhp1YcSRMSjV21ehNOZJ58/Mk8+8aQoTYyRyYVK6dRWCy/w9e61qclraSrKnBfo7gsspwGxowaE3ehMjlJsx1zfEjAQ2ihlUUx5xi12E8+uCOgzBuBV4A9X22jmEpDWPXU8z1uuL2ci7cAwLgkgwpvECZMO7x0j0inJIcziQgVZgAcJoCtkB591fgemeIQubPPRoOYYZHQFURvAA7TcFC5Kb7kkjjLf6eqSLoibMO4CqY4eJsxjGjBElm/gSAayYsP4yt8sGThE64YJklNRBY1gKjZZC3RqgagYwBrRwCkgyu0wd39tQaNbsQ6BdLxjXAAPJC1gKBTQqqhjArh0DlEFyMxIW29xxw7uM2JhH3mN+XKiOWCwCXzfkHvwKSXMaojKAcgsqRnNNr+Zz3kXTmRQCSEXcb1QwlNfULNykgOy0V99wOKMhg7K83zeE6ezwSzmvsGix1Vq7JVnKtKkxUYPj9AYCrAnAOnOXhfLVQcUgBmCQJcQFgsoDYqMDmZpnck1xlbcZehdgQghCV7eVbrj9yKTWOX2vQTn1mLYrjIXG+ThtI4FgSEtXVFwsQPb4qDQQA7DAeLHuUY4K4uSGxjy+QASDw4vHSPXIdaUesH+tAxaDBjOVELlibZrQEAOjCKwu2FMWMDIpDtyI/815rrnfW1O31MB4sw5hUrFpgjaBXa45sCU5F0aRyvL4Zeoy0W9JlMp1GgoXAaa2JQ2HHrgruLVZGaMSP3EBneaERmKF3X5nxIAgJYebeijEpXN4HuuL2MqNl0GhZhJN374SRaSAfQHLxA1Dd/llZ4SzIVgEthkpcD1QMwXlPXlpvFdFwERgugjOJ3XA+B06yhra/HYEyHI3E6tAFJgVZcQFy2kHJ4sdXDY1NArLOC6CBMC0vVDjJhUmZRFyIQBCQSNLgFmST1oCGUjK6DYA0yeGyXoi7AM34kgqPthCJl4e5bwW524ylbSm0Yywh1sK1hgyanJgXgwCAWqTE1smkcamgIRJGpa7JzBBGlT+xISRhXz+V6t4Mbi69JzZYiqochefWD3S8dE9jPHIvvOAxqC2eMGBTv1NhHLXAUas8jl/uKPkYHZsUZJLgktV0A04ykKYSwAtBWHGLz4jkfu1Klkl0+ggWdYX5oD2cA3oLskxfhqoAj0fyUHb6YP8gMhmYYigvX94Dw1s4SweFIfQWJPCvQsdV/qlKw/k47QAdV1shmgfjH34kQHr+Zahu/2wtDHwQMfiyPXAASQaywrBBBNfb64P+km9DxUC2T3OYMrKkescWYF1ZHYoriAic92FWj8j1QrRZNmlwEVWOkblC4jDFQMAS7OD6B0SIF8OaQVUjGasXtCq0OcmFEQUhbZpaMlUwxQCUdkIMywd+wGRCYNry2oTMmJJ1Aqtx4DvOO1HfPcAynhb6K5RviQ4bC5WY4TM3gQHNc9fbAbLdnn6dczIcjVBYRp4AvSxK8A2WiZ7cRczVn9NkMOyfdzLozPUxXlmUcSbeUlRFxgt1U/igtgqJyI0m53W1IqDLnRWLVbcph6AZQnkn0crqUEA0tpRnNbhUuR6nJ/Lvobi5c1H4ZkyE2QTvT0c6fQQLkTAqVwVfPhkDt3QoMK3giqlK0Ni7rgAJ6uthkiRYF+z/ky3A3KktF2WKtvJCgWvXT5J7N1EkYNghveAKVLfdEoRTWO8qsUDSHGb1SLgcN7cP3On7bUQDE+uhC8qFCY2zPvbs337wtI6zeOtigyxsId2+3m68dI+82CYBlZV3k3hGpVyXIobFzsdaxJcRLEUPjAgWJZlaoBOByAgDk5sDQm0dtNFTk7LyFX0H784zgBcgkSsldo/CeDi4IIzWgxkDHj3oV1fcAhr4MaYElH5dzmUQCvE8Ip9DFsXCNFCugWmd0hhiG8dYxstHgyUxYgciiYGBSFxgg2UwGXQX9gNoZtabYlWYsVcSoqvz8+5dnUnmQTIeyOIViHCvdorUzRonMMPHjjzKhKpRDbgBYMaD+pmb5VCw6wpbj04bwULOx0iCVum1qiyHGyyBK9HO1HphdYkZj6hRq0ahiCYRhg6Al4/CAOCOtz5UOETnZ9XcXCVIFHZAVTYefrVuKLZoTCovgK0iBmqQ7z8f46V7kKwcBFUjcDYXkGucZHCUYDagydmQGS4G15zr7BELSy0zb8GkF1wxcV/79U/C9vYBWbeRUxTuhVp9fnlg7lxbGsz1Sx6j19RN1BYowVWkrjgvFF3aEXhxrP36YHCcFMlEjRiXbbnFBPJaJ++FYzmHNMk84ssj4oaLqI54i3luHwAgHRzCCh/A/FxvDfJK8zga8+9FzMrqEGnaAcroeVRqWVCrw1EoiwKIUOl4YVMcuTO8A2rVg0gg9JE1DnbCuCPEY3HkTqykCwCmq0k2LSWS3BSAIEFxA7yikgGuhBkugsoRnL7TrqotlrZ1vN0x7brCJtJpI1gAFv8pGQnW2VKsAV3tbG25WAugBLLMx04i5IhzoFT8stRfABkDLkYSnO/0Gy9U7Cogq+4HqtFfaQZYC04z2K9+HPCWRtjPFuKGGxxuXEl6r6tks7SDat9FjbwDzZ4+2YizDlxXLJhk5SBcp498//kojtyJfN+5G+6bXPgt4Ns/K8I5qQVI+I5+M5kGDBqQQG0MIVU0Vzi+Qm89KVMKz4tJRQMn04hPBG2cnQhNL0jYB/RNBHQNGfd+/1AgMQY0+PtuyhESDzkHc4P5m9UjMKMl2PmzAAjqiqmJ5IuFwXiwDGeSkGgaNGgVypqX5WMN45XFNXXBwrGjc+g9Kw9+TZSu8MxWddyMjFfmIkbt70OfR941NRvBMhoOvUXiaii7PgBaPmm4CFMORRHJuiBbeM+AC/PMrk4D2C4RNXOJdqmm00awcFXCHr4TZu8BMdmrMaq7vi4rjZHYCtAIPgrrKYEkAdIMJo+C/86C0kxcTz7Yz9aC3EjcbBpk937noEVprCSKK1ApIAEaLct3w9WCWrMHGg787lw/VJI9UbWZpqV8//n1n0j4bSZUlNLzL1t3XXHP7TJPSdZwX2ksgGMXDTsQo7ZISASPIu1i2LNauEQE110AJVko/6PMMj+jWZ5pvLLYQLLpvRS0nCdb1rXldGymjklxHt1LhcpWI4lZAIAxMONl0BkXSuwO4q6KhUsgW4LIIDEJ2HmUmLfCQAadBXF9aSyGTYrRcIjKW3uJk7kBO9jeGTDVuCF4srMuEuGSdIL7N5AXKo3APyBgDjMCpzlGq4OZKENkiwCBDsQMsIUZHvVzKChK8ghFbnkIeLQCHo9q78T2RwXa9YVNpNNGsMAYUJrBLR4GpRmo04UbDkDGBJO+QWkOyjLkD38aive/Afl137fh4e2n3w2YMSjv+cBlBR6KoOA0D24zAHKuNAeyTs18YncOEHzEyb0fCPuZ94KyDtzcPrjeXpTDYUB4acA2Ljm+3SqxpxxF6J46RuNdT355zLy0b4cDwTgLGi9L7gI7GOtjVgpDt4UAEIpB0LY3EnIbVQFWGg+WIUEbUysZHsZL1QhUjZBecAWKQ7cKAyQDzuaQnXsJqttukWcryZEs3wWyBZxfF1+f8YhGM1wM1kmSzwlSMaLV4UgSbf28OUoCCGGu18V4UAaXkvEgkvKuLwMmQXbWRQAQvsfLR0HGxwZNKq5LoM5dsoW3EiTAXh24eNO5mpqcBeBzuyL3l6LAqBoF9KWiDqkagasS7EE7XJXijZhR5j0IMLuCZSKdPoKFuX6InJWSIXlXXFhVWQsWkwRXF5ei2W0mVAAgufIx4bf70ofB41VxqTkPJa5KuGIk1g0khpOefSEwXALSDG44kHGYBOj1RUB5rTa5/JEAAHvnF+Hyfsij8AeCowRYA1j9xqG21RDTpHhDrNkPRyMgyRsuEc46npGuCiNyldR9oArJN91/2+Pt9PeE3+XBr3nGfMaa7SblfGgcSiHp6qYr7/oyQCTouyQTS2ssldaZTMgVomosUFu15CAwdJ2nOEcIQB10V2vDVYHxtq2NYIHrAnUpxlYBGZBCYUqBsOMYLJbxYFmssQCBTtdCp8mARkswo2VvFSYC01dQjH/v2VpQrw/yv2cVYyECTH4yRTpPHjp9BAsgLoS5BREuKkyqQuImarmYBG60Crhl9J7808d2mvtILbnqI28CALjRAKbbh+n2QT5uQ3lXXCJeIE0DxMzOvQRZ9F/8ygaO+RvPSpmS4njDJOp1u0C3izEZmPGKuKMU9VSOwcUQbjwUxWA0gL31s8gf/rSZjU+1/a2SWk3FkTvBaVcQg1HGOScZOmfeC9Udnxemaku43l5w2kG31wsVjHV+9FsETEtJ0cRc77Zz3QV5htk1YmT53jNRHL077MNpVywXt1pDjU0q2ftkjilZsjh6t8DR/fmVNC7CqiSUUpMv5KVosqZPRlawDpeFID0Bef+9MjkTItq1WNah00ewaE7A4uEQbCdjwM6Bx0Mgy32QvkTvO3/ymE5R3fQWmP4ecNZDcu8HiiWydA/YWlRH5IXLLroUMAnMN/+HWV1ZgLDu0rFTZ2E/hqO5UFzRrB4RyyXJYPIe3JG7BIpeFhi95QZ0n/jsEz1kAFGMas8ZYZm0ByZ0AaTn3XfifuvFNbTkC3n0FmwF7swL4/bMm2zhSxpXa/Z3eT8cuzhyp89JSuGSPeKW0/p6x3a5sq8mCtvKu5lrFKUZLkr2v7OhUrEqjjHwpuH6TnOJl44Gda7YjIjMzubunKp0+ggWW8EtHq4tFQjCC5VoLFwWoCxf10oZ/uMfwnTngkuh85gfXLuRMYAPptpPvUOWpZmUitl7AACQ3G92pX22k0m/S0L2M++VH86id+VjoLk6dvEOiW8UQ7Cz4KpE/sgfwPjtr6qZ00lKx9ozfiIlUvnbJSIsyJYidDt9IOs2Ck+Ol48CSVaXbgmJvumapOBpabx0TwPBRt6SFOSXWGKcdmDGizCDe8S1DYC6Ml43lNgVOysozKqQd9jHOU2vLy7ywoLHI7hiVAN5tkm0G2NZl04fwcIAdYQRc6H4dRviKICYxevRNJqHPXIQqEpQfwFsLUyvD+r0Qoxkl04esl/5mA+Y15qr/cx7RXP1rhYerYCHWkpd7r89chAwBsM3/v4xu0pPZrJf/TjowMUIladNCu0/BMBDdTt1Fn5EnT1nBMDBeGURplgVwaS1xDQe4g9W3f7ZDYEQ44GAX1SgIBYqAOAqmCXxBNgjd0vkxlm4wTISr8hl1z4lHK/6+Ftr68TZ+p02iXeJSyx0lrSLCptMp49ggeaneKqK+r9nLnP/6Rcm7jf6p+uDBoSqQOdxz5q4XecxP4jyQ2+U6iJJAq7KRlB/l04ySjKQFu0kU9czMwlcZx4GAJlELN1ihNW/emkz/+M0I/elD4vrb/UIsrMvRnHUK1rsACQBwSYotsk1tfIz7yWFSjVZVCHb5aqHcedNSPAGRNVYqoD7fCIaD8BH74ZbXaqtCpPIuxa926bXBzsbQDf2U+8A5d2gWErOmhUrxVpwMYIbLIfjUGfjuNzURARKdl1hk+j0ESwsbi8xe0fyIEbaydz3/9LE3YZv/H2JiWQ5kGabIsSyhzwZ1U1vAYxBOkO31y7NlpKLHwD75Y+Ke8QTw9fKclYC+aUE7tlZ2MN3BJcYGQM3w+zsk4HcFz8YAtzJyiG45buB/RdJWaJUwvkhWG5SsVpMGvqwrA5H4vqpxqJYlat1YqKWUWqVcyF2qO74/LpxICpWgzAzw0WJVxa1QFFqKIwe1k9pBvvpd8MNlgT9ORzIe1+VMHN7oK0nYBI5pu6XZTNTGoiAJNsVLJPohAsW3+P5TwDcD/J8/yiAzwL4KwAXA/gKgKcx85HJRxBiZwXtBTTcX8D6QmXltS9BsucMoCpgD9+B3lOeN9WY02ueONV2uzRbWv2blwPGYO77fn7dbeyn3gHMi5sE4wHIdQKSiDOJj1FRgIdLjUQ5yrsga2GXlrDwrJfs6HUcT7Jf/ig468BovCLrSb8iW3kAQw8Yl1IV2qS++nRRV49gh2LxEBLfhbUu3eJCCwgAzcriraKjk8rpF0fuhCmHUgqoHIuAr4pQz486XaAqxcJIEgHg6OE7XXGFeUUgXOvhO2H6UkrG9PegOnxngP8bD37gqgQPlmYytwDB7FosE+mECxYArwTwFmZ+KhHlkBoQvwzgHcz8MiJ6IYAXAvjFqY4WVTFdT6AoJXvOgJnb4+HHk7WY1Tf8LtxoFfM/+GtTnX6XZkPjt7+qBmIYcTtigzhYddNbwFUB0+tLWZ00F2RSNapLePjabWxSUHceXAwBOwQPB6BuH3NPes5xubbjTVSO4QZLnmGXMGUGaNkjbdfLLlQESC78FgBaLyytg/GslYUNksFBOXY1luoAPA7VD8jPf5zYunZMI5iVQ3AaZ9HleVfudVUGwA2A8Axo3KT40s3eMrEhQRmo46j2yMG6BqBJ4JaPBitIFdBtE+3GWNajEypYiGgBwKMAPBMAmLkAUBDRdwN4jN/szwC8G5sIFinj4uGSne5UgVfKuxtCD+96+U9h4fL7gooRBn/xm+g//Vc3PeYubZ/Gb38VAPggrG8tkGKN8NeKCYrQM/0F2QfCTJEkYHSbKCUiYYReaFGWIzlwLuAcqo+/Fen9n7Dh2FZe+xIYn71th6swWQp27qRUPNwX/h3IenB3fzVo7gLPTSTfQ0sVaTl925zffN+5KA7dKvkiQCgJY8YrzXiM9jUiA/btvkMw35+3vPOLMOOVkIBKthRBkWXBbV0nF49EqPT6Mt7xSCwYk8CtHAW1Ep41tYC6ffDqEtzqsiD7nIPJu3C+CR8AuJWjDeVzO0SnoGAhokcAuC8zv4qIzgIwz8xfnvV5TrTFch8ABwG8iojuD+AjAH4GwDnMfAcAMPMdRLR+6rWSFpW1FrAWK699yVQvu2hH8qAduf6FSPtduLLC+Iiv6+V99K5ci5LZpdnQ+O2vAhcjdJ/0HIzecoPMuS+5Q4i0WGdR3fk1jP7peomL7T2A8kNvFHRer++ZiQ3l9uN+OyEGwAw6eie4LAPjoiSRPCds7G4b/t3vAQDseByCtuxjMUHgAMIAi9FJIWz4qM/P0QVl6dsN51IbLQ60TyioqC6s4tCtIHiGTNI7HrYASifznfryReMVqQxgAZiyTna04l6zX/8k7J5zkFSjkLyoQBgA4ZuyPFgtgZyF6c7V+SrtHBafDM3DgbS8LkbBBaYBfcq7oLm6MsJ26VRyhRHRiwA8GMBlAF4FIAPwfwE8fNbnOtGCJQXwQAA/xcwfIKJXQtxeUxERPRvAswHgwjP3Ck59Apzw4O9J7OSs5/1eY7kbLMHs2Qe2FnYg5TFcWcEVFSgxoMSge5q6R04Ejf7perjhQNxVaS4CPSqxM3rrjfK/0/M5CWWjBIc9cres08oGVQmkGeyRg0j2nSUZ9PCooawHDJfgygK0cCZQjUGJtj1IAJTg8TAghezdt0kCHQA4h/HbX4XkwHkwvjyLuVTevaS/R6yV5aOgxMB4aCsAuKqEybsSJwCw/Ocvxp7//OLjM7kTyC0e9q4gaQ2BNBOGbRJQ1gkFVAO8N+6o2iJtmKX3wnUlIZI7qaC6fN8gJBl4uOzRVz0k5VDiOp0+OO3CJZkUY10+LGPRlAC1VqpBo3WFybtwy0dEWHhgjuaosMZUIekCDIDHQ2lVbAwo78IevlPGb4zsPxxs6KXYChHRqRa8/x4A1wD4KAAw8+1ENDspG9GJFiy3AriVmT/g/78eIljuIqLzvLVyHoC7J+3MzDcAuAEArrn4fLZjeTHUujj4e8+DyXzvlJZmoZopjwZwo1W4soLJU5SDIc786f8x6+v8hqfRW28EgNpnrrXcvCKQnnMRaP95YlmsLgpMNIp92YO31QJF0UjR/vbIQXBZwPQXwJ0eMDgCN1iG6e8RxucrIwMAdeZAnTlwOQYPB3Ajz7Ri18w6xGUJVywL49IOoCaBq0rYURGUErYOlJgTKlziwDYDIHUrGhMC7LBlaCss5fXziceiYiDl8g2Q7RcrphocljiK822oqxI8WBZBpgH3JAElmbTWtmWw8PR8ML6xnnMiNLRgrP92Q3k/zdwecLEsMGRjgvUU7lWnV8dZxkOx0Hz8hjrdEGsBADucZYzllBIsBTMzETEAENGO9eA4obPCzHcC+DoRaRbVtwH4NIA3AvgRv+xHAPz9FAcDW4dqMAJbh2JpEFapcDn0+z9Xb28MKMsDXr4cDFENx0iykzvr+pQlZwPj4KpsaI1qofDRu+GWJRPb9Pqg7hzM3B6BgPs2BsE/7pmOWjXqMqE0kzhLWUr5nfEI9shBuKV74JaP+gq4Tas22Xc2svMvhpk/AzS3AMpy2MXDKL/+OZS3fwXjz30suN+o10fS3yMCDgjXQcYgnes2GA1bB7YOS686MS6x7NqnILv2KWGs1GvyES7HwuituqzSRn2umNJ7XYV837nN9ghAyHdxgyVfQbiQumt6b8oS7NtnS0HYIliqoZbfaFXiJN5KRXB1eUXAmCBsqNeX99baKE/Nu7/Go1DOyY1WUa2soFpZgRssCXihKoICMCsyCU31OUnor4nojwGcQUQ/AeDtAP7PTpzoRFssAPBTAF7rEWFfAvAsiMD7ayL6MQBfA/CfpjmQK6vwYpsshUkSOG0zbFsvTFWK6VwVYOvQOWMP7Kg41TSQk57Gb3+V+L3Vd25MaHEQlqlbyyQBLsrFSALrPgG195TnYfiPf1iXPdf8hNiCyXJhniYV3zoZ0XgHS8DyEV+AdID0wLng4TLs4mHAJEj7e4BuH8neA7BH7haXUd6FWz4Ke9fXpNJCXCrIJ92xc2A7hslSUJqDK3l+XCEWcyh+CkwFDNgpksKoBuSrOoeik3oP1GLxFYGnHWt6r6skV6i7BxgOfAZ/HS8JpVV81nsjDcC/l5R4K9SY4NrS3CO2Pq8oIMV8uRaTyPurXWAh6QaUd4NFpMIjm5dkSgXerLx2hlByOnX6sZD06/4rAJcDWILEWX6Nmd+2E+c74YKFmT8GCSi1aUvZh8yM8dFlecm9cFBRUq161EhisPo3L0e1soKkm4NHZdAqi+XBrgtsxlS89y8B+ByRKFHR9BfENTFYAleF9M1JpE20PXwHkn1nC8OP8oVGb72x1lydlf2dFQ1ZXSpGasKhJygmt3K0kXCnCXS894CAA/IuKNXyLgNQrw8+WHqLyol2nHdlXd4VAeUbRQmU1QVrOHavqCuMrQiW3pUP3PnJ3oDi+nXuC/9el12xEmfi8RBmz35ZVkXVgKcg9u0fqNcHGSPCGgitK9hZiWto91ZveZIX8uRLsKgFSpm3OIsReHVJYilVCR4NUC0tIunNyX1yFq6swLaAyVOwdXCrR0MgX4V7uSKei6M3/DLyA/vhigq2mE1ZF5qRK4yILgTw5wDOhbCtG5j5lUS0H1vM51uPvAvs75j5QQB2RJjEdMIFyywpFips64dLtcbugb2+X4sL2wBAsTyYcLRd2g6N3nKDwH+1zIYmvJWFDyZ7aCoAt3S44VISjbf18vuqCtTtC9on7wJpv9kWwbefJnawh++EXTwsCoV3AVGnW8doTOKFignB4LC81UlUl9FoIOeCBOqD8PBAj9gqpsQgSXJxzx68TZjuQYFRr1cy6HiQVt12X/owuLcAKkYiDJyDW5Us9uwhT576eOkFV6C6/bPg+f2g0bIAJ6wFilGoZKDuStPrB0uDAXFhVUXTWoHEhrTMDpwDD5Zgx2PvkRiBqsILFQd2LsS2nLVI8kzefSuCJ89TmCxFMRxg8XNfxtzZ+5pxnu3O52wslgrAzzHzR30w/SNE9DZIGsax5fNNpn8noocw84e2P+SN6bTx+5Cv2xMXk2TnYH0gP+v3UA1GKI6uoBqO5XswCjGZXWtl9sTjobg1vIYK3xPHHjkIe/jOEGshXyEa8JnRapF4qj7+1trygI+j9KT3DfvSHepbF+FSgfIuTH8PzN4DoWBhGFfpNfViVENdnfUowTNkG2/dqIXkVuvAMSAC0GQp0oW9NcwYWBNjcWWF4q47YBcPC0hk5WjI0zmRZO7zYCQXfgusTxx0q5KNvhWhopSef1ndorm3B2bPfnENFiPwYEmUCbUuAYmHlDWU2C0flbnv9cHDAarbvyxuyMEyeLAUhDh5haRaHcGOCpSDIcgYlIMhbFnClWKNqFBxZYW9P/abcl19UXBcUQWFc7skqLBkqs9GxMx3MLMitZYB3ALgAgDfDcnjg/9+yjaH/K0A3k9EXySiTxDRJ4noE9s85kQ6rSwWQHyr7BxcpD2e/fOvDL/v/t2fAQAciJbt0uyp+8RnY/VvXg7j/e2mvyB1ndIMbvkIqNOTpEeFG5sE7BPbeDySuACknUF23sWguYXQT0O3gbMwe86AO3J3DV8GvJVhpPDgcCDacJqJlaNUlXVQFxLA1yxwFVAy1qMeAu339SXazZ59IUZBaQ5aPgK2Lvj2TZY2YnZkEvBgCWbvgYaQPNGU7D0ggffCbr7xRse58FtQ3f5Z6ckChMRGtT7je8bOAf5euqXDUt6+Owc3WBJ0X1WE2IkG28kYyS876pFeXols55epJUOJCTygGtTzve85L8OR66fOaNiYdiBBkoguhkCCP4BjyefbmL59m/tPTaeVYHFFBRvloZz/q9ev2ebsXYFy3KhdTXrxxl9F57wLxIfvLOBMQImZfs9DSkc1eshTedsXkew7G+zdZG51OVSjphh67F0vdvFw8N1zMQL1+lK6Z3VZhIv27gBkWXfO16YqZPtuP/jqkwPnipV1+M6QW6NCAv0F+TYJ7Eh8/QYiUKwtgntMffpsHfI0Q7L3AMZvf9UJdYkp2Qvuh+TrH5cclW1Sev5lodIwALGEBkviDlWB4qy4M6NyPY1vIJRicYWkACTdXLwLI0knUEvQZGkQJACCQqn/Vagn3dwDLWT5vue8DPhvv73t690i3PhMIvpw9P8Gny5RH45oHsAbAPwsMy/RhITVbdJx629+2ggWW9QYeVdWuPBFN57gEe1STIs3/irGR5bRvXdfgrSt8vSaOMmjQQjEj9/5577zXyHutDQL6CONuTggaMBm7wGBGFsp1WL6C3AeLVTd9XU5ti/fQt05QaHNnwE4K4LDn5eMgdmzL1hWWgDR7NkXYjqoCo9mK1AtiZbOEVNTPz/8+KrhGCZJUB49GpBvo3+6/oQn4OZ7z4S9Zx9MOdxW7GH01hthfO09Hg4kSN/p+gz6Asi74MESqL/grZlhEDhxFj2chavKRj4QW4dqNA4urGpUIMlTlIORJKl66ySef0DuweE/eAGSPMPSl+8AANz6op/AvX59Vghb2koHyUPMPAmkJEciyiBC5bXM/P/84qny+bZAb4JPaYJ0vLs3pODvVds87ho6bQQLAJSDEapRgYt/+88233iXjjupnztk3sMDK7wbS9BJ+zwcORe/e38BdjQIGdSSqe2RS5FVwqMBsPdAyPhGVYC6cxLoHSwHgcVDQX+5ZUGMJQfOBY9WfZHGusy6PXJ3jQTzrjV4F5j1bah5JLGCpNNBtToMwWRKDKrRGGwFNRZKjuQSS7JH7l4T9zmRlNz7gXBf+Hcklx5bZY/Vv3opzL6zBTKuMTKtQrBwAO7o3eIy9GV3QkAfgB2syBzlXdjBSnBb2bIGR5g8Dd6IgOJcWoUtKyRZCltWyOa6AQ2mKDGTpRgfXUGSpzB5zerueOmxtSZvk3SQnAkqjADcCOAWZn5FtErz+V6GafP5NiBm/pbWeR8I4L9s55jr0WkjWDSwtytUTk5SbXP1s59G/34PECvAJOLWMpI4GZqtaRa2D6hL/behJMZ5ocLWgoyt4xUe1eVWjsIo2gha1sQFgWSHA5BHIJnuXIDBhrwY1MgzzatxwwGAgVRpQF1Bl4AQD4hrh2mgWf8DCFp3HnJepG/I4T94AQDgwHN/Z+cmfwpStNhmNHzj76M6ek8QoiZLkXQ6cEfuhj1skV10qdy/8VCsxSwLbi7NY+HhAON7FoMAsKMCtlgEO4e020GxNPDCuUDW74ZzxaWW7GoztqKxLecc4J8166ta25FB6o8zPiqpBrOhmTX6ejiAHwbwSSL6mF/2yxCBsuV8vmnJo9AeMstjKm0oWIjoe6c4xoiZ/2lG4zl2YsYFL/rjEz2KXVqHbFmXPEeaAeMh3KoPlvuSINTpin+9kAKFoby5Jtz5goWaUKloo1Cq4/Cd8v8sYV5u5WhgZqYfNX9Sy8QLMC6LNb3UTVf6yvN46HOg/DmO3A1XVkj68w3IMnnkUkyqPZMxQdi4skK5tIR0rqjbIkMqaZ/zC/9rxrO+lhZv/FWQMVvuOVO8/w2obv+yz3qvhSUbF/J6TH9BytUrSm48qkv4APW98oJYK2TYKADvikpch15QlINRWKbzJ6kEpRc2JUyeoVyt75ONYixpN0fW70qcxTok+Qx1aaKQx7QdYuZ/RSiju4Zm1k2QiJ4f/TWQOo0HZ3X8mDablf8DMb82iiI9CsAJFyzZ+d90ooewSxuQMqOs3xN0UJqD0jL02NDS9aa/B7YYidDxFosSAXXCnC/FI9aI/vYxjVu/uCYrOwT0tWSPd2/ZxZEE/n1BSo2baM5MCOjnkgND3T5I8z48RNYNB4H5xX7+pJsHZBglTQQTW4fO2Wdh/n4PAAAMP3fzTk19oCPXvzC4hA7/wQu2biX5emqAMO3y6FCubSAxqAPPrfsfjd5yg7dGI41eM+qrAsXSwMeeCuQLcyFeovETOyrkd5bCuTpfyEWxVABBuAASWzVArQSMxqgGQyxcfB7sqMBosIzx0RXxavziH2zt2icR1ec6RSguOFlBYi5v2IkTbSZY3szMP7rRBkT0f2c4nl06DUndPQCQn3V2XStKyWu7PJZkPbcsycWx8NBcER4Pg+XB7bLp/ju4n6KSKsp0Gs3CqjIE7wX+nAWhAqDhZuPVJUnMnD8DSDO40aowu1TgxuxGwd2nQfu4JpUKGy07xNbBDpbDPHQvvmR7k7wJ3fHSn0S+Zw7V0jho2bf9+n+Z2spPzrsE+XXfh4O/9zzJCVseh/I16g7SKuImSzF3L19TrNsH3ChUOeBCcslGh5eCIDFekGiCczkYoRwMYfKsYc2wdSgHI6S92mrSfjgxMkxcdBmSbgeuKDE8eAT5nj6qwQjZ3Iz63QOg2bnCjhd9mpn/Jl5ARP8JwN+ss/0x04aChZl/aLMDTLPNqU6ff85Tcd/rXw8AuOkpT4CzjLSXIskM7vfaN6273zuvfmj4/dhPfHDHx3myUrvSATvnM7PVNWWCW8sNlsU9ZW0QAq6opDy9b4vA1iHpdMIxY604ZnTtZXreYMno+tGqwIY7XXBUxl8sI19a37vU4u6DbJ0P6nuh4RPzADRcJO2MfLYOlR3DWQuztIqkm2PhWZs3pjtWuuvlPwVXVCiWV5FkKThxGB9Z2dIxkosfAEBaTxz8veeFuIe6mNhKMrJJJHHUDpZhRwWy+brJlj1yECbvorN/Lxa/codYfWWJcknm1KECSkiQ3jmUS4MQh0m6HcneTwyqoc8VSgw4EavHeKtQ1yXewkm6HazcdhBkDgfwyMyIADM9KuxkoF/CWiEyadm2aWoHIRE9DFKzJuzDzH8+6wHtJH3kux4Ptg4P/qd3TL3PJ5/+JPQOzOOTT38SAMBZYUdsHe7312+euM/7H/toD4HkmZaPOBXp0O//XAia5nskCU4sB9+LfG5PiHW45aMAUHcU7HRRLAvT0ZpPxdJA3GlWmJq6mKrBKLie0n7XHyeyEpyBgaKFIG6xTk8y6gdL4kYrS9jhItKFvaHQJPxYQwmXlZUadZSncKPVUFJENWeTp6hWZV9n3brIoaqUMWt+xk5RO3bzmR9/itQwO7AXX/yZHwAAdPfvndp6KZZq4Vp6FJfW4rPWwWRyP2xRohwMkfV7SOe6ki2/tCoghj1zGB6UpFKNj6TdDpxzsB5Rp4LCJAYcCW3AJ0I6U1s0XmAr9Bjw8RpNkhwWKAcjdPbtwVd+8UcwKzoVLBYi+nYATwJwARH9frRqAeISmzlNJViI6DUALgHwMUhvOEAUvlNKsCh96ge/Y0NLI6ZqWGF0ZABnGdWwQjWqBDO/tHGukS0sbClTdRKVzT6upJptsbyKYmmA3llnhA5/msnuVgqY+TNqyHAUFynuPohiaYDx0RWwdUj7UpY+X+ijWB5gbq4X4L2AD9b2u0j6843GbbYo0dm3B6Y7h2ppCdYWSLoAxiZUVdagvmrbmoQH+PIteZ2Mp8cEgLTXgSuqAC9WbTtO1FPVwqjV5I/prIMbjI5rF8JPPv1JyPfUVYBNlglkd8pS8uN3vxZ7v/kCkDE4+rmvY3x0Gfken6zqrQZKTJgfOyrQPbA3CN/Fr9yBajAMwXkASLK0IWDVpWjLKswTRoXMaaSoucKCktod5axD2suBSKDYokKxrMKPxKKaUQCfZhS8Pw50O4APA3gypEuv0jKA5+3ECaedlQcDuJKZj1vm5qzoA4//VgDAtW97F2xp0VnowJbTWRE3PeUJsP7hrYZVKBNRDeXBf+91D8cj3/++Nfu5wqEYFGBv3Tz+cx+d0dWcWqSMFhDN9p6bv4zzH/UA6b0yHknuh3PIAWC0GuIn1coKysEQ46MrwWJh65B4VI/WiAKEQY8O1wmK6bwIFTsqQpKiSSTp0Q5WQkmQfKEP03JbVavDRkHJEDCO4iKSG7EcrrHwNalUS7ajImheYYxREp8KEXauduvMEqm0CZGhEKMYH10GJaZxPZ/58acAAC7/k79r7Lfy2pcgv+hSoCqQXXAJDv3LewAA2VxPXGBWrivvzgU0XDkYBSulWB7AWYejX7gTeT8HJeSVL4e8X6PA2sU8Ac20r/9bX34mydcGzoMw99+jo8MIxcayz6z6sdCpEWNh5o8D+DgRvY6ZZ1PaeROa9on+FKSk8x07OJaZ0E1PeQLKkWcYpu6XoO6pckVQOjc95QkweYL7r+POAsTtFWthxaBANaxgSweTEJxlvPPqh06Mn7Bl9M+e25mLPIlJ41EayI3dEc469J/+qxj8xW8GZm1HBYrDUmo9netiePAIkiyDHRUYHjwCV1RIujnSbgd2VGDu7H2h4ODK1+9E0s3hygqdMwTwoogjoA6Upwt9FEdXUCwPYIvJPXuSPGuUU4+Xs3VBeHUP7A2WjIsqaKsbzJYV0m4TkgsArAwo0q4BIO3lx1XrNT44fvmf/B0+9YPfgawvsSpX2CBU2rR4468iW1iQSgR5F3e//Z2oRoW4CxUgkWewvqCrKhNsHdJujsGdh1EORsi8i1LfSRWyxaBE3s/AlsHWIut3vNWj7sW1+iy79XVc5xWQBkIvT4Ibe2Z06nWQvJiIXgrgSkjmPQCAme8z6xNtlsfyDxCX1x4AnyaiDwIIDmFm3nop1B0mfWhtYZH0JWlOLYxr3/ku/NujH4XOQgfGazv/9uhH4WFe+4rp/Y99NPK+YPCdd2t1FzqwXUGuDO4eoBrKg/+2Kx6Ex99SW5jlqIKzjHJQorPQwQce/6249m3v2ulLP+Gk/vo7XvqTSLIUxfIqxkeXsXzrEVBikHVTfOG5T0P/grPQPWMPnGdMo6PL6B3Yi+LoClbvuCfESkLbgyTxWdcWgzsPBwZVDkZwh5ekRpcXEuVgKBWFl1Zh8hSpFzzVaIxqWDfiKj3Ts0Aova7BZzImLC+WVxuuquHdR0KAuFwdhcxvQEu62NppHemG5IxcV6uyLmUp0m5n5vdiPbKlDULtfq99Ez7+tG9HuVIg7aXAAHjAG/55zT7VcIzxkdtAiQkC3Gmyo8+C16KQw7uPghJCvjCHpNsJdb4U7ZX3c4yOjJD2UlTDCiZPfPKoCmsO4zOJgS0t0m4KtlwrjNE7LkH72m0GiDXaOWMelJgAullDr56Ne/p4ujFnQK8C8CIAvwepdPwsbJxKcsy0mar0uztx0jYRUQLxAd7GzN+5nQY3lBjk/QzWl6o2hpDkCaphhfdc+7AgZKpRFUzq9ciWFuwfdJMYFAOP4XfsS2GXTZy+pyQzGC9ZAAlMLus/+MTH4qFveec0l3BK0mef/b1Iezmyhb6U3xgVGB1ehC0q2MKhty9H0s2ClTK48zCqUYH+ufs9jFT87sPDi2sgxHY0FriotZL/4DXRcnWI8ZEB8oUekm4O4xmfK6sQMxguraIaFY1+9OrecqVYQ7HV4XwQGoCUAvEurLjCbtCCvVCJcyt0vPrbeIsHqOHHdX+gBZgkQefsM4/TXYJ36Qov+eATHxuWs2Vc83dvnbhPsbyK1TsPo3fWvoYQdEWF0dFVFCslOgs5kjyByRMsfnUR3X0lDlxxr5DHM7zzMAZ3DZD2Ui88OCh37XGVhRWB42S78dI4uMgabQkcw8EBBRrH0m1nmgw5gYi2VCvsZKAeM7+DiIiZvwrgxUT0XoiwmSltBjf+FwAgot9m5kaDGSL6bQD/MqNx/AykB8GC//9CbLHBzfKnb8G7H3wd9t5rD8j3mS5HFdgybCFamrOMzkInLGPLjbLX77n2YVGAz4TtKCE463DdO/8FH3j8twbhlOSTH6pHvv99eNsVDxLIpLdaJpnzpyK97xGPhLNuYmxpfGSA3tmZdPLMM5SDUbhuZx26/S7yhbpyMFuH4d1HAaAWNsPSM/MStnRIMgM7KmELsSaqUYHR4WVQQmDPnGxRYfVOcacVS6sofcJeNZLmUMXyIOQ7JN0OjPW1vJxDNtdb26d+5EKAN6CTRkWAllqNuXgrKu5YqBTHCky30ziWCqFieRV773OvmdyXaSlWph76lneGGOQkofKl5z0DgLj/ABG0y1+7KzTGq0ZFAKlUI7k/93xe7sOgGCDJbkfW78COSgyPjOAcwxYOtrQwCYV76CwjyQ1soZYKBZe1zlmSI7jK2Afs1XJx1oEshQoHCoU+HnSKucJGRGQAfJ6IngvgNgDbLcU/kaYV6Y/HWsb+7ROWbZmI6F4AvgPAbwHQkgPfDeAx/vefAXj3tOdaPTT06JD60vRlkmCuWB9qjpMlvO8RjwwILnYMykW4JJlZ48Ka1qX1+Fs+grdd8SBUwwpJlmD18OrmO53k9N7rHh5M/3c/+Do85sPvByBxFZOI+0gzqgEEhlwM5NPd1w+B3WJpgNHhZZg8QTkoMF4cI+2l6O7r+fpRDqMjoyD4tVmSLS16+7rI8hzkldQ4QW50eBHjpSHSnuSikDFIcilGqPDV2LoAEBLpZMxShyyPkvaK5VXY0RjW78POAWUUP7ESfKbEIPFJfpo1zolrZOCr1WSyVCye8RjGbWw5z5LaUPuNnmcdsx0V6JyxByu3HUQ1LDA6OsTS15fAjtE/u4/xolhoqnBRQkh7Ke7+5EEs3GsPylGFJE+QdX0b53JtH5NYqFAkOCgXl1k1dCFOknr3mYMBLCPJRAl0EAgyjcYbMvw4v2xbdOqgwpR+FsAcgJ8G8BsQd9jssNcRbRZjeQ6A/wbgklansT0A1qqsx0b/E8AvoFluYKoGN0T0bADPBoBzfA0j9cMCVRAualIDQDkoUY4qZF0xyTPPsB71gX/De659mBw3IeT9bF3XwDT0lkuu8ZZOhcW7T+3Wx++65lrk/Rz5fI7hkbpp0j9f+kD8x899FNWoCK7FNPKX28L5uk419HN8dLkBGy0HBYZHRjBGkHfjZIxqWAVUHftkVAuLcqUEJYQhRg3/PCUG46VxYFwa2GXrkPW7ISZSDUskeSLxtzxBNSwjd1sRIMUmq6vpuuVBFOsxHlpco4rSbu7jPwxKgDRKwrOFZPurBZdGVpCDCJhiaYD+fWYeO90Wvfe6h2P+nD5MnsAkhOHhlSA0TJ5gcNcA5aBE2kuxdKuU3DGJQTmqBNQy4rDt6uEhUn9fylEVOirq8ZxjD7IxXrlLQtMxFRwqJFRBdIVF1s8CujMg7mAAeBDFaByUhZ0iIoI5RUq6+HDD05j5BQBWIPGVHaPNxO3rALwZwEsh7iilZWa+Z7snJ6LvBHA3M3+EiB6z1f19o5wbAOCyuT7ryw0ICoQtByYX/NyRGQ4AxUCYxPse8UiQIfT2dT0iZTZaJFtGZgzcqYfUbpCzDrawEr/yc5P3xSoY3CWCU5Bzw8C4x4tjr2XWbonVw4MgOLoLneCuhCE4x1g9tBrcZxqwFYZkgmsMQMOVmeRJEExJboLQSPLElwcpkPVzsKtCwJcte3Sf9cH9AuQDwLaoAvIr/q2kcbdqVKEYlKIxO27kVIiAGQYLTZP+dJ4SH+dJux0gzTF+92vRecwP7tDd2zoVgxJJYbF6eNh4L5a/uogkS5D1MyReyHQWOsjmM9jSIskSOFuhs9BBMShBPsYJ1PcsyUwQKCpU2Dq5h6N6rtPInRjnKjl/75yPxdjSgoy4qxMP9Ej7vXWtFjMBpnysdKq4wpjZEtGDfHxlx5nRZjGWRSJaBvAtPtgza3o4gCcT0ZMg8LcFX3tsyw1udKrU7yqMQx44Wzp0FjriWjG1bxbwWrVPshOopEOeJ6BtBv5sYWErh5XKYWgdktl3gztu9K03fUDiTwmhHJSwhcUZ37QXJk/wrmuuRXdfV5j5QDv8MVwhjDdmFKt3rwRGwJYxWhpjcNcqTELI+lmIRxWDIlgWeT9HOaowOjqCs4y5A70QzAVEuFVDsU4NHIqBRe7HMF4StJNJCKMjQxnjqIYbq5Aa3L0c/qurzwWkVx0jsmXtcqtdN1wLtCyBHY1RLK2G+E+SpaiGUljRoQwJiUk3l2zzfhe8ugTT34PBX/wm+k//1Z28lVNROZA413jJiiXYz3H4C0cCJHjp68vI5kWpyPo5nF+W9lKQESj+6qFVJJkJ9xGQuTQ9nfukRoIFtFed60OJCJkkq/OKnGNJYjUEF7m3rfXCOjNyfw0h32MDejCm91z7sLDvtunUgxvfBODviehvAAQ3StRYbGa0KfdkZkdEHyeii5j5a7M8OTP/EqRWDbzF8vPM/ENE9Ds4hgY3lBCSTJBY6n6JKfMadjko6yS4yCWTdlMJ1nuEy8ef9u34zLu/2hAKK5WDZcaP3fOZDcfyHV//BF5/zlVQGTY8xYP3q4eGSDyTsYXD6qEhSs+kr3vnv+CmpzxBGGtpMToygi0sikEp1qExOHDffXCWsfL1JXT2dpD2UpQrvv6WZQzuXm1UK1DGXUDcbMVK2YibSZCWwd00JNuxJrAGZSKHKywqyy2BIAFktbhs4QSIkZlw/gSJh+Zy43zxf7VsnXe9lUkFWlrFeGkctHRXWCTdLIw7yWp3WL7Qx/ioz/IfnjzuUhNZmWk3xcrtKyidw9GjIxw4q4/+OXNBcdN8riSvFTYyPi6WJyFAL4JW7k/i5zkWOJQQqqIK22RZilITIeGTTCHC3fllzjISf3zAgaP8mPFRqbzw/sc+OoxF3W+zo1MOFbYfwGEAj42WMYDjL1g8nQfgZp/HEku6ncpj2XKDG2MoZPRe985/wfsf+2g8/F/fG9brA1Z6yLAkOtaaiwaH9UWohhVGS+OJlsa01sdT77oZf3X2lZhPDb7/7k9Ptc/JSk/84k1404VXS06Bc1g9vIr58+bD+rSXYvn2FQzuHsAWLlQr+I6vS2hO67SVgzJyT3KwXhRuqgLE+EIoxcB6i0BLvXBg8CYhjJck6J92UzhIPKdEVbu7IsavZBJCNdJzmjCWygeZnWUkGUIZH1nv1lRsoOi4iowaL4nVFreSyvqdIIgUbaZlT4qlVSRZFhSd5T9/Mfb85xdv405tn2ReC6zeI+i6rJMABZBnYkXYksJ24hWoEV1qoWhcRRU8RVlWowquaIJrlMifV4PxISetdCB/35U0TmoLsZDiZ8KWFgkSLN92ZE2eCUVJ09slOsUsFmbe0bhKTNMKll/f0VEAYOZ3Q9BfYObDwNYa3MxfcXlDkFz3ziYSenho6H3gFPJRpNaQiZIqHYaHhpKzslKgHJSYn89hC4thYZEbQm+LD+WpLlCU3nzx/QFI/o8GzsdLRbAEjn5lEYe+fBQAkBkTNNh/vvSBYMs451vOwt2fPSIC3BfnrIYVBp5xdyMG77z1UWdoi0WiSXWUELi0QCJ5E2Tknjq/DQCYnvjinauFVmXIB5hNiBnpOQERjnk/8wxuLIIlJO7VGeCxe0xJBR8lhKybBqGlrhyttKt1xShJkPpOhs5a2FExw86G26MkS3DHXUewL3IHd/bK2IoVb4UmBlZjXAZhWbDoHAdAR7De/JwnmQgfW1iJTRkC7ORsep1jLtinEdRCJ8RBk9o6hAd0FCulj/nUjD/E42YlDE4xVBgRXQrgegg46n5EdDWAJzPzb876XFPNCjP/CxGdA0DbWH6QmTeNe5ws9OaL7w/j4Y6q1VTDyrtABGKa5OKfRZIEJgYgCKJ5RRNtUEridKe5M3sYL2nJcpKYRDfFu665FquHhij83HS7phEgtYXFnR+7C+VYGEFnTy4xlij+UjqHhH1ewtgi6wCAaQh1Xik9IihH6isgaFDfJBRcIcqMVAjasokiCoLGnz/JknC/nWUgsqTawicWJsoU2XEABcTE1sF0U99nJJFEyiTxMb2kju05B1uUoR/JZ5/9vbjshpl7J6aid179UAFLEIV8sIbAsAwHBIsSMDBdcYfBB9PVRagAGlvYkB9GHj2W97OJgkTnDBBrp8518XXW8nBWSaRMBAigHgdE8ROtvtFZyNcWnZ1FzJMotGA4Rej/AHgBgD8GAGb+BBG9DsDMBctUopuIngbggxCX1NMAfICInjrrwewUBajxqPIvsS8uyGI+y8NfM4y4yJ2WIlF/PCAurm80+vavfBy2cJg7s4fOgiT8iVvEYrxUYGVcITeE8+59RtjHJBJkLUorQjw1SFKDcqUUhs9A4RiFY1gGhlaADoUToVOslrIvM4ZRjESZXag2XdT+ehcJgGooiDL1+QPCrEqP5hIkUe3712sKAskfV2M/ca5TmymqUKGEGq43Z0WIVD5pE5DCjWk3D+XnNdZXDoa48/2fQjkY46anPGH7N22L9OaL749iUGJ5cYyEENxLHFlrqvU7/84oxF9dgACCa0qtBlsogq4I11sMyjXWTVzLS5OaVagouULcndWwTn7mMEYX9o3d3KoM7Qj5Vsubfk4OmmPmdmHDE1c2H8CvAHiIWilEdBaAtwNYpxDPyUXf8fVPRHklte82SY1/CVyj26D62kULJZT+5QCA3Jy66K7t0uNu/hDeftVDJJ4wrDBeLDCo5AXup4lvfpagHFvYUYU8SxoMSS2PJDUYFjZYOAkB1sP6xGBQBiPfuaGwrd7DcmmM8XKBfC5DkhukSRpySZTJ6ziBFGlvbVxNLZiyBYtVN4vUqqqrHOsABZAQFVL0fEOZrpJW0x3efRRZvwNXVsi7HVSjcSiNoqVm0l4e8m1GPkv9w0/6ti31DtoOyfthcM/SGJYZ86lB6QRx1e+lwYrP5zMfU7HhXrhhFYL48e8w17ZuH0FJHZdQ11U9X3VKgFo4kvPEHt0px9F1MakwJ817cYIofNA/vG0HZ42aHUlPfjpERJfAv1jeONiRwsLTChbTcn0dxpTWzslCT/ziTeG3+v1jevwtH8HbrxJPH1sOiCbRnFi0Z8cB5fWNStVIamVZD6O2DPQSgmOJZdz6ucOw7IVFYTG0ToSxZworlcM8ECyTXmICYi4hAUaoYFEhnhAwn4qFaaxnvMywzLhnaYyFLAkwcldY2FDOx4VYj1albpMrLBy8MPF5MppQq0JFS4lMyn+ICyYCCC4gjd2Njoy8e4h8Fv8AiRcuJs9AVkrxOyt9XrKFPrr7hkHLfknvmwEAvzb8wuxuYos0fjYYVrDMSIiQGYNBZQNQJcmTOoZRVOH9KZ1DniW+6Gpd+h6IStj4OUy7KaxHX4b585amBuOV2DHKUuDGznJgNrVLTcaV9tJguYQEWSvpBXF8ZUfo1Ot5/5OQvL/Lieg2AF8GsCPJU9MKlrcQ0T8D+Av///sB/NNODOh4kXSba0qJx938IQDA2654UIBQxnkYlhlPvev0CMYfKw0HBfIsCQyocA6FA3JTWxoqIAB4ISO/c0PB3QUAPc0n8bchIfK/KRxD1+n5HHMQ8CrARtYhHUW5JC0YK+BzM3xgVymGDxsAlNTbqTACaveaxhCAZqA5+PchAsYY8kw1DQm71gNFtPQLexeZmUtDRn/SzUOds7SXNhjtTpItHA6NRaj0EgPLHBJ6EwJGowpdH9OKLVBbiXKQ+WXWyn6GFCFmfIzEn8crF5XPwFdyHrDhsiRYO3W7gfodVStREyFl7M2cFJMQHKQQ7RWveuNM52kNEQHpyQG4mIaY+UsAHkdEfYixsLzZPsdK0wbvX0BE3wdJaCQANzDz3+7UoHaa/qNvvKWaWhsyqCXw//nSBza23wq9cuEyDK3DCwef385QTyp63VkCnZa4hzB7tSqU9+q3ouf2e0aswiDeXmt4Bq3YWyttoaLrYosxzxJvNYnlInXgCEkGnyRnvMVZC5LKu2ni/BTNvWFHtdDQOlQQphe7SXUZgOASU+GlqMPQ78Wvj4WQFko0WQr2aDAAdUfHPEPW78CMSnT29dFLDBIC/mDvZXju4me3dsMies2ZV4RjqYWYGwrxwqFl5EbdjvV+Ggfr+utTJCVbFfCE0rlGdQnHjKyTSMFJH4BP86YSF8+rXjs7mXfyFo662toUoOe2WUTWOUaaEAyArN/Fl3/+h3Hv333NMc/ZZkSnWB4LER2AVDJ+BAAmon8F8BKPwp0pTa0SMfMbALxh1gM4kfTtX/n4hutjgaJFF99yyTUNt9okeuXCZbDMmPBOnNL0jIOfxpsuvBpFpCUmRN4C4SAM4jhJLTTq4wRXGTetFUAYmwoYXR+7x+J9DFEQOKNRhdwyxssFso7Ee6yDr6ZLQcgkuQmADQCwsAHppG6ZalQh7aZR3TlBQTmYBuoIeYLEUHB/qbUSrtMzYYEw150j2Tog890mfYFHW1ZwZRk09WJQSDsAAvYeo0vnxv2XAxDLMDfkQRByn/T/6866Er2kthDjKhHxPbGFC4jK4aAIYAvL4qaEFw6WBTpeji2MFzpwgPWJxbm2s0i8ILdaYknui+p4tftL3GlxTEsFStwwTJOjAc1FczsqVOREOJkC89PQXwJ4D4Dv8/9/ENKe5HGzPtG0qLDvJaLPE9EiES0R0TIRLc16MCcz5f0sWDDTkAab1Ud+upCtXGBKiujKDWGvf6lVK153fxZGJQJEhVAzeA9Mtlg0yK/blM55940IF1tpp8oa5Vd3r/THL7T2VHOQ6haLYbVKJtKcNSgdJ+8p3DhOxgyZ+n550+JxoWRM3H+9Gha+nW9dGeIJT76vn1PCqw9csf7EtuiGfZf7vCsT5lnn1/r4lMxrU1jr+hXv5lK3pWOZ33JsPYqvFvz6HOhyx4zF0qJ0alEiIP/YynFsIbW+nI9h6nxqx0h2LnRwleVNLS3OKXK+zlitPCQoB2PsPNHMUGFE9KdEdDcRfSpatp+I3uZ579uIaN82B7yfmX+Dmb/sP78J4IxtHnMiTasKvRySSLOXmReYeQ8zL2y612lEg7u10OLm0fufWfosRo5RsmjCP53ce4dHd3zoNWdeEZiSCs5aIHAQOIm3JNqk28SMrk0xg8qMCccxreOJBiy/9XyWGUnafKRN5DaR75qh6fLgz3cuQI21Em8ch6tLlpiG4NFAPyAVIGwpAkYZZjWqUA7GUk3ZC5U41qPdMevjOV/3SmqmPfC7L508WevQH+y9TI7rGf9K5bBYSt06QNxe6sZKSNxhKrj13sQwcBUw+tFlYfzR85AQYdHHUtRNqZ+E0HCXqRUU3x+Zj3ouGjExx76vkmvsw1ZbiBPmzuwh6+e48s//YUtzdqxExkz1mYJeDeCJrWXal+q+AN6BZiHgY6F3EdEPEJHxn6cBeNOmex0DTStY7mLmW3ZiAKcKPf6Wj2B5cYw7Dg+Di2EzGlqHewrJw3heem/8f91L8PPZyVUifauUZ0nQZpXp62/97iYGxiOLauhwk5quLjSYlfJytUiMD9rLfvUxupGLJ7jH1JIoLIqVMtSpAhCSYmW9L69e2PC7WCkDClDLzYRuh4o0cy7ktLjCBsanCMLSZ5vr/nJsyXuxZd1SYLwkfYOGh5YxOryI0eHluqJyKb1oAIn/jZfGwTK4Yd/mz15CUiHCMuOewmGlqudH53il4sa8NfeP71FTwOi9V8umVjBqgaVWbLyutlp8fIooFAhtWh+TBUyc8yL/XSijo8mtnYUOTJbi8j/5u03naCZEs7NYmPk9ANoV478b0o8K/vsp2xzxf4FUrC/85y8BPH8nPFDTxlg+TER/BeDv0Ox5f2LSg08QdRPjXQObWy0vHn4Bv5jfxzO+ybGGU432+rwUZRwxLDify9Ab1IloGsAtRw6mFXiPrR2dmzg2Y7k5SZqaqsdo573EFpDzrha1cDT+YQvrEUs+D8ZJS1tdBogVpLlN2tEw5LAkFI6vhTWBqJy7jwEEzTviJXElhzgLXTpiCgw6VGj28YFiUMLeuoy0myLJDb7l0RfhI+/66lRxO0HoAYVbK6yVwfcSwkrlgotN5pMa9zW2OtsCRu+DUlxkVazHGKjBYZneL7FYpAJ41qkFh7YuTrIEidH5dCEGE+cUtVGd6fEuiUMEyqY+55lE9OHo/w2+7cdGNFVfqmmJmfdsvtVsaFrBsgBgFUCcDszYgaqYJzPFEMxpqNbayPutBc75ij2XBrTU/jxBZ0+OhXvtwcodKxgvFvjO2z+5Q1dw7PSWS64BINBeDbAPo9+xywIQjdQW4n7pRUy26UKJhUQdMI6FkEKM1QWj7jRFMSnp9nF+RUKEzJcXiYXNylhKliTqoklFiBSOoW26BNFkA6N3lnzCYzPIXLcb5pCfoYFlhR2HMWZShkSqayuSrGaS46WxlISPkkGX71gJsGN19924//JNq2ur8NC5GVqOgvTN7VQAxDDu+h7VQiZervupJRUjBJuxsSbqLzcULNBhYZGQKA4akNc5VsSYBvEpiSpaWx/jSqRpmDGEfD5DNSp2HmLcINpKguQhZn7wTo5mGvL1wS5GxPtPSNl8f+JnbbSeiH6JmV86myGdvPTkOz6F15x5BYZWXBLPPrLxy/0boy/ieem9G26e+VRQPip0LDPKlRLjpQLjxR0sPbFNskUdtBXttKnVjkYVev0cw0HRcF2pNSKCYK1EXq9SdM3EuAF/BZrCSbeNGZaOMyEAY9uIA7VjA0BtqeQ8eSx1k6km/DVGLlmLBnIpyZPQkpctY7w4RuYLmobGV77svBZ2BKTBVmehI3GelRJZNw1dO2ML4NUHrsAzD0/2TsucKrwbPs+oZvpxjlFb0AtqrP6v28fbiUBvKgLiekOkFLAHaZhoP7lHlBBWRhbzqcFiaZFHcyruSZlrg1q4oIhaGDiHNPHFRnODzt4OumfMTZyLHaWdR4VtuS/VRkREfwrgagA3A6HY244YCLMCYW9a1v50In0PNEi6GSUE/G/+Cv43f8Vr+rUrQfMAVm5fkdplzHj9OVft4Oi3ToqGU7SQkmrE6tKyhUXmGYnW91Kfuy5ra7O6XCkWApPiM+pajLfRc8VdOlWgaUxA4wNNbbx2v7WD8c1EymZSpVLsilGrRTPwpU971UCLaT0tRa3pNnp8Te7U2M94aSwtmj2o4MC+bri2jUhRdypwm4KCQr5KPE+xJdMGX8QuTxFWzXWxxRPvH2+nx3Z+rhOSkkq5oWCtxBULYtK4lua7tIPh0rXS4b7XH+8KU1KEcprPMdIbUfek/xFM2ZdqA/oPzPxgZv4RZn6W//zoNo85kWYlWE7hyMHWKA5QTkO/V30Zv1d9Ofx/8fALAaWjdbPyLEHpxFWjL/FfnX3ltsY5i/1v2Hc53n7VQ+AKi3w+w3wnbQhF9cerZjv0cYxRcFvVQkfnLNaWJ7lcgBqaGu8XUwx3bW8XCyb1+wuqiSOBVJ8z5mGGqCE4AATIawwCACKEmM/B0P8xYkl7xyjF2eZavFEqPEhJlPHSGLZ0HiRQB/+TrK4YEDPrjQL5ceC8DSXWa4/vT/PTdG3peWNrpz2PahXFQmqSAFSrsp9KgmvPC5JQsy+4u/w4IwCFJr3GDcWC9Tfa2f7265Ix0302ISL6CwDvB3AZEd3qe1G9DMDjiejzAB7v/2+H3k9E22MMU9KsakZMx2VbREQXAvhzAOdCTLMbmPmVRLQfkrhzMYCvAHgaMx+ZzVC3R88+8hm8cuGybRWjfOHg83jdWVfijDO6Ps/BhYKNeZagKKVG0+vOkmfgGQenKyPzmjOvOOZxvWLPpSgc44WDz+MP9l6GszoJckOhLpfJhRHMR3BeZdi9BB55RA1UUNuN0mZscSKkUB0gBpQhUoPJxVp4nEHeDCpTYKgxgqmthW80V6GrpI7M1r3XNa+lboPtgMQ3CPO910OfkXA8GwRLMShCgcrxcBw6LSrUWas2A0CSA2XJSHspipUS86nBPYXFYlmXxlkz9gD9rq9fL6WXGCyWrmGdxPOj8S6ZWg7zHQuROsi/Ft2n1orGztpzrLEvrfWWxy0LfEKpzLO3qLJkTWY+ACTewkmyBOOlMa75u7euey93jMiAZlTShZmfvs6qLfWl2oT+DCJc7oSAsEhOzVfP8BwAZidYjpXLVgB+jpk/SkR7AHyEiN4G4JkQ/PbLiOiFEPz2L85mqNsn0YaPHeH1gcd/K84+u+8T7Qxqd+ex0w37Lo9cU8c+tpf17ytlW7xwGHnmtRL5+VVQxAJDqJnVHQsBRSDFgiOmOIAfHzO2ZPTcEoQmJFQLjRrN1MzSn3yeZjBa50rrXxkrLhvywiN2hWmcJRQ79mX8Yzis1r7KfI94Z6VjovZw166Kat046+CGzje+csEVp3kw5UqJbD4LCaAq3Ne7x3FpHL3GvVkigXIjgqlwTReX3Eupaqz7aGwrFuIx2ELLv8T5Se3E1ub9VOu1tr5UOIjQqOdYkXcmIViHRumXWqBzo/zOcSfCqVbd+E8B/DCAT2IWTGcDmtWs/M2x7MTMdzDzR/3vZQC3ALgAs8dvz5Ril8srF6aLs8QUa6TVsApZzQACnFeFw1YskFj7BKZ3h71ijyTgqQ8+1vhledPHHpMGfCetn8Rk6sB67BKrrZi129bbxIKsLaBiKLOeI84Ib1Ps8gHQiM+Urr4X7VpVTvt/uDq5L3blKJJMNW7nWxqzd3WJ+8uF3iRqpagVozk0ahlpIcxypQzXIZUOzLqlXmL3oyLBLDO+586bcZd3GSUkAqouBCr3fbFsJkDGVl/sUlssm/Gztuur7TJT1+QkxUKFiForSVYLG+kCmTSESlyPbenWZaweWp04DztNBAIlyVSfk4S+xsxv9Fn3X9XPTpxo2pIuLyeiBSLKiOgdRHSIiH5I1zPzf9/uQIjoYgDXAPgAWvhtABPx20T0bCL6MBF9+ODBg9sdwpZoK3GWNtlSeoKHFqs+lyKufzQpsL0e3bDvcsynNdIszhnYjF6x51L0EmFSwmgoYkZNphIvawsMjWHoOWPX1Yr3pev/Wqg0YyVtql1lcR2yZiwlzCmv73ar3T7x9s2Yi/XCvV1RIBYukwoiaoKfJl/GXSe1aZgKHECqJ5eDErZwKAfS8KwclBgdGaEc1cH8OPFSn4m2kjGfGrzmzLVlXn740C3++ayv485RnWyoEPHYHRYH5tv3PiY5ZvP51+PpHOs4Y3dcjNSrz+OFsU84DdWQvUDROdffSSatqLWE/3hpjOGRUWiJfdxJUWGnTqOvzxDR64jo6b5M1/cS0ffuxImmtViewMxLAL4TwK0ALoW0uJwJEdE8pMDlz/rzTEXMfINHOTz4rLPOmtVwNiXRBKWPyGJp8eIt1AP7zI8/BQCkxHtukM9nsOwT9RTH75Fj0wiHvz33qgBf1rHFWqTGadaj2L+uGnDs0ojRQjGyKg6Gtxl323rQZU2EUtPFJds0k+hixqdjXVtDbP2A8yQNumai1LrOdmVlalgxQN3XHair8wJ1gFkZpFb1Vas07aZYPTxEsVJ3TbSlxfDICMWgbBS71JbNo5EIqtGoCgpIZ0+Ozh7x6a/4ys4AJtYQW/H9ciTDfrLlK7XEmrktChtWS0WQfXo/6zlXRSa+p+373qwhhsY9zIwJxUKDlWbquW20Qo4EO/vOoMWgxPDIKDSMOzFISjrVBEsPElt5AoDv8p/v3IkTTRtj0eTnJwH4C2a+hzaBPE5LRJRBhMpro0SdmeK3Z02/NvwCXn3gCqxUbkuOylue9WQUgxLFSildD73VEr/0lBASJy+4ug42o3YQXFE66yFzYnrByufwyoXL1uQz5KY+ro5PhYoeP2ZKi6XDfFrDTOvSIS4kNLbHXI+V1zCnWrjUSXhxYD6OlcSxhPr4TeuonbzXhMzWyzNjgissobrv+5pgPSQbPy6hz652XSnDFCtG3E/loEA5KHxfGC9gvOtM22XH4xF0HcOyQy9JMF4uwpy2c0ted9aVDZDH85c/FwAZP7PULLev8bNeYvyx6jnRe2UZPt4SPxe1hbI/TxpCQ7P92/Ov+6jAUkHkmJF5RJfzfW5UUCvFxSmlO2UBIPfCW1peA9K99ETRqVQ2f7N8xFnStLPyD0T0GQAPBvAO35p4tN2Tk0inGwHcwsyviFbNGr89c3rm4VuwP5eGV/3E4Lfm7rvutl963jPwb49+FIpBiZXbV6J6VgIt1cKJ6krJs+lelFcfuAKFY3QTM5HZAvWyzSyXewobGJbs14Siarn15xz9TNCG6/hHjRCbT01kbTWhxhp0jl1ZOtY44C/L6++2C6veb7IW3ra42smB7XWxxeSC243CfYnL7LdJ2uiaYKmoMNEA/ujICOOlsXeROQy84GHrMBwUWBlXKMcWI/9frzHW9ofWYXFUNRB3OuZeUlsNrz5wRQOC/Pzlz+EFK59rjpfj34zYjdWOkVluZuvXc9ec97WoMdlWx9b+6NyqNaJwbp3nJEuQdZt1xGLhw5ZDT6BjdUfPhIiAJJvucxIQEV3qQxmf8v+vJqJf3YlzTSVYmPmFAK4D8GBmLiHlXb57Bud/OASl8Fgi+pj/PAmzx2/vCOUGyHxpk81iIbawWPr6kpT4zpOAJlKKq/Jq5d6EhHG88+qHrjneqw/U0OIjRRVpjvXLq783EypajTkmTW4cWifZ0Z7p3Lj/cvQSg/15ssYldU9hcdB3IlRBE8daFsu61W2MJooZU9v91hQwTQYWM7tJwIJ4eVtQxRbTJNLyL5pTEcrkRxngmh1fDoo6GbLyPV/8dqXzpeZXy4CwO3xkJO2uvZDQpFPLCNvEyYbtOIWOOQ7ex0z2hn2X443n3Q837r+8UTBVhU4s4GNLsLZcajdZswZYs1Noc/laga4UC/Z4XkOxzqiYp5ZziYER7eTVYrX0la/lmEulPUFlkAggM93n5KD/A+CXAJQAwMyfAPADO3GiqVxhRDQH6Zd8EYBnAzgfwGUA/nE7J2fmf8X6UOVZ4rd3hA6OLXqJwZHSrmFqAPDe6x6O+XP6EpQdVaDEIOumE9vaKnSyjEqQAMBZ586vOa7WGoshum3kVs2oaVOhpxQz4TpIi9DvJIy55XqKhVltxYjmv1K5KHaz1jqJczFUMDZzI5oTG8ddJrlg4vPE/3WO4njDZm5CtnWL3izqMlrXD5P1bmyRz2Ww4zIwzhBLYZ2vGqmW+ETMOJ4U3yNxc7mgWKgrUFo219DgZgypnru4BM7QMl65cFlwh+m59J7Gc6PlV2LhoS6s+H7EFQ+AGOnXBG3EgkbnPRQCZYaxUd8aAOS/5QA26lZZuwnZcqhRp+fe7D7uJPHJIzSmoTlm/mArjFGtt/F2aNoYy6sAfATAw/z/WyEQ420JllOdXjj4PH5r7r4TGeA7r34osn4WgozaLlcpzn0AEBhVG5VUDMpQ3FCpdns1S8bHAVJlFPH6N553Pzz5jk9hEr14+AW8Ys+l4TriIC7QjE3EjEuZUOzqkO3JF5+stePYMomPOYnaLpk6uN4WlM0qy/E8tLedVKZEl8X1zZTiysdxjAXwJVyKKlybHRQB1OHGzT7sahXE4ANtCRDfMx1XLQDrY2gXyJjiuayTUmtB0b7+SbXtNPCvve71/rat0fa1xGjB9jgSIvRTg7SXYmWlrn+nY4qFizp9Q+UCb+nVTbsQyrwkWQILi36ahERLPe4JIcLJZI1MQ4eI6BL4hHYieiqAOyZtSETTVPO8h5mfOWnFtILlEmb+fiJ6OgAw85BmFb0/xelXVj+PX+5cgszPxuvOuhKLpcNZHXEV7bvXQshHyPqC6KmGVY0oCslyURmL6EUZDgp0WxnWezPTYrTq9onrRBH6vRSDYRVKxwB1u9pJ1XFjv3gcmI2TEjW5rZ3JXict1jGZ+bReL24xbgie+Bgxw4qtkvb42vuqu60pKCYLE/nfFCpxAzFldjHEOQibisFWti0rh86eBCPrwvFU2KoFFrsl4xwcvba4xI2e657Chnug1a9VadF7ElsGtbZeWxs1NJvCs6L3PSFqFK5UqLI8drX1IWOk1j3W9fJPn3GlWAnRYPpwUETCsL7WzJgwr6NRBctAz0rRScC7ybwb0Y0ZhsS9qO9H2kvhVktYZuzLZ5XjfSxEEmc5degnAdwA4HIiug3AlyHtiSfRFQB+fINjEYA/XG/ltHelIKIeakl3CaK+LN/olBBQMoeERI03nNURvD3yBFk/F9+xz66WwnkMQHIWkjxBNawaL7USJQbvvPqheOwnPgigibZRUkajQXbLEgwNjLRqxoEmVWeOfeNNy6npSpJt18Y0aotBNlqpOPRQj4WGWi5bcdPpceNAc+1uaSOSaOI+9bJawKzpTBm5q9azqEwkDONCnELtMjVNhFRu6lbB9bwAAHtlhBrzmBmDhFlgxstFQApu1AJarqteH9d302TY5y9/Lgil+F7EykHT8qufCUmiZFw0lzWEnVoSSjo/8T3OjMHIN8BToVc4Rl4R2MaCXa5jaB325wnIMrioGmV2ABEy7ZYNx5VOIVQYpHzL44ioD8Aw8zIR3XudbX+Fmf9lo4MR0a+vt27aWXkRgLcAuJCIXgtpk/kLU+572tNvjL6I+y10ADSr6X5ttRI3mEcCxfh8zYOQ5Lk6Ia7JuD0T7W+OKonRQiF47jXgGLUVn6NN7XpcbYhuzMhj5qPxmLbbbDFC7sRos2bL2rqse7xN7G5bz4de16uqxxf7/uPv+nfzuDEjdFwzWZ3TmHRbx9LqYBoqHIe6XvG9bebbyHcvMegmRpqNESHPEsl38ggqQzXQYRhZSzH0WOfRRYgvdW+pQKqTVSPLuBXrmjROHask0ZpomZx/qZScktI5DCob8rz0mJY5CI76GXHe8nTBPaiB+Rr+7FCUNgAhbOHCs6Vxyc0AKjtBTAQ26VSfk4TeAADMPPCVTgBgYkloZv7r9jLfznhho22Upu3H8jYi+iiA/wBRX3+GmQ9Ns+83Ch0c24YLSV82k0gL1tJDSbWBETvfCtcLoq7vZlgHamsGX0SNoYDaPVIjepo+enGn+AZSRLDsApOwTEEIxT093nTh1eFF3rung8VlNUjrwPtKZQMTiJm95rzE59d+HPvzJBQ9jAP7Mcggdk/VLp6mtRELiTpWEwsO/d903cRWke6ry2PSZmJtaveCgXMN95tlKYOi9yVWqHWeATSKd8aorswY5FquhOos+zh/Rsv9JLlBOarnMq7fpd9qcRhqui7VMppP5dn423OvChBwzTWqt+Mw5sVSLAYVjNLfJXY16rMhNcBiIEfhXDh27LazXMPSdZ5iMEESWSzxePTaKCHMpzJ3ztb37W/PvQrfc+fNa+7hjtIpEGMhossBXAVgbyvTfgFAd5N9XwfgvwKwkDj7XiJ6BTP/zkb7TVvShQB8O4AHMfM/ApgjorUY2G9g0hcjN/LQ/8zSZ/GClc+hd+Yckgk1nTRnRclFWnUb7QM0+4DEzFlJGZ24SGqrRTs+xi98TK858wq8+eL7gy1jbyYv+z1LtZezjiFw0Hpr4VIzlqHlBjS1l0gmvyKQdHyq8caWQ406qt1ctTXSnJM6ptScn/pYNbPS/5NcYbEVo9vHY1iP4mvUccaKQLutQnv8uZGsc8sIsQY9rwqVugS/5muIcBHEYH3d8RzMp/V9B+R56vpnQc+rsTkdq153/BzpPKxUUltsaBlPvuNT+P67Px2O00sIF/TSYAXXikLzOPcUtlGhQeHrcZ6TUn1/a4ESKwENF6GH46vw1TEdf6JTBW58GSTD/gzUGfffBeCBAH5ik32v9NVQngLgnyDI4B/e7ITT2mh/BKmG+VgALwGwDDGrHjLl/qc9TQqGv/+xj4YrLKphhc5CB6uHhiBfUC/UBauaWnczcC3rFDGmqK4avlmjeZQhtF0+bVeJWhsxQyzHNmh97QS8xFs8mmtRW0hNJqLAgRoKG7vJmkJRqie7xvWu7UBYbz/JFRdTe94mzUG8bZu066Ru2xQWTYDBpPmMGWd7vPNpW/iIS6ub1AHs3FAjvyO09rN1rksct4ndUBojUWYcKx2lc435bVuV+/ME9xQ2shrr7+ccledZvwEE6/b151yFp951c+imCphQETuuzrAWlSfPw9Ay9mb1sjj+Vt/jZsxHE4FV+GoL6VC40is77QoEO02nAtyYmf8ewN8T0XXM/P4t7p756ihPAfAHzFwS0TrO9JqmFSzXMvMDiegmP9AjRDSbRgSnMY2XikYxvdzDjzXpy1YKOa1LsLSDupaBpdUyMPpX7Lm0wfhVIMTB5pi57d0jbW5tA61UbyfadRW9yGvzFeT46l83jXIzscarTFYtn9jdVVslup24rXQssXtFYi6xddS0etqpT/F118FoNJhyPD8xA47ddvX52/GYpmYdB5djIV0LG41lyD7xGOSauNHULYmFCjNs2USWxdcZn7+bGAwq2ziHxiSaQqgGUMi55JhZJwEK2wALrHH9TaCn3iXuph8+dEsArNRChUMi7HxqcHBcNRBxMj6OrBaZX7U49J51uwmKYTPFQucmFjo931WTrYMdbcrvZk+ngGBROgahAgB/DOmJ9XEA7yGibwKwaT3HaWelJKIENSrsLOxwPf/TgTRoT8aE4DxbjwzzzLUd2Jbvte6bhAjdboqzOgnO6iRBKKy3X1hWSNAzdtu0y9xrjkTbBaXMop1Ip0zqmYdvibRcYRRxMuXQcqtMjPrMa0EYC51Y+MQFD1X4xDGEtlWjFM9JvI3+Vquivbw+Vs3sm79r95nur/MnlqPELuJS9OKiqTXq+FutFQ3El04+Ov5mXGGtmychibm00WEaW1FXmwb89ViGautoMKywN0vq2Jp32cZWymakLrLYmtO5V2stDtarK1SvPSY9ho5Nt1d3ajx/+vxTIlB+SowP+h9HlxjRqVaEcsvEzL/PzBcw85NY6KsAvnWz/aa1WH4fwN8COJuIfgvAUwHsSI2Z04Xedc21oUkRUFdp1eSvka/9JJp9zUjijOJGmYzUwORJcH095+hn8Ad7L8NKxdif18UCY3gvAKyMK2ggVWt5xcHs2K1Ta/nAPUUTpRYzDdVMJeAfo4ia26v1EmvsCoeOS4nEFB+jmfAolkpsnUxi/HpuoZqpxdbLSrXWQoktpnauTX1NMSqOWsy/GR/Q5XH+iS7Tc8bHjo8Vk6Kkwv6pAbQETFG3s46VCnWzqWXc7aYT653pXKubzDLj3O7WmeD33/1pvO6sK1tWr8zJWZ0kwKnjuY3npHDStK2O19VtFlQR0Pekp/k93hOgHgDr22Jb5g0TgWdNp4IrbDtERL+2zqqXbLTfpoKFiAwkkeYXIGVWCMBTmPmWDXc8zrT6xc/jPdc+LDDuR33g3070kMQfHKFS1S1W+szsNkMD1vqmY0bjvNtiaBmvO+vKUNZlX57iSFGF7RWBpC4JZSBqqcynTZiuMl4Nzp/VSX3JdRcx5LqGlL86qYsVjbWXmHBNteZdX1sMhY6FSltANOejHmP8X48dM/RmYLspMNtzrPsObc3kYndkbwJ/VVdO7dJTZlm7rWKNPSGEOmuKwNJxxZZJOydH50LnOq4I0E6iNWEOgTyry54Yqmts2cKFbHftkhnnFJnoPrSfx62QwprjeJnOWyyc9bpj0vHUFq8LrSliF6itRJgq8KFcqRGTeh1JavDmi++Pb//Kx4/5WqYjOqXyWIjoHAD/HcD5zPztRHQlgOuY+cYNdhtEv7sQEMCmvH9TwcLMjoj+BzNfB2B6G/l4EzPSbho68r39qofgcTd/6IQNR3umt8tqa1ylDravdd9omZSYSWWWsTSWLPqFuSxkLAPUcKvkpo51iHXA6CUJNLYhpdIZezPCmR0RSOrzloZcYh3dM6zCcZ67+Fn81dlXop9K/4w7j45wZicNwWFllspA6/pWkqewNzOBaaugiV1oMcXH0cz92AqSbdqusMkCpP0/zuOReWnGI1Q4CZOr3SqaYAggQKfrdQ65R3mp8NB8lLj8/pnzWSilj+WiUa6+di/WlgMgjDLtpVIi3jcKg3PIswTGx2PqasyyOu2loSyKkgqakXVI/PzHgfBQXshtjopbjzRg/uoDVwRr4x4/5lhY6nOvcThdXz8X9XPfztWxXBc2bcyTn5okN6BImL7pwqvxHV//BN73iEfCWQd2jO6+DdG1WyPCKRVjAfBqSHmuX/H/PwfgryAV5icSM/+P+D8R/S6k+vyGNO2svJWIvu9kLuPiqtrU15fqTRdejTeed7/jPpZ3P/g6nxHcZJocMbam66a5fxPF5K+JGf008X5lEzTknu+LAdRadFNblDyLgx6qKnBPh2cc/HRIYouD8Lkh3yOmOS5FGWknxKWy3lfHshIJTTlWE8G03nW2YzBxLGaSBRMz5OZ8rnXdKZPSY7Utg5i5affMOM40KRak1sTezKCdKBhbGjUjZXR8Aq0rLLJO0uhbPymelhvC3Jk95P0MnYVOuNbMKyqUNOMlbbJVE85edyatt4/bN2iuzKRcnq3QMw/fEtxQGr/bmyWNxmDtoqZAbQ23rUydizacOX5HVMCWY6mWrF032TLe/9hHrxnjei0Qtk50qsCNlc70SY3inGSuANiNd1lDcwDus9lG08ZYng+gD6AiohFEVjMzL2y823EkZhSDAkmeNMo8GKLjZBbXJD28KZRvIZ/IpUw5DgJLfKW5v/7PTZ3jELTXJYvbjgyRkGds8xl4sS6doi+cxmfi2I0KnZipxvEMbdQFCEO4oGfwjIOfxmvOvCK4uSzbIIhi187ezDS0+fq8ynCbriq9rnY+icyP/mq6yNoWSHPb+ti6PI53xOvqGEAbVt0s86/3Sr91zmvXDLw1VruV9Bylj2nFwIckk7I9zgp0tu0qCu4elp48EjtwYOcw30kxLCwoESSUVm7Qvi/qAjYJwUHcQSEfBvCdSWt3EXkrRZls1klgR7MrdPvkOz7l4y7Nfi31fdDncW3Xz6Ywr+ugxdsEoZ+a4AXIjAnXBcC3L3C+Z46cQ1o/z0qw4GQSGtPQgIgOoAZh/QcAixvtQESfRA2LTACchU3iK8D0mfd7ptnuRBIDoRUshwcrAfv+J8eL3nXNtaDEoFopa42wlBLgABqMvw7exxqcMrqmgBlUFljW2Iy4cgonvdaT3KBnTSMXpZcQ5tMkJKkBdbdIZX7KCIe2buQFAF/3Bf6eu1h3HtRWuG2QgWr69xQunLed9R4Lmfj6JlksavnIsZoVfWM//Xpur9giiQWcMm2tXaXbx2CGuDRN+/gqWBMC5udzFKtlQ6DoGDITWzCM/XmCbjcNFoO4tsTyW/C1faUHi1xvbIGw5fDs5PMZiiMWSfQs20gj6Sx0QoUHkydIk2Y3y2pYAc5Xd/BCRzs06nsipWdmx3SfcVCC+nFMTWNXltEoUhoL2aZiUguZOMdJ5znx3Ti17I3Om7T6TrB6eLXhlraFXVMt/JiJ6GQq1zINPR/ixrqEiN4HERJP3WSfuHVxBeAub+lsSNP2Y3nghMWLAL46zUmOhYjoiQBeCZGSf8LMGzb70h4nxaAMPU4kCfH4CRVtyFWNpHyLg4+xWOl+5yJcftNtEy/TIKYJWpmuUxeTBt8Lxzi0UgSfvr54dTIjB0ullxCuP+PyEEd5xZ5LcUEvhWrIaoVYXgs3TYhC18jacqmr4Db/r+3hEgsCiQFFWvwEVFis4a4NvNffk+IB+QQFUretA/2T0FfN87eFkgq9hKQXy6RETtWY1a3US6Svu7rBtK87JRwsi7o1NSGfk6zBalg1ysiLAKmCFaOkVnl3XzfEF53l6Pk3od8J21rDD2Ou6mrCgFhfzzw825Io8RzFIIH1touRZfU6QTXGFqESJXUcSjPxrWVknSQkT6bdVOryOQdbADNtQTKj6MBW+d2xEDN/lIgeDcnEJwCf9Y0bJ41nwWfcL7dWLRARmPmejc61lcz7BwLQNm3fAkmYOUBE/5WZ3zrlcaYinzPzh5DukbcC+BARvZGZ10+pZW66A2Zp7k5BcZdHDdjrOPTFNQkhswaFq5PaRENsBmDjjpSxNhy7YOKktvi3uncEVtu0MhKSMv8v69dtlJtQ2MnXFufbxExBmDNHcYpaIKxFu9Xfk3zl6gJRoSNuObSsubWWiS6L3SOxQG67wmR+6v+xS07PVwv1Gu0VP05qKTRcMlSXGNHy+pQQsn7eEAbyfKZwWYJqVElla6BhVSe59DIRxUgEke6bdVOkvRTFSgmTSwtfFVzVqEKSJTAJobuvi8HdAwAG1bAKx1OhRYlBktYaPtC0tmZFCsJQFFeSShvneu6b8PK1rk0KuTn6nCu6Ta03SgidhU5oRZH20FAu2TLIl9cXa3FWKXg0E1fYMfG7YzvPTwJ4LTPf7P/vI6KnM/MfTdj8dRBr5SNoFuGD/79hnGVawfIVAD8WDehKAC8A8BsA/h+AmQoWAA8F8AVm/pI/319CWiGvP9EeXukAkPHMypdPOR5CxkXCQwVKuzmUIrlqSG7cMbFOMKyDx4BmTavLRjXpGjEDJBRXpq3dOoAIl4F1WEilnfCN+y/HCwefx/VnXN7wcRdOqvA+f7nZI12pzseouxrK+dQKabr5gCbTkDFxYOxtF2ATGbW2BL9SLEzisdW5LGgJlbWuufZxYqswRrhpflFbiJXOoZcnAW4N1DENk0s7hCRPkHZTqU7sWyLEAibNE9jSgq0IFe2WCIgFkvdz2NIi72ewuQnPUu9MKWZiCwtbiiWsFR30nCKwGAv3Wqjb/5Y2xPhiikEmT/ziTRPv/XapcIzM1BZLnnn3FdeWbhPAUd+/hMi7ekWQ2KJ2Bya5EUGaaxfW+j2gxPhE5Fp4xvM/K5pRHsvW+d2x0U8wc+ih4iuo/ATEcGgQM3+n/773sZxoWsFyuQoVf7JPE9E1zPylHQKKXQDg69H/WwFc296IiJ4NaZWMs7M8+FGDpeBmpZlsTG+/SkqmxcUDAalkrM+69TWZxEpp+/IVXslrAuAqeGI3FNCMwWhsZa/X0uK6Xno+zaJXhv2co58JzZ/Uvz/JRQEokxcGoEgoOX9cO8u1GPTawHpdCiZqCWCagfm2ZTRJg5XlNRpNtllfAMXB/7WB5HocCmNuxklqZqeJh5ZVYagtU4rgu2kvDZaDMkN5Bhzy+Sw8H1lXLBdb1i14++f0xXVjuRELSDIg62fI+7lYOef0kS6Okc/noMRgz/nzWD00hC0sTGLgrEN3XxeusCgG4u1wluWckcJjJgWUZkgqrNVaaZyzqAE2gMR/NAambkURCAZp7u/TfN1CYk1HVhgkmQE7BhlCkhmYxKAYFCJcPRBipjS9YDmTiD4c/b+BmW/wv6fidzMgQ0TEzBq8TwBsWJqLiB4O4GPMPCCiH4J4rv4nM39to/2mFSyfJaLrAfyl///9AD5HRB00UgBnRpOe9DVcz9+YGwDgsrk+x+avcg8yBsnsLfyJFAsU/a+97AGE3AbV2idlewOxRdBsE1tbCfV+cgwKfcD3ZomvYttMgktIOhSe201x/RmXN+IoMh5h+L8zfylesFJbLa8+cAX25wkSknyPxTKOs9Tjmk9NyFvITRK5N+ptarRPM0dHr78dU2q7seJrV+sk3qe5Xezqas6vrheqXVm6b6jlFllgbeSSsxxyRmKhAtRWq2aEW49MAgBn08DUk34GdgxnYwFSC6IkEQuEjMQRskjQ6G91jZGhhiCRVr4G3TN66IxKLN+xIq6wSPnp5AnKQTG7YPYEeubhW/D6c64SYeytBn1PKCFkiQnviQqeBLWSpjclgA0iN7MeQ6ldRTzJJSaV2hS2dDAQq2ZW18tE4OkV60PM/OB11k3F72ZA/wzgr4nof/vj/1dIn62N6HoA9yei+0OS5G8E8BoAj95op2ln+JkA/huAn4VMwr8C+HmIUPnWKY+xFboVwIXR/3sBuH3DPbzfVd0JJmEgSXbE/I3p7Vc9JECKATRcbypURqM68Jpw7JqJk/04lJrXYHlu6mdrsaxrdvUSgfbmRlFNAFAH1TMjBRDnU/h2twlyA9w5atbJkqC+AgXQsGhiGloX8i50mzppUASC9tewLEUI63E1XbNitcg/RWolVFtZMi+1cJlEat3F1ko7SbMZx6m3UWHUPI+Og8J90/Xx41M67UnimbMXANKsjYMrDPBKRqRo6LbVsApatwJOYnepCiAVLp29nRBPsKUDfIFKSgzyfhaed3ZyfvbxlZicdegs5KGskLb5LUcVeF8X+7/5QDjOTpAWrQSAN553vwaAIIYGx//1GnWdLtf5YUchjsLWAUkCo9fgw1UKWjCJQTWSemIzFaLMsNuoVBDR1vndsdEvAvgvAJ4DeTHfCuBPNtmnYmYmou8G8EpmvpGIfmSzE00LNx4S0R8B+Edm/mxr9co0x9gifQjAfX3bzNsA/ACAZ2y0AxEh7+chwxaR35USg/c/9tG47p3/MtNBqgvMWUZianPcwAUoJ2DQjR5mts0qvbHwqIPfTW1ZSeMzMQQztho0e34+rSHHEp+R45zVaXZzfO7iZ3Hj/ssDKklLmb/+HGkC9YyDnw4Z0pYZ9xQO82nNuNXiGlqpV7Y/Nzjoy9UoaOBXVj8PAPiDvZeFcWuMJb4uQZE1LZJJOkH7mnUemgi0teihSe4ynXM9lh5Xrcp2mZj6WKJ968K4K6hB/TwoOcuN4Hwcj2v/j5WSaiRBd/3vbJ30KNq5uM+srRFglJiQRwUArqzAjhvvhjLX1KPIdlKotKlwjK5/N9SFaL3Vp8I0FrRtSzAE3v1cJa0b7aI4i4muS5BhjM7eToBhz4JmZFZsmd8dCzGzg1gg129ht2Ui+iUAPwTgUd59tmlL22nhxk8G8DsQf9y9iegBAF7CzE/ewgCnJmauiOi5ENMtAfCncYxn4hiNL2dhHVzhQMbBJIJAyaZo7XuspBoU0NS24jwBSpJgvVBCSELpjHa/8Roptjeru/ZpZVdl8Au+Iu2ibwWrnf5Ug1fmrlYMgAAA0H4Z6g5TAZFQEyIMiHaplsGK7xuzUjHmU24wXn23JalSjnHbsGowY82Jec2ZV0QC1M8hx9WM40rCsixOWtRlMYCg7TLTOZ1U/LGNbIv30fUyB00k3BpYdAvXHMODg2tHtfBCguy6jQoAACG4Lhea1PlOpUWn10E5KKOsfR+TyA2c49DSWoSJXpMImiRPgkav3yYxQCJCKMlSUGJQDgrc9bE7MH/+PI4HxdYLALzlkmtCML4pUNZaNTEFD4GdLihvEgPK6xjYLIgBzMJgORZ+dyxERPcF8FIAVyLqHMnMGyG8vh8i5H6Mme8koosgsmBDmlZVeREEuXDUD+RjAC6ect9jImb+J2a+lJkvYebf2nQHUl+sgclN+O4siIZSDSt8+EnfNvtxtuIqqlXZwjU0o9hNV/v0m0Fs7f7YSyRQHFeCjUtiJLnkrWgQvS6j0uwDf2eoJyZkuS4/DgA37Ls8INL02Pvy1B9bYjJapkVqYNWd+uRYUiZeQQOA9msx+Ob5HPc/aw7/eP63NObrhw/d0ohZ6HDqvjLNnIZaQLSTH7GG2SvpnDSh0Ru71mpkXm35THILxjwsuKF8YLh268h8xNDWNgRem7fFVa/jGl+azEeGGmgu0fS9UCqtCJhybW0wyfCXRExnnd/GNTT6+Hz3/+s3T56cHaaNkGjNnJ3ajacCO8SqgvtR51re/9BMDwi/JyHjtkPMPNVniuNsjd8dG70KYq1UkBDGn0PiJRuN605mfgUzv9f//xoz//lmJ5pWsFTMvDjltieM9MVRLUYfrnw+Dw2BZkmC5qmryCql3RRpN6196VGwUgsCxoxVmWScOV86h4W5LEJf1bWsRqMKIx/3qCsP10JI3U2C4KKGxq/uqLaSlxnfzyL3c+ZBAIrk0hiIBPKb1oBSNzHI5zL0eyn6PsN8En3/3Z9uuPOAunVubDmoVdLO5dH1TVh2LWgUzt20UmJodxNIoHPTdnvFx20CBCi4npzlugho5UJWfTWsvDUSu2Y0z0UEUTkoGrGDslVSJZ4/ZZBaZFXWWxhDMIbW7N8QNE6YcjWqPGLKwJYV8oUe5s/bi8uedmK7jD/xizchyaUthAAh1lqDk6wSase01qG62KZXOCcoDMdCarFM8zlJqMfM7wBAzPxVZn4xpCvwGiKiZSJamvBZJqKlzU40rWD5FBE9A0BCRPclov8F4MTXpV+HNNiZZPIQpV0x+2dlAk86n5rmbWQY0HSbaKKaBNjr4npNlJV/EfKk0ctcmalmwSspU64TDGskU26kBa2eq58mod6VxmASIl9WRJhZ4d17mTGhEZRYJs2Ci6rpx5p9NaykN3shbr9BZfGaM69YM2fPOPjpcL3ndhNc0Ev9tda5MXpt8SemOEO7FhTN9XGwXkvoNNc1jxNbRvW1NgP+UpLehaKHQF3a3nEzIVYZpVqwsetU0WL6rKjLKmaS2k0SqBlkOaowOjJCOSgxXioajLgYlEHAjJcKlB5qHAu4GExgsgxkDA7/wQvat+i40uNv+QiSNZUFmhzZRO+Zzq3Gk2I3mlou8f7KD2YK5mFEz9DGn5OERr4NyueJ6LlE9D0Azp60ITPvYeaFCZ8909SInFaw/BSAqwCMAfwFpDXlz06573GjxGci60MEIDx8ST5jRIgn1UKVYoGi+QJxQFb3SXKDPEsmaM61Bl0NBU0WV9tVJl9XE/bXTgr7pUaNrYQQSoVYZox8PkzcClYF1aCyGFQ2HJOSOvbT76Xop0k4bpvJx/81TmOSZsZ0m/ZmkrQZ76MCL76mdrJk2/KIXXSxpdJ+oetkvGasJX752y40Ffi1O64JmRWk2GTOERceVcZeDStBYzkRvDWybK1CItaNC3GY+PnR47etnDohsIa8a3AcEOGlsb5qVIDtbF1D26HH3fyhYJ3oR98fFSoNBGacEGkooOzCshDTEuCDczwza0VpVq6w40Q/C6lO/NMAHgQJyP/IZjsR0SOI6Fn+95keZLAhTSVYmHmVmX+FmR/CzA/2v0fT7Hu8iAjI+1kQLm2TWmGHn/7P3zWzcyZZIrWf/APfGI8xazQuGUcduDU+lgIgxDA0ES+PtLDYzQXUbiANuOsyXV6XZJdli8vj4MoqnOQL6Lb78wTn9zKcvb+Hcy7ai3Mu2oun3nUzVsYVCt8orOc17pGvkRUXllQGPbQuNBvrRQioro8bxbGWN198f7z54vuH8WiVZc2DifNc4usPMSZqVhLW4zRjMM1y6/X65rH0fLFwsFwDJZTiPBtbiVU3SaDE1YPXq6IbB/hjUqbZfpaqUYXVQ6sofM6JPHcUlJpqVAWXGAAYQ3BOwCImkTIumm8Tu5tcYeHKCqt33gOTJFi88eRqCqtxFQAh4VkToHV9ENz6PEbPngpjvRdsGc7JZybjg1SOnuZzIomINI7yMGZeYeZbmflZzPx9zPzvm+z7IghM+Zf8ohzA/93snBuq8ET0D9gAUbdTqLBjITJ1lrP2ZGj7XePA5SwoftAdmtoUgGAhKfw47Bdpk8Ba7V+r0YZgcIQWk0XNpD51f4VyKHMZ7KDw+1Ij56OXaOdHAJB+LrHFlXVTvPPqh2L/gR7uPLgahFS7LH9MyrDn02aw1FmJFXUjAf/Plz4wlDzZu6eDlZWiUWFAM+CV4nhOO84xCZrcLOfStGB0rmNBUlsvkxMy42tcz/KKj6+dGmOGprBzjckos2/DjQGseYZi1JMmYtaQ4yQE6CVb33l3j0GSrA1uNyxr65B0xZItlgcw+XkNC+BE0Duvfqi8T2jmguk1O/h3CaYRd1GhQoaCAtmmYAEZwizh1SePMbIhPYiIvgnAjxLRn6OVkLlJQcnvAXANgI/6bW8nok2r3W/mG/pd//29AM5FLameDqkfdlKRYvLTrpR1UCEj2qELsM2doCQ3vnJqM9i4UcE7rXDbzdPQrlj9922NV4WL5qdokL7W0r0mnkm2dg8IcQAt12IZ6Hbrc+U+o9skkmzmCgurTMwY7O9ltQY4KNDr51hZKdDvpaHuWbO2E6EoLfKQuMaN32+68Opwvhgxp8LLslpd8P/1u64srAKxfe0qKOq5aSZM6vWu57aaCCkOkPC16+r1sq8WWZxEcR0woHaRArWrdlqW7ooa2eWsQOq17D0gQiS4jiJBIhDjGs4MAKYrYyoHI7iiOq75LOtRu7ZfOyaiFp2DQeLjLO2afJPiNHHQfpYC9CQKzG9E/xuSYX8fSFHJeFIZGxeULHyCJAMAEfWnOeGGgoWZ/8Uf7DeY+VHRqn8govdMc4LjRbF/VS0XFTLjpQLZfD7zvizxQ27QDNLXL2kNN62KuiaUbqPaWdyYSTXOsJ31XR2zuud3roKrcjDUDErGaKO4VHnXu0I6iqSJmIwCG8ZL49C/QoWKxnqqobRGHkV+/di1lKQGCWotM259SxHzS3spTC7jywCUKyVKV+fhtOM1cZwmtiy0y2UctK8RYXH7W26AFdbmsdTCTK9FrZpYeMTCReuJuWh57AbT63bQLPx6rtkPWi0QLXUfWytxyRK2HPJgtGmVauXGD1xjLbErSIWWjXJjtGijNJ+TMidkDIrlAdK5Lg79/s/hzJ9udKM9LhRXB2+XcwGa7qQki91drnHNck0mzEtsQSvNSoAy42SKn6xLzPz7AH6fiK5n5udscfe/JqI/BnCGL1j5owD+z2Y7TRvNPouI7hNV37w3pEnMyUNUv8xpNwclFYAydJQEhJGq+T8LetzNHwolXVS4TOqXEYKOSQ0oCMP2QqS9j66jhIDChpfMJMmaY6jbTyvVsq2RXYpisixM3BUWnYUOTJ5geGhVEvm60s9C+3kACLDZtuabpAbwjbiAuG7Z5JgCx5IAkXvGF180CaGzN4c7Om4crxZYaPxvI8Hikv1tfpQZaVHQLO+yVqhMorYAarvBJlknkxQLg1qJUFcYJc2ilQEA0uodNJEpRkIlWLXRbipEHAycLx/TtJYcyJJ34Rqv3ADDw0vYM9dtuOGOJ4VrtXVpJIFFe2EYKWjx9kptRFlMs4QYt+kkQnxtSB4N9sit7sfMv0tEj4cAti4D8GvM/LbN9ptWsDwPwLuJ6Ev+/8XwVYVPJopLWSSZL1vRzUNdJABrAqOzImGSadDWnW88pOesS7zU1O5JrhaEbif/TcO33q6pFKrlauHC3KAauoYbqZvUiZUmF41VUXO2sHDdFIgCpXGw00ZjdMwwUZ2vujIxBXCAurWMra9DLKD6upV5xcIln8u8RSSaf5IblGMbBKMy8jiIHncm1CoGMfy5VBgvrYV96q2Iqwe04zNNl9v6jCnu+khJW1NOomdBLNT1NOZ2+ff1tmsLHHWJUSJBe62sLOu4UZxRhUlzrAbjI8vYc+HZSHs75y5ej9794OsaVaCB5jOi72yw5uLeNVkszEXoxqX19RhArYC1qyYcKzEmKxgnIzGzI6KPE9FFm1UmBgAi+mYA5zDz+7wgeZtf/igiuoSZv7jR/tPWCnuLLwdwuV/0GWYeT7Pv8SKiWpurRgWyfhcJUpSDkeSNdKXcC4rZnjcWVOKmMIGDGdT1w2DtmmBtuw6S1jcLL1MrsK4cj4xB3hWklqy3QFTCA0AImCtkc26h04Cm1tqqCy2dY3hme3ya2FlG16sxDKBm0E++41Nh/ZsuvFqulxnOx3W6+7qNuFNdkZqRzWehY2LblaeFC3UcjmPAAgG+k2VUSioIhNxM1izbKLJJy6NpbxRPbM+TxvJiwRHunb+PDiJodF0bPivHnLA/EBQm07B0pEx8DKOlpM5MD25Xz3zTXuZ7lLg152LnMLjtEM669mqM/ul6dJ+0VY/J9kjjo1qaphE3SZLGO6MKJDsOrYnrFsv+uYnmI85fo5brbLt0aoiVQOcBuJmIPghgoAvXAWH9TwC/PGH5ql+3Ibx2M1TYA5lZ0QBjSNfIdbc5oUTiXzZZiiQXuLHx9ZCqUYFqVKB7xhzue/3rZ37qOF9FTXbRjLKAUGNXCwX1t7cx+rG57yDl/rUSbTNpzoZcmFqrrY8pL5MwILUKdB8yAjMtUTOWtZaUWxMQdf7YKFRLq62GtjWgFLTQqhYSk5LetE+ILS3Yj11a6ta5DMGVhFq46DEBGZNCiBXBFsZBtMaFVltc07GGuOx7TJPcVcK4TZiz9v2NIbMuQvzHVm44/gSB0v6vSKf4fiW5QJItJP4YtHbtumjUEjaSJJkYlKtDsLXg5aNTzcmsKEZwhvG1Auw6S5SYBgJU9jEN92DiIdWN46owirwas6BTJHiv9Otb2PZiZv5EeyEzf5iILt5s580sllcR0WOAif0ClG6EwNFOKBEktpJ0c1CS+MB9DudzK8gYZAtTARqOiTTgrfQfP/dRvOfahwVkWtyALLYsDJqQZY1pGCAEbGNoKgDYsq6km+QE47P5gVqLTrt13w9brh0f0HQpxNaUswxbNRm7rqOEgKpdLkWY9DMONhvetbXI9c6lwjMmPZeWoo+TA5XlqFvOsfQ4V0uop7W0WgH3Oj8mbgXddn+tJT1GHBMBImEbjXmNQA6KxdrAfPt642zydceSJ3VNLFML3EmkJUzibH+gdinVioUJsY3hHXfBJAnm1p+OHaG2EEBSd4uUZ6RWwkxiGrBitdzC9WVJULpiAaVFOGdJp4gnDEANxpqSNqqB1dts580Ey16shae16eBmJzkupC+Zc6AkgfGJi9VojKTbQdrvhbjLTpCie+KyE7LcAHABGomgZSUNrZxtDREVd0bNfKVqM4cquXkihS61DS5QI7HKUeVLhDuYXi77RH4dW1okeYasm6L0uS5thpj2UsDDgeM4S1ubtsz4/rs/jdefc1UjV0UpWCHWhBIvj/3EBwHULQd0jorBOJxDYklNDqCxI0rE4tNcH6AWMI0cj9TAWA55JSJTdfzNFsWTkF/q9msjvmIrM+RWtASOUoi3tYSNzrlakZrvFAuV2FLRa2sim+p8jHZcJ1grPvAdV5zQoL2LKyyPxjCZgFoGtx2CyVMMXvQTuNevbwr+mQlpPk4SZc+rVQLU1kYWBejb8Un5broBATSvPakRdrMgBsOdQs4wIlpG7b3LIXiewTolWj5ERD/BzI2HgIh+DCITNqTN4MYXTzXik4TiFy3p5mDnkC/04YoKSXfDDpzHRO+6RrqHxkFoQNBiMTWYZPSyJN089CKPX46Y1jAkv63mgwBo7B9vr3GbtJeiWCnBTnJ58n4GW1hk/dwDBKwv/eFqKyFiXHVZEu/LTkVAanylXQpdiRIC2TrbOyZ2LiB5VPhVwwpZP0dZFLV26q0VF83NGkbeYriouBamUTxHYzOGYsFCjeC7bge0WhT7ZlOxZdEAJCR1PxRRGPT3+mXdNX+oPTdt5Ff8Lfs1UU7tnBiZXw7CSTX/9RBfrqiCYHFOnHNZv4u7Xv5TOOcX/tfEfWZJcVViJDUYwcEFIQjUbTFEOESABEOwpUO+gcDQd3DmtcJOdFr9FoiZG4mNRPQUSNX6SfSzAP6WiH4QtSB5MEQgfc9m5zrxGVEzIw4vNACYPEXW7yLrdzF33n50Dyyge2DT2mlT03uufRiAJnqFjMHjb2kK81gDVaaUZKbhDovLz9QQZQqfxvHypC6oZygKfkveSeqtBNXU2LpQ/oOM7wnutb241W3DP6+5Di0tuXQS+EzyBAkB33Pnxi0jvvWmD0hNtH4WKj6HazYTXDQT0Epr6qv1s6BxKsItFlxJniDr1JacVms2JNeSdTxAguqclpCVHbm82taBHjsWuMGN1Br3JEGqJUn0dzzPG5GORc7dFDZaskWXrWftxEgwZdQmet70GthauKKUmOSwgMlSHLn+hRuOb7v03use7rtfNmMk9bgpPHPxnOgy3a9pmZigHOmxdyL5k6G5LJt/TkZi5r/DOtWNmfkuZn4YJC7zFf/5dWa+jpnv3OzYO+cb2oSI6HcgyIICwBcBPIuZj/p1vwTgxyAR6Z9m5n+e4ojhJTHe3DcBctxB0s2R7p9N6s37HiFw8Lh0zEZ+cWcZuX/wYxRLI27g3T91dWT2ZToY1tUw5Cy4Ter9gxDxKJkcQGehE2opxS1w2+UuVIBUo0qYcKTVxtfmLKPbTQMgYH5KyOa33vQBvO8Rj0SZrO3a5yyjGJTBXZN2Eawmtcj0P1ALozgXJjDsiMGyb3TVhnCr4HV2jMS7xYyNmnJ5C0cz2eOkzlhotN0wsVUSx18mUVwGSPZ1jfseU9stuBY4UEPP05aAIrPW0lGKnwG2DiZPa6SYc3BFidFojHxhDqt3H8HyLz0LF730Vetc0fZIn81gtWAtmCRGdRl4wajX14JWx/PQVso2ikcdK51irrDvjf4aiAUy8QKI6KPM/EBmfheAd220zaR1J0ywQHDRv+S7p/02pMjZLxLRlZDWnFcBOB/A24noUmbesAwrkWbdijkPSFvW7oG9MFk6M6ECyMtqBzbESdrtUQHRxAAE33AcbKzhqS7kGWQtP3ON4lGLqO7/0h5LZ74DYyS+YhKDYqUIzJkS8m6vLBzPeeZZjSqxYBrauYewBmEpUOY0Yu4mITz25k3drIEe/q/vXbNMsshRB7YTX1ctYeTz3RA/yfo5ykEBMsYngDo/bpkfXcYeFKHFHfN+JvEmH5+SOZR9Ogsd3wOdGs2ugDrJMevn3l2XhUZaAQae1IFmFfQxlJuMCa2JYyulDj7XsTRF4OmcrGWGJM+FwpR9gUndXsEPjdhP5AbTmEKoTxfmXwVl6tF2ZTinjrkcjFAsr+6Itq9UtwpYy/CDJenjp7GFG6O+glBHGwSgsSQR4CZLZ54AerJaI+tQDBGuIFbId6+z7RVEtAYVFhFBYvATadrWxATgBwHch5lf4ttTnsvMH5xm/0nEzG+N/v47gKf6398N4C89vPnLRPQFiB/w/Rsfrzb5pdR4JRn4xvu8hwNgBg9VEBjqz07WbzSkvnJl7opmCS8MTB2s9P/VemgcJ3LNxKig2G0AAHk/9y9W7epQ4VINK+TzPijQCjdptdy4LE5IvIQINf2d9lI85sMb3oqp6LGf+CDedsWD5JjdWmNuu4aksKLm99RCkD2oQSG1adfAlg5kHUySNYAUWVZrtKFKQXAVCkTc+LbAcszavRkzfZMQrEMQMnrMuKNhjP4DIlepWVtgUinefj3S5ydmv4oe1HGE5dFzV7fEblqjk4SF5tXo9sXyqp+HnQW9ALUlAjRLAgmqzYGctyyzlsUdxZco8UF/o/e7fg8AjeulE4XYsdCplCAJAMz8rC1sfvnmm2BdZX9aTvtHAK6DFJ8EgGUAfzjlvtPQjwLQ3qgXAPh6tO5Wv2wNEdGziejDRPThI6MxTJ4G95cdFbBlhWJ5AHYO1coKUJVY/ZuXb2ugtd86guBOKG4XM+m2tRLG791KaU96yNTWSe36iPMVVEiETnhJk1HomLJu2ngB1Z2iL2vez6W1gGdAJhdrIPXL4jHoONJeit6+LvL+7EriZP08NGGjSCnQa019F0qT13Ok16pjZMfBalCEle6bdtMQY6nzYZouq7Sb+koEcZymvldpL0XWF0kcCz1B15lQU03ms74HGhNSa0ID9fEY9FokVrb+qzi5Wm/T+tV4S2zxNEoHBahzHVuJY5Jrjm8MqsGwEUOaNd30lCc0XInh+Y7nNKnnRq08vcfqxpXromju6/dD92mWQJpR5j0DpeWpPicDEdHLiWiBiDIiegcRHSKiH5q0re8wudnn1vXONa0qci0zP5CIbvInPUJEm8KsiOjtkKrIbfoVZv57v82vQMyy1+puE7afeGeY+QYANwDAt5y9v7FN8D8XFdAXQZPO9QC3vcZGJjcSFfI8hkxUw6hV6ylGtChjj9smA2iUsVBUjFo4vX1dKaDpGadmWCfdNMBVAXGPxDGebD5HNayQJP9/e28eb8tWlYd+Y86qWrXW2nuf7t4rSBNBAbv4EK/GXhSDiEYxqDEvCuYl4UGiQRKC8jSKiSR2kSTyEsUutig+I8EeNAqKoqICXhQUAfUGAvdeTrv3XquaOd4fY4xZs9Zee++1z27PuTV+v33OampVzZpVNZpvfGPMDj4zJpHBKD5R1sU019UNQ/SMXeFQ+By0UcScwie/9nWHmrtFeeIbfwev/9RPg6zJnniricHLpzlClURouYfPAVIDR22nTKgghbcCspJQbdaRbdajJ6tyNzzeah6ycRZzMvaZz13Md/ncgcssQnHR0GmXaJOd0cfCWj1+J3QFdP2ulvVcMyWZssFSz34Hs8x3XbIjDd91vcFsHLIfvYeNGVY1QN7lJKwW7DgkTcADABVJDVByr1KPjNFdl/SZ263wMe0wENqAbHxUDNGdy3OfcXkyM79AV468F8CXQPIn+66vclBZ1bDUROShCp6I7sQK69cw82fv9T0RPRPA5wN4EndtQu8F8Ihks4cDeM++I6QulAcAlyu8og52aFu08zn86HC9kEIV4LSgj1tGsVaguiH1IJ/+u7Ja8+v+1if3mDmpcTHFHYe9JGfSbdspBql6T1YCVG84bT+fSloMaasG5msF5tfmOwoljVAQIoNGxjDaGMF5wse+8tU4LvmU3/pNvP5TPw3OO8kRJQygqEyT+co3MlSbdYQBO4+9v/6JLzx8UjNEjmLBaar4nSdQnsVjhNzF/TbbTb9TthqPYlrELsPp/BkbL8JWBu0k0W2veLJqpRI8iTaBnRTjNM+wTHZS1Pssw0X4rc+6k+R9KkbVl9+3wDFCYUDX72txhcd+lNGnVFseKc0nLUcF9LOFtjlHIbcaFIaoDfFUAC9n5g/QHj3wDiOr3jH/GcDPAriLiF4MyYccark5InoKZGWyz2DmreSrVwH4CSL6Lkjy/jEAVsjldMlloP/whKpBO6sQxiP48QRbP/OdmDz9+Tc1bmvLLfvucOCwYGfFm+peL/NEyRPQdg/EYt8n+czHynRTbItssGa76SVo7bzTh9H2nyqovMxkRT3dd60JbRtrM2vwt16zlBBypDLaKARHTwyDkRscXJ867R3GF0rMrs3hc6894AQKsrlr66CeadYZzTbAF8KSM4ZeqzmVvMwwv6YLo6lRy5WRZ7TWtmqRlxL1kCfk2q6nm2+Ch+8ZQl/0C/oWDU7qNPQhrL7Bks/6UU6PcJC7hPlnuZWORhyN5oJCtdVPyXuhGtd1fA8kxIMjbNx6zz/4vJ6jkN6P1mfNxr60W3RiNNLnrl9IulgX1BWFHhUMBuCWq2OBLHfyNgDbAP6pBgjHshLwqk0of5yI/gDAkyAa/GnM/KeHPPZLAYwAvEat5huY+dnM/FYiegWAP4FAZP9sP0aYCSVhcTsTRWFskLhNfrgcQZqAt9cpzBKPkyQQ7XfCttLoRY1I9M4VTjEv2R4QWyTKJxCBedBO8eYIDSwoqLYOWoXP0ZMenRtFT88gMxuvS6qd4XEiRgUA7v7FX8MfPe3JqNWg+FyiF2nprzmKBepvuTESGNFr3qIwYxFi144InU1dpDU7TzHi6eeuhHU02hihvlHBFx4j71BtVrDFtIC+4QmhW144JM0N840sGrc0iW7LGliR3yI8CiyBvBb6gKX/R6MV+gbO5oicg1fWF3kH73cq6vS1K7pnw+cZ2rqRiOWI5B1f9aW9923VIldCSajaeA3Jd8W/KXmiI0R0uUev7ZuscNjOu99CR6HO4mgjr1stYmHmr1MG7jVmboloC7uzwg4l+zWhvJi8fT+Al6ff7bOk5Z7CzB+2x3cvBvDiA+4Q3Lbx4QhVA1dIQ0rnPdx0LIrJHb6dg61EmXYFXlYvAPS9qtQYsHLxzUNLE9OhZXhLRi5p/w0gQjvUUuxam1YtC+zTJoqra+5I3qHerMS4VVIb4woPulYhS5TpSYok6lvkCr+lkF+sF9LlffMyQ8ilPfrO1RgZVGQgL4npYk26DKSECOs84MpMIyUHbmXdeJuf2ODQdx2EbZxmgD26HJktCSxJfS9OQ0oRhkKSLkQKrF3zxW7Si7DXYsLfxOpfaEneZVHSwmFL2lvzSZs7n3dGCAAoHN098OfP+WJYEa8YeZnT+kYVk/a9801Yij1jmjg/vTlKmmsCYhh7cGACiR3l0hknkWMhoi8B8CIAHwHgE5j5jcl3B6r5Y+bLyetNJF2Oj1L2M+F/ADHMBOCRAC7r6/MA/grAo45jUDclybPknIu4sayU14JDQF6MAbd7a4u95E1P/xy0VYhwlC8c2rpTBL7w+KOnPbnD4deKCLWIYlMDkncNJy0Rma7kaIpQxt5RJ+NpJpALgBiNkdeHR5cFIOfgCvtN90BlZYHQBuTTAs12HcdLzmG0UcCaFhbTHG/7x0/Dh3//Kw88Vzcjf/Plv4g/ecbfif2tfNE1IOS2jXPBvjOo5YUSHLrOzz533fIIaAB0HQ4sgQ90SqitWkzvmsgy1nVAeaGMzLJ6U+o6UvhRSBUhUrEtb5V6y7b/HB0Vua0Cco0427aDuIxMYd0SLBdm0s+79JWk3EcOoTVCws78gkuMhv2/L+U4yc+41u2fSF1BLFKxeiMAmF0WBCYEjs1DLTKNY9EasEUIKyuNpZfF0oKu8LR/zi7PuswCdqd834wYK+wE5B7I8vDfm354szV/JyH79Qp7FAAQ0fcAeBUz/6K+/1wAeybmT1yIEJtP6gPUzirUm3IDjy6si3KdbwP5zbFCLDlrRXnGpgoJ1AQA+TSXAj2v0YAanh1JY4U4/DRHvVkvtK2wJLJDs13HnA15B5d3IX730GV67CxGQQGALzK0lRROOmXD2GNlD7p4z5lWXYuCKy+Mj7yYbD/5yB/5Obz5Sz83Mu0AgZxo2uUzrAuBKJeqB4PE3NM4Rz4ddfOjyfs03yLbqZOQe5RKpba+Wtk4R7MtxiX9XTFVONOgLKUKWxI5L8tIQshiTqRrfNhjpxUetRI/LJcGdOvsLHaftvobY9DZ/FgEtEy4Dcim4y4yzjM0s7nmVbpOFW3dwE2yWMfi8kycs6pZaoAOKq7I0GxXmF+bx3kwEoTNua1YuijZOI/QnDRizWIuqG3nyKclQt30I5OYq+k/U62e+1EIg08ECrO0w5JE+03V/J2ErAo6fjwzP9veMPMvEdG/PaYx3ZQQCPm07Iq8gimPApbA92UBrmZw44O3z08TtT53qLXTMIAIoQDA7NpcHpgEfrG8RlvLQzvaGEUqrM89fJl3rCjfdbs1WUYOsDVnUDU9j04Mj3hpHuKhxe/0YbQkbXnpHJrNbbgii5RSgcHoxI1KKtaaxuC+rCzQzCpRkj6DrdeeT0td1G0keQvfeaPZWJZM8Au4ussdqG6i0baO0aOo+Lu5yqcjhLqJ8+ksmb0deq11bD+m4K2rMzktYLT8jBogu2/yhHxh9TBAv2mpdWaQa6hFtgvsPZeUo7k863nlaX1QmkvpIh81rqoK7FyNTSbLUBzuXnB5hlA3aGd1jPoyLfQV+nhH2150rlIID+jqb6Twue0Zkl4u1XsE1GBFK3w5ilDfUcoBApY7iOiNyfuXabnEYeRhkOJyk11r/kyI6AKEEBXb4jPz0dYRYHXDcj8RfQOE78wAvhzAA0c9mEOJGvPFpKRx903M6Gz91L/H5O+9cOXdp9XWlsOwYzgv7VSMMSTwR1Jop8lGDmxrfGG0MZLooLRWKx7OlJFPCrucA/kQq/J7UIZCfl6jsz7mnCNUdfJeHm5SpZOVI4Qg3iyAGNkV0wLVZoXZle2V5+YoxTzz6Ik71ynAHOJZ+67tuyw9bd1wxaAs9vey7aJi1XV5Ql2jcRVCLUbVYMI04Z1GhQFy/2TjvKeoAch1nNW9iDK0IZIrmu06Oghpe3iBSC3K7nukzjuQ6wgABoFa7U53/I5iL/U8We/7xbHKcbOeMmbfV7jZuNDebIevYSHvwbOqi/hLRPix9yy1cs/n0xLVta14Tmk3gHwq+jDW5RirzZYv9kn+RW9/chKVWcR2VHLA5P39zHz3bl+uUvO37Ge7DGu3Y/xjAM+FlHC8CcAnQqKbz9rtNzcrq87y3wfwTRDKMQC8Dl0V/pmQNEwUGClDHRZCY71BefPagfb9p//wC3qslK37t2LobjRZ5wijjREmd4zh8gzbD2xGLNxWtiSNRkxRpkYitp+xSEu9dGEklbrEspNlAOo6Jl6dcz06qC0VAAB1G+CKHL4cRbgjHss7NNc2uwcxricO+NqjWC9PPIEPpG1XVOmocfVlgXpzFsfu0NFlAwSu8Tp/vhRyBZUuQiimUKrrkquUfeTI1SgHreewffSMNMQwO3RKutmuhBiSy/LXxcYU2BBjxRpNVde3BKbR9kIAMLu8iXw66nICMbLOe7maSBduu/cuiarSvJw5IKKku/tqWT7B2FHRoLRdE8pQNzp/RdfA1R++mNDnGeZ1E6PzdlYjIKBQSneacJfuGTnKS+fQzuYyDxqdL0bctqCfXKMci8bTHCibE2OKLtbt3LQw0B7REpL71fztIget+XsugI+HMHA/k4g+HAdbVXJlWZVu/AEd1NkVR3HNla6SOFOopIyfu0kGZDlwQGaI9WWypobGZsnXHOobFYx3b4pvfGkaPWF7SM2jBDSC2K6EVDAtJVQvA5rN7bgKZlq8lk/L6Hm79UnElF2RgdoQFZdFHoAYEPP4gmLl+aTU8+miFlOE9eZMMHgna7f46V6LyB2PWLFcxNwLS7jnGEXGXx0NYqjrqFCycoS27ucEXAJ/uFyUlkU5TSXnaXOYTyR6MwMspAEH9sKg8nmGEALa2RyjC+twC46BKOdanQiPYn0Sx+nyHNy2yKejGBFZRJaN8xiZyhx0CecWYhRbNJE+DAANqh6ddrFDMTlZcjjUdTyXrBxFBZ3Wudj4GyBGf9GY5cKqvFl537d/dRxbPpXr4woPB4E4681ZnA8zAEZx9uUoMj3dgoE0GMw+bxcgLuccvOaWgL4xyY/ovmYA9emuTXzQmr8ZM8+ICEQ0Yua3EdHjjmNgqzah/HUsCbGY+chDqJuVxcRWbF/hrSlljXw61pyDB/IC2698CcZPe95K+7c6hAjTVC3KC2N5OKaizFM825LhQJcPSSEJX45iCw2DJdgFIHkYfFkgK/udAlJP05Soy3fuv4MJXMyhGFvOjIwpVFNeQfc32hij3pxHw3iSEg2vnpPRxu2cAURP1eeZKkOrsO/fzkEjDztn9qGnVIqNaczdBNS9BLcowLyX37H5ApJr1gaMzsv6SewDaOaiIgyJN23jHl1YR6hqBDTw3qGtmuipc6JgXZ6jnc1jxGD/W74iS5woc0SiMdIVIS0qSxVwWuPRh04z5Av3kBmim5X3f+dz4zHT6+PLIt5r6b3bjSVX58H1ILt2Vsk18E6iedcZ1MyLE1VvzToCz8LY43NwRBHLSdWxaAuW7wZwJ4BfIKI3MfPn3ETN371EdB7AKyH1g5exSleTm5BVZzgtUy8BPB1yImdKvHrnTqOGFB9OvS6ez0DT1Rb9evOXfq7WKFBcvx7KDIuKO4lEAETPq8g745JPxhGyShX84k2eF2U0Glk5QqaK0CKU3jHMq5wm61U4GY8vui6uzncrajrvwIk3GtDAKcZvHiIgOYN6c4Z3Pu//xKNf8hMrzdVRiBVChrpBsTFBPhnH62gRA6CQVJnkWGJkk8Vzy1S55EWJWguMo2I1iC3P4NeLHkziywK+alBvyW/yibCOIjHAFzGBbGJQqzkC5JOCxBxoNAfm8ww+z1Bd3+z93pcjyYnliIYphToXIU8A3XmooU1zCDGXsETJmjJPK/JjdFL6SM+3uTz/rH9309ezurYljWCNsac5PUoiZVfkEaoSRyDvVf1bYaM9a36qxmRa9nKLKRkhRma2nyUOyqGFOdY3Hacw88+iS0MsfrdyzR8zf5G+fJEGC+cA/PKRDHJBVoXCFhffeD0RvfYYxnNoIdclJM3QCOzQghRDRgn4pgKyvfHjP/77T430R2OrTO7aQHVtKyZQLToBuofGxDxf+a6D44CdLBZfCFYvbKQyJk4tx8BKGzaPMhsXaKumR7PkNiDfmHS/t+4DS2C/NBFq8Fkzq6KycgDy6doqU47rP/Kijj2lsv6MF63021Te/bXPRFYWKNanqLe2e3PFbQDlWcwPNJuzyBAy5pJRq2POxaAVhRtbi9R8V8/RzKoIxQCaS/GSqM8npbDL2hDnm7yTiGEqOaj0mnhf9CIBU3rmZNhvyTmUF88JU0rzPy7P0KrxiMbBHIDEAOWTElwK4y2E0ItckKOjCavYeFjPoev8kCb4FfoLAfAe2XgU5za9pjcj5J3moMYx92VGMy3ILNYn4DZgfuW6jK8cRdjWrkmj+/NlEa85JdcS6J//IonDHJCjisQZB2KFnQkhok8F8Bhm/iFt6fIwAO866uOsCoWlFfgOwMdhOYPh9IQ6psxuRVCpAhb8fP86Ip97FBtj1JszuDxDPpH6DkvYCm4+7iVB04fRFIcptFS5tUrxNQUYPUlVNClsRd5FxhIAjO+8oJ5gQHVtC6Pza5hfudE9rPrwmsIvFqqqfUIWMOVUaO4GkCT3orFI5eoPfENkpZkSYtft//qPvAjrz3jRgdh3Bn/UW9uasB91ijZPFEXZwWVtW/UMZ1YWaFM4MPFmlxUHxnPfmPQUUa5zI4tDCZMwGio1buZUWJ2QLwu0vuqdUwpR+jyLvzWF6xJj5csiRlmAGCaDvlyexUS/nVemy2/XamQzzcMB3b3eddFulN6bePTewec56iVG5LAGBZB7rPpf98HnGUbn1zv4Ns+Q67ybE2COgCsudE5DCCjWJ921HUtOptmcxXxqmkNLRTqF90kWi4bmKORWaulCRN8EWTXycQB+CAKe/hiATznqY61696QV+A3Ewv2jox7MYSWtxwCAfGrJWEnSUpGLojYFs0t7l/d8y3Owfd9lFOtlpK+O77ogkU8IyKdj5FOBaPJp2fOa2YXO620DcpS9BzytqzClBHSUWIsYem3OzYNVZRFqCettyWXZV47JXRek+M05uMTzrDe3I+wFAMX6NB6T24DgWjiFU6yo1BLZtn8AuPZD39iDjOrN7dgQgkOAV+VmcpC1b+79pn8CV0hewWR8aQPzyzdiriXfmKC+tgVuA8pLGyDvMFcWUy8SDB0dOZ0/9qHnXNj5NVuzHZCkK7IYcZihNiMaSSK+20c8vusMt8u7a9uDMeOS2QWC3icOWTRyFuUapNkm0CWAaESikUjHZueRnLd5+TG3ltS6WA4qnT/7/DAKePPl3wJf5CDv0ejaSFlZRLKIteVPFq7ctUmkGfiOUqyRX9sfYxq1ks2xRmku72DSozIsErHcOoYFwBcB+FgAfwgAzPweIlo/jgOtalg+gpl7XTCJ6HD9549auOsCm3q2sZgqgcTC5jZC3WD0wQ/D9qv+M6r73g9yLobhgMBDo/PrMdpI2UaZUkgBRIUdw/IUy/YOflruiKAsB+OLvGcIWqWqRqWX59E4tOrNk3fI8jJ6s+SdRk7y8FkC1t6zCzGiMskWKpUze+Bah2J9KlDU5ja27ruC8YZ4jPe95HkxSS3zsx3PgVuBUNq6Bjb7xghZvjJJwjkHViZTsTFFs10hm5Yxge7zDEENuDkN2UNGaOs6Xo9oBBZvj6R+p6fYbJjjUcThm+05fJHDawK81esDoG+EqxqhbuL8tspQs7HZ3Fh+q9meA7k4PPMr1+XalWLkm9k8Rice3eJniwbL7sPxpW5V2EVD6byX65MYRV/k4DxHW9dxXDF6zfM4HymU5IrV2VNbP/XvAecx+ZIXYPvn/19QUSKbBo1W1hTinPQS+UascRp52ZhGatgMmrVzdOiivQiDKoQcr70aY5urrCzgS3FMjiy3onKCLV2OSipmZiKy5U8OXim+oqxqWH4bwBMWPvudJZ+dnhBFGmU+HUsCUg2JKRV7LTdkLvUsmVBp0nW+r//V++SGTB+ytt+DKIWJUthgN48o7V3mvI/MmHw6joYvWOI8aQ4Ysf+klXk27my6KS6LJBbhHlMUlhC3B9P7IhqnRh/MbDySeaobNLMKxfoEzazCe77lOVqXsZnAOnmM0ixvENlXvcrv1aiqo/NrAuldkLxO7Ayc1N/kCWMutC280o/zYozGzXvG0hUZfJ73trM5aas6GgfJPXQJd5uHRejEvP0s8e5tvzEyMkJG4l3beELdwBc5MlWgkfqbRCPZeAQkjoTzHsG1PcNlc0DeiaECekn3bCxRrJ13M5vHe4e8Q+b13HyX1F4mrpwATb1609YsB+UFZr/8Mol4xlNUD3xA80DWtLXqRUcuz5Av1tm4zlFKn6V8Oka9uR27LaQ5wjSSSyMxM9A+WcAsn47RJFHxYeVWgsIAvIKIvhfAeSL6J5CVe7/vOA60X3fjh0CSO2Mi+lh0lZ4bACa7/vAUhJIcSzYt0WzOoqJOla3dsI0WXxmM0czm2PzfH0CoxFv2C8YiNQyp8kqxbMufAB08krKJ7MaO+1qAHXJV6gB6ysne2yNuME2POpx30I0dWyjHPh7b5VlvoTOaJMWjbUCm7CfvC4wvncP2A1dlnzqu0XQ9EgTSfS7OT5y3pgL0/dZPfzsmX/KCpddu8+Xf0kuoknNACj9pTY3AhSOEtu0pS6CDPa0A0q5B2NyO25kCsvxJmtcyiDSNOnvwldFaJxItmoNiRigdewplGnsvbY1i96gfjRCauqfc7dpFuKrtxpRe23ifWJ1MkQFV93tf5LHQcREuMnhUoKqFThV6TyIcrBP45OnPx/bP62rlWQEqSrRVDZ9nyKalOClbsxgNt4kTkOa/bF5tLixiB9Bn3KnzZfuorm1FRt9iLsUV4lzNq+u9e+WwwrfYCpLM/J1E9LcBXIPkWb6RmV9zHMfaL2L5HABfCano/K7k8+sA/p/jGNBNC1GsOLcHN12TJYWZ7EYsNiZRSV3/q/fHKCW9sck7ZOuSNLR2MNZ/LG0zYscFhHGTT8eYX74elQvQedyL3lj6f/qwpA+dKRI/GoFDGw2cJOE9UHj46Zp4mVnSzjUEUJbDA6AFaMPlU4TNayjOrSM0NSgrpLo8BIwunkO+MUE7q+DLAlvvfQD15jZKhWDM4Filtkm6+mA7q+Jx91oHh4PUgrTVZZBzsWJdqq+r6HGmPbt682ckhEmJrG7QauFjqJuoeNPi2Z5DkOXa8bqKRljG1CJbWwNCC9bGl3b9YyFfCHBZjnY+j4bADHd63VyWS78w3ZddLwDaIFOWza5vbMYxxnvBBbSzKhrz9Hvz7O14aQRt52bH49CClAXZbm/1or/ePWgKvqlARXmgBfHIOVCWg6YbqN75Vsyv3MDaw+6MUVocpxmAhIzgvAe8RF32TMboK4GduQ0oLEKxLgF1Fu8RabDZGZ7q2iYwBbJJGXNTRwaJHWHl/UmIQl//k5lfo4WRjyOinJnr/X57UNmvu/EPA/hhIno6M//MUR8cAIjo+QC+A8CdzHy/fnagNQbkN5035rIcAXVMJLezKkYobd3A6ToTlSaCreBsR5K1kJqD8tK5Ht5rEsNqvSxpDyv7PbchQjKmLAzLD1XTS3YDnYLoeiF5UXyJJ0dFGRtMUlZorYIt0CXbI7TgthWF7rwYmLwAz2ey/Wgc94Wmhl8/D4QWMMWX5aDQyoOb5RjfdaGrM8jyGO0A6Mao1eU824LLcriilKglaQoKANVv/iQAoPi0LxM83jnk6xMUeo0Ekx9FJW4KJZ2b1Djbaw4B+foEfOUG2oVoL31NzoNd2xmSROnG987HaIvUUFMIcj7x/pBr58cTcFNFI5W+JhuH7QsL4lw0+PnaFBySbgNZLgZf5zw00sokm4yjMaORGI1QN/DjCUI166BI58FNDXK+O7/Q9hwWu37R8ISEKdkcTN9QUcb6sK33PoB8WqI4v4ZQN6g3t8EhxNzP6MI66mtb/TyS73JjNr5FssVi7gmwjgpaOKqfZdNSoDZllZHzS6Hrwwjj1jIskFZcn6aNKH8VwBsB/D0A/+CoD7QfFPblzPxjAD6EiP7F4vfM/F1LfrayENEjAPxtyNou9tnNrTGglebOlEAKA7mmKwYMrsfECgvRRrExiQaHW6lKb2dVpPPaDc1tiLBLCptlubSRcbli/E4e5FzzIKnHnHr6aTFXHL9FGGny34xEVsQ2IQghLgVgv+GmBtrtCGe46booHPP+184j3LgiBiYX6IKrWcTKEVqBNEILOA8/nkSljCyH82V3bKAzYNrHyQwNFFbhppIEb5Yju+vhAIDZq38gjjnMtmK+CUC8LtxKIWSEp5Lmg73GiXV3zX1ZdFGFDz3PXPbRRmNoijRGVM6D60qUquYYbOneXtSlRtjaA5HzoJHMiSloyrtWJOS90CrzHFzX0QjH650Xcg3nMxAQDZjLcrBrNapqoyPhk6jU+RLkJE/mNDo0B8IMXDRsoxI+tPp9uvaJOgZmS5bQ9Rdl/hs/jtET/wHmv/pDUhPmPPz6eVTvfCvqzRnKSxtotMjUKu0tmnDlBIVzqK93q5KnkVoqKUFGxtpd/5QtZ1ChzRuALnoPAdlkDA7t0rqumxG+xSIWAMTMW0T0jwB8NzN/OxH90XEcaD/TbayBZZVyRzGjLwHwAgBp986bW2OASNrh68NCkAp7bipU1zflZvY+cvaLDUlMA+h5vVahPbqwJtx/jS7ycgLnParrm6pgXVfcZjmPPMPkIZcwe+AqAKE5m9dl7TbMA08fFD9dR7t5vf+QZIXi3C4qP1ESnaLydz4MXNfgrWtdsni2CX/poeC2RYAkYVk9TwKAyUb3+3OX5Dd1BcoLBOdBWR63d+vne8oX+lB2HnxHjABU6Y9K0KiM20fDowYJANqrD3T7m8/i/rNJ2XWyDa6Hl8s+OmeBQxuVB7tWrol65cUHPRTt1QciFLr4255CHZX990UJynKE61e684AqfkAihLrqDEM01rkYYtfG+eGm7jGrKDmGvecQOuNWlIlBL7uos4EYieT+jv/bb8dTGddoLGsOOR+35yohdFp0mb636+s8aJS836eAGBDjAgBoKrhzl0Djdbi187j4cR+D62/7sx7BoVifivKfrolDEwLa6moks2QbG2iuXetyfmtr8Ry9RmYRPitKoJr12ZZqRIOdrzlJybxzU4O3j2bRRAajao6WaXbMQkT0SZAIxcpFjq7dcyL7QWG2YtmvMvPr0++I6FBFNUT0BQD+FzO/eaHP18prDBDRswA8CwAeceeF7oEtyu6BVe8mYq9lgfkVqY2YX7kRq4BH59dirqRYn8AXOaprm2hmFULVoNmeI5+OMdEQW9aXqJCvT+DGU2zd+x5ho1l1Pzrqa725LcnSskCW95ks5B3qy5fFm3JGNkgebhVWr1h+48F1jbB5XRRhUYJnW/HcqZxIe3zv4TYuor18n3rQ4u1yLetUkPegooTbuAiuZqBak+1ZATQJy6woJbrRKCqNrkxxcgigpko8xBbcqCKezyRKCGoU5ppQVzhOlGS9g1pqr3tGIct3QDQWLURF6Lo26j1DreOSL7oIkCKEVXfnYvsz6Cu5HqIUE+jIjIFznbflPChDz2j1RK+xXGvXHSMr4MqJzLfClJwcMzpOEerqoDqu9ZrlRfwMzsNN1nsGDKEFpxHborFacv8tE4vkOAS46TrcdAPsMtCo1M4NwmYb3XERozzvInCdN3YCiTWbM2Rra/CXHirDmylEXc2EnaZjjIY5095oi+PVc3alwF/REDsPNFU3TytEYyvJrRexPBfACwH8rPYZezSAXz+OA61qrb4bO6nFyz7ryV5rDECS/09e9rMlny29erpQzssA4OMe92h26xfk4XJevJmiBDmPYuMGtt9/Gc2sQlYWWH/kB6GZzTE6v6b05BKj8+s9ZhNlhVA+iwyjOy6iunwF2WQM0kXCqK6QXey8w/GdF8GhxeyBqzGRiFxosaML69Ew+dEI7Xze98S96yVZAfQxb1OmCSwDAOHqAxIhqJctSfIC4fqVCEuEG1cEkinPy75DiJ5buH5FWsJP1wUbDwFhe1OouMUlhOuXe14rFWVUwG48jQbC5gyTDfGOmyoq1rC9mSjQDnayzsEwaM45UaIxx7NwsRM4Z5ni47aFK3M1aHUvxxOhD+ciNGeKiTLpS0V6fnH8iZGSyV5U6k7H4QBXdlGhwYBZDq4WfmNj1aiPqy5SiIbG5lPrW6jQcTon94Htz16r185NDRqNo1Kl8TR65jSeAtrlGE2lxsjG0vaMECCRFiVQ26IYhOnWzwNtC7d+XiC40SQ+qNvveFtHLXZOomh9NrmadeMNAUUpXaANavTTdQDXu2dkId8VnwHrUu6c/DZLlp7QexCw+9bFeXfrF3Y9t4PIrZZj0QW9Xpe8fyeAf34cx9ovx/JJAD4ZwJ0LOZYNILJfd5Xd1hggor8J4FEALFp5OIA/JKJPwMHXGBDRh53GU2kymeVKd/XwoxHKS+eUBz9GtrEBPPCBiOkLHVGbPWoYTeMpJg9XBZgXkgQ0CKicxgc/htXOg5xHeemcJp3lZjaWVn4eceVKDg8goInwG4B+vsCUjGLe5MVIAKpQqpnkRbI8KkCuZh10Nd8GQttBAoAm45MH1HmwnfPmdVWwiTfvHGi60UFP1Uy8Zp1j2YeL+QIbdw92gXraGmGlXiWNpNM0skLmUBWnjdGishTr7xEUVLipJN803YgecXjgf0dllCboEVrJgxiVNrRIF0S36CwqIVPm+vsYiSZwILKiU/LmWWvU18vjpLkaex9zaGaAunmk5HeLUJ3MX6mJeN1nup3eE71Gq2Em5A0Ucd8SxYQYQcSxJzJ79Q+gfPJCkw01RjxP8jqjEuyUrTWfRZLF6K47UT3wALKHPDI6Qew9WnvORiXc2nnwbDPOERUlsqJE2Nb1gkZjNcSzbox6fTiJXqNRt2tv95r3gB/Lc5EVcAnEeRi5BXMsJyb7RSwFJL+SAUhL/68B+OKbPSgz/zGAu+w9Eb0bwN3MfD8RvQoHW2OgE0tqn7sknuqog8Z8U6GoZvHmLC6c34GT02RDWCnmsWa5wCzOwW1ckpsf6jWNSvgLd4EvdQFZmG3CrZ2Xh6OcalRwGfU73yr4czkFzzZj2xZAYZ9yIh5dNUO4fhnNtWvIL90Bf+fD0LznXR1cA/UmC4Fp3HRdjOioBDTsd0WJAIWHDOKpZkkuQK+BwT1Q5VxOlE3XzaUrSoRqJoya6bpAEFo4F5J9Sm4giPFxLsIsVJSahxE8PCrKpgKaCuOnPS+2fYlYeGIIaOR777uJbrskuCady6c8C9Xv/AzC5ff34C4alcC8M3Zc111eJc0ZpZCVkhYsmon3qXrEURL4TGArhaLUAJtTIEy9LpKJuSc9h+hZm8F0DhLuGjFCnYe6km0WIgzJ1bSd0fFC8BCDLTAh5d21k+jXCcstRiaSj5CoYsW8QVOJ8zEqQZNzAHcRhi8L5B/yERIhbFzq5l3Pz5+7BKhxcuMp2tmmsBMVDuO6gi+nEt1UMyC4GJVbpObOXQJCi/byfTFSSfNoNBp3BIksj47QKjDfqjIYluWyX47ltQBeS0T/jZn/8iQGdBNrDEShcgI33RCFoVAYAFEAQC9haYlS3t6Em65Hr42KUsJ5VYJuuh4fCq8RQYwOIHCAhd7Z+TvBxVggHeeBpoZbvwCabsBfeogYjmuCG5Pvcg+AKAN37pIYwXOXQNMNGdt4Kg9MOQWNSsmXZEUPqjDFblFH9CD1IeJKjuMiFyMRe8h6np8qrJEYBKtpsDxFjGwMoktpuM538I334Pl24vmLUrXC1dkvv6xjky1CRoveaBIVGIQUk+bOo/mDXxAjWyTkgQXs3aKgqKCT/ZrCT48PY95tb8r9Y9+l8KDmUaJXHCGbhHxhXnRy/9k8kjeCRuhFZ71IZSGaoAQCQpD70fJnERpKHCOuZhophUi1judo+aEEYjXPHwBGn/WMHbcMZXmMWp1G70wO1FaAy4DQIvugRwLqBMUINyWGTNbh1s8jbG/K83TuUoyc4bw4cVmB7M6Hob36gEC8m9cE5pyOezCXP3dJYNmqY7pJNJ/Hc4mIhJv1r8UhJDBjfgsl74nooi7aeOyyao5li4i+A0IBjnHkUS30xcwfsvB+5TUGopCwZqhcQ3vfvZ13pg9XaGrAj6MXQ6My5i3cdEPot7NN8Oa1iLW76TpcOUU7f0CUjhotC+lBDiAH4gAeTcWYhADObWncdVBbYfTYxwPZCOHy+8Br57sHBxCMejyVcZYT6Zel+Q+DusgS75ofIefie0CgLDftAkoObcTUaVQKW0aTtxbxRMhKlW+EANXoRthBDYLRXOM2pSVgTfkFUeBtGyEa2z+NSvGq61ohtWQsmpeJpICmTiCgrs9bNCqWYE+ousWjPiqSFwCAJhuixGab4CA03Q6SShhYSSS4WyKbsgIYQ2jAeRFJD1HyPJ57rH0x5W5waLxFfT+XpuSS+H0yx53hCDFvYYadzFFySlJIc0/xPFyEY02hm+NBWRHri5AVYqAt7+R9ZAmmc5xKhEjjcTMxKuQADghb16OhDkmkFsdf6MJoCsXZPUnzGbwiAG1TiUHS59NyQTQay1zbX1GK86X1U2F7U+435yOUZs4Gty3ckvzcYeQWi1h+l4jeBOls/EvMx9c2YFXD8uMAfgrA5wN4NoBnArjvuAZ1syLUzDnc+nnQaAK0dVctvXExQjh207u18/FzGpXgqw/AX7hLvadr8fPsIY/s8h5ZDs7H8mC3WtnvC4BIvGxfCDMmNODcRaiB1Mi56QaonCBslaALd8XakezSQ8SYOA9utjrPPFF+buMi/KWHIGxeAzmPNknSA4gPFGvCHJbQ9zm4notX19QIW9dkwtSTpnIKyvPIMnMbF9F+4H8LCQDm1bsOKkmjpQxxG25qhXeS65F3njvlucAnRoFOabYLEiGjrBBjq8ZFajy66HP8tOeh/v1XqcEqgdyS9rUopqaWaxlarSMpOoJHjBxCl/dIFKpAZuL99woMFwxEhKCSXEX8fCEaIVtNEnlvP3Y9onEzEgf6if/0ulkOKebHkqg1Lr1t81ZYRNgZHp5tieI1QwPE+pbofC1I8+ZXi8Gy+zrv7neDwopHfxTqd/8p3PpdOr9tdGyicfF6X+c5nBNDk33wo6IjkT/ysYDzQjDRiKSdzyQfU826+cylfsaVOTCeClV86zo4g8BtStPmppZ7xmq0jkBuwRzLYwF8NqRH2HcT0U8B+G/M/GdHfaBVDcslZv4BInpuAo+dqYW+SBO+weAjlwG+ADUzsHn4edEVTlnNhbGCnEP+iMcAHMBZCbfmwPMtzWF0RgU+B0gMhhkWAH0YQwYE9gVcdQNs25+7C5RfAZVrcOsXgWYuynw8BbKRwF+b10SxOS8w26bWqJinPN6AH03EM9SEJ9ST4/m2bKc1DQYLgkXhUD4CK2vMoAJA6amjCciYQ23dOycOLVC1vZqAHoNr8VoovNVRkVVJbmt+qdAIJviEQaZEBItIjAVnUJWxu5wHUEc4s3r9K+RYk/XO4DU1qOgr93D1gcjmivtMoaVEmacRiUUH8Z5JoBajCi+DuQjoIhLnehFcNB7eS4SXSty2yyHB9rU4x+jokukYxDnp+rSlMJcdAzONXHWMneEMvf2k0t7za/1zTCm/8Xw7UgJXM7kuwSJRF1ldgDhkNMmAraua69SxFEpDDw2sXACjMfz6eXEAmwrc1Go4bD7HCOU6XDaKkU6MdrNRhOWQjSLJ4CjkFusVxgBeA1mW+DMha7H8UyJ6M4CvY+a9awUPIKvOsIHd7yWiz4OwtB5+VIM4EnEeWLsEV22Ci6lECi4DOIiCVZGaj01RvhZ5cAD7jv0D74Gq7sNe6c1IJL/Jx51nqZCYQQJMDiCSm5gcEBrwaCrGj/SBJwd3bgTOZUlgXr8DWLsI7/4adOEhEgVdvx/t5fuEtgvIviAPrT1YvL2pnliInjmaWrZ13W8AgMo1qdr2uYxt3hWLufXzCJvXEDavd3Olyft4nEQidNPUXRI762P9gCpZ50AaWbWX7xP4r9UowqAaWL5oeYGeKIp+FAdAIR8fz98IB0DCU1+SeN/hufYMhNtpNBdgrfh/uq/eMVxf+cZtOvZWXGxuyVgoR9fdIEY1rjPw6TEXcjQSqSbGRj31COVZPi5LDNTCPvK/9TTskCZxOkx5E8k9lmUSvVSbUrw72+rycaEFubI3H5yPxOkpSnE6NJIkQBy8ppYIRR1E0hyn03uRyjVw1XWXIA5oJxdAI2Vfat5HesRLjRGFJiEsHE4YfEtFLER0CcCXA/gKAO8D8NUAXgXg8QB+GsLUPRJZ1bB8CxGdA/AvIfUrGwC+5qgGcRTCROBiLDeNKe5WbtSQr0suxG7CfIyQFRJR1FsILotGiF0G4iC4ei6ekxkfOU5XCR9Ga6C2AtVzUNtEKAyQG1g8rgzsc6me5sQbbJv4YAFAKNdBzUzG/kGP0ge9AZ27C379DmTNHGE0BTW1EARcBs5HoHoO1JVWXZfRcGI0BtdzYDSNEZZh4Lx2B6it4bavirEL2j7FF8LQufqAsq2KSDKIUBZE2XE167xyoE9XThVUCFGB9RhU5onafBQ+YvxRTGECgB/LOWZY2I/v1TPEyDJt25EY3KXGJVX8CYPLCAQxN2HQ1WJFukYlQBLtpBFK9J47gkIsBjXmWephp/Pqu4hJDMKisfXduVkkEI1H0R1jthlJHjFKTPMkVd/oLYtE/Uc/Ce3bfrMrGgXEQcnL7lm4fj8YENLKuUto3vfXvYgopUa72XXJg00vgKqZUte1+zYHUD5CKPXZDQ04K8VZK9fFeNTbcn/r9gihIxDYpS/WdO4q2SYd+yHlFoTCfgfAjwJ4GjPfm3z+RiL6nqM80Kpr3v+8vrwK4DMBgIi+5igHciRCDvCFKHRY7qMRxapGgpoKnJfxZgUgSloT8YAaj6yUfXEAhQbBF+L9WP7EDJDLAMzBXjn8elNbDoBdJnegRi0mnBUyLg5dREGuNxYupqC21nyN4t3UIBRrsN5QnI8kIrMaAnKAfkf5SL01HY2Ol5o5EBqEch2cj+E2lShCBC6m8OcgEVJUil5yVuTA1TYIXfHZcs+/S/JD4SJLvnIjdS+UFWJwxkXHjFODkDLM4v5DiPmRCGXlBZzWE0UFabUwCXONMs2bLCqVFD7KcyD0E/dmrNLeYsuaSAJ9+GyRABCPkVLGF/JGcZsFskI0AInnnzKfug7BnXG0vEPMe5HrjIzW6iwm5ikrIqFjr+JIKsZJvZPv7l3nAc4iAYNriRTcZD3paNDCeXF+wua1mPNzZQVYBwkzCr7oIFCfIYzPAWo02BeCGJADj6byXFVdsW6sp/HJeSiawFkhHSKOQBhA1RwdEeAE5HG7JeyZ+duO8kCHARv/BYD/eETjOBJhu8nzMWxFyei9kEMYrcO57V7kwApbcTYSQ8FBFK/mSEwZQz0m9rkm7usIf8F7eRBMnAezer0cYp4j7jNCZWZQSLaxzxce2HgncOgKIn0BCo08JC5DGE3h5pugelthjiDH8j6Ok8n1DCJnI3BexoeTQgP2BUK5Dp9dluO10lIELPtL61TS80WifDtWlyrdZC2pjgLuOtp120qSfbYZE+XR2LikG0GbGCBd8yPi8rafSPFViE5hMX/uEpr3/VWSU0sUd+wIvZw6uuO84nkvGJUlXv6SnXW5HaP77iaL0U5qVAyKQpfg7+WlAL33694xl+WS2HJr6HJH2cd93o7hhL/4vX7U7Vx85qjalPtwehFu+6pU4ddzKeQdbwDzTYTrl8WoWMShBifkI7j5ZnxGKDQIxRicT0D1ltyrdt9q1A1olG/PVT6SiCYOjgAvKASZ0dHn+MiEbw0ojIh+DjAfc2djE2b+gqM+5mEMy7LWK6coFBWyRQ8IoQdhufl1cD6ORgGqWKEGheF6NzH7TDxop0agbUBtLZ69zzsDlJWdoTDlQuiURmqkACEAcNBIhrptslH3Oo2eigJUbffCfFLYDj6TcXKQh0aTk1RtCvYdgmDfgDyoGm3F/E9oEPIxXLWJkE+i4gnjc6BqC2Tn1QqUwMUUGBHIXYuFpJSPwNvX+zCN0mmj0oqRRytU8JRqOpLuxinElNZqRAinmsWiQukDpdFE8BEKipCQ86LUQqs5oxrZnQ8DBy2oUwZRpCnHRPsiPBYitTcaywUDtIOCbPtDP6qIkh5jMe9CDuRCx9wK3aqIHe04MRy2PbnOWGajeH9SlsfrRqERJ4OafnPGHBoB5jGX197za/Af/aS4TfuuP0wICZ0hpbYCB4VkqQZnuUQXCbQMcqB8BJ+PEMbnwNkIwVCDSua/VaPg5jcQinF01ML0kjx3oZHng1meA43qGB7UzMWoJA5Z1AEuE8cJaoDnmz2o7DByC7V0+c6TPuBhZvhszajzolDbujMmWQH2ylDJRkC9LTehwlkhG4FqbetdbsBtXQaXBbiYgOY3ZL9ZIdFMtSlGRaMQdhmQuy5qWYiCgC7o6CqSk9CcHJQaFV/HiAsQBWaQmC/kAanzeG4wkkBWRkgvTC5EuC7Fpm0MXIzBHEQJWERntTCFKGpqKmHS5RMgL+UBJQeEXM61qQVOGG/IfCTV1lHhhaQocSznFG5c6VrBjzakZ1Xd5Up8OdWIougV0sXpMgYToBX5yVwl1Nz0foi/1RwJN7WslaJGRRbcUpZRXUuR6mRD6NgxJ7MzQtlVYqTWRRo7hJcYmcQLT+czGqc0wpEXyc2F/u+d73J72SheI2orcDHWa79wjKwAtOaKYuudJZ2Nack4dF/EXb3NjoicAzifiPHQ5xRtI/dXIRArzW+AXYa2XBdKujl2um/bpzh2EpEarMX5uHt2nIsGN065yyR/CdED0egcUpiB5gQMi9YQ/h0AFYC/APAPmfmKfrfv2lXK5D1R2a9X2HUsNyAESW+fLfEZ0NYSkWhCPhRTUFuhogzF9JKE6Qr5ABBD43O54cp1wWWZuxs1NRg+B/JSlDoAdgprJco7FYaTpH7qRWnkY0JA/6EA+kpD98suA4qJwlMVgHGE2NgK1BSuEjhvBs5KBMstcZDjOicKxh7erIifA0nuJyvAbl2UlBojMTgKOVjEBXTGy6A2y2slEZxbvwie3ZAanCwH8hGA6+C5EAEwle7Uki/ywOa1DsuH5ECoWshZLIGeaDSWBDBbNCTdcYVFNAHfSCni0iiTshycdY0XXTntHztdAMtgNo244uJmadSyYOho0Qgm0QnaGj3mXs9IAORz3cZ326eKdnECDE5SQ8JuFCnNEuFKzrBzBJQ+7yUXBkDJEC3CO98I9+i7u/snNDHa5qYGeUT4yZ6baHwUeo2vdb7je1PuRijRe5tdBspdH361nIi3KMiLo+gysFNUIo1Qso4UE6+b5V59R7A5rJxgxPIaAC9k5oaIvg3SofhrV127iohewcxfSkR/jCX6nJk/5qgHvF9Ll/WjPuCxiSaewQE0q8R46FchG6GuA1pXYjyVBzR4WSGR+VxM0LejNTSBkSMAynKBeUmAeHExYY8kD5J1n9sDqGJwWoxAgJ7ny3D97xeNCpKH1OdggzWaOdz2FbAfCRyGqTyAbS3b+SxuB0oe1GwkilZzNEYYiMYLAFzTGSuSKAZEcm5K/QQE43b1Fth3CWjOR2BKPGNm9Vglh+VCi3b9g4SRpn2yaLwOnm+KUZmcB2480PfYLdmckgKAPjSV1qTYNUEthaFtUsSoHa97bWnS3ynEkjYq5LruFYValCetgEYxcUzOacGrUpuzEbiSlRNj4l47KUcxWFTvp56xsEg4Qqehn38Duog53iypEQvC1vJNl2uIUW7b3cehkXwUuUg5NwgNAJp73wqMpsp+TAxzWwtUWisslY+6ZwDoiiYtye4LoJ5pJKwsxdB2EJ1F4Xa+6qiE0Vo0SuyLSMRx1absS+eK00iFWeeKY56SqqYfxR9WTogVxsyvTt6+AV2fxlXXrnqu/v/5xzrQRI5lkZfTEIZDTRnyrARnFXi0BvY5agZIPWvvCEwFZi0DTUDmHDLX3WjcuZgx9xEf3KQeJX14oleG5LfpA0sO8Ave6AJMAJ/AXgvtMeI2dp6+kNxJNpJzzEZiPFphz8B31GkziP19cvdwJePhxFimry1iYZcpUy4HOI8PboC2aSeNznSMVM9BHAQvVziPSIrfqK3E0IxKULkmNGpfALPrWtDqQeMCLr8Yi1Tj9OoaJQA6Y5I0E+XZZtdy3qr3kxoOa/9BDWIhJ89nyRLOCZxlOaF0LRGIcYjbmRK1XmNF2Xn1PgflfZiUjPZtkE5y36QkC5tTcRTc3onnRXgtfi75CAqNRC2a5KY0F6eklF7UWc2kqanu11Xbeh2pi1ac6wyeOV9JHmjH+HwhubzF7zRfKfedUqcp75YPsNwKEGGwhoGWgTKJ8iwKA6BEGHWEkmQ1K9PsqOSUFvr6vyBdUIAV165i5vfq/yfS7xG4rQwLo2oZyMYoigbQByobj8AMFF4NC3r3mvxWH2jHLVyq0NMEuhmY5GaOODIRGD4yu8AhoSKjC/et9cWSqKRnaFJYBIjJdgAIIIABBgGFUm19iMn86FFq/YwRBKKxUK/Xtu1VISfjinkjhcUs6ol4t+siuTSnZQ85ZznYjZNzkYiSNYKB88CFhwEa/cB7YHoeqOfRuw+jqRgiPCDbGMRkDDSjA5vBsCR3Go2h7SIfW6ALiNBMrBKvq5jf6RTmKHrBrMw9LsYRGjRCA+WjvkNhc6mwIwBJGtv9ZuSQ5HqxRY3ptdcoVc696cNoS+6TRaHQgOZaL5Xe9MEiYK+kjhHQkDooSS+5rEB4xxuE8KLwFc0TlCVG4JJE54Xol5qZ/q4AVVvw1RZCuYHIlFwctjph7DzAyXMIdKiB88i5gXN5B93Z/NlrnbNopJN5CuR3PP83KwesY7mDiN6YvH+ZriclQ9tj7Spm/h+6zddDGvP+uP1s2bB2GwARfSKkDvEjIJ3rPYBNZt5Y9SRWldvGsACAs3INDZWDz9EGBgMI8R/AE+Di3SUefKs3nEU3MSQ3Q5MyvBYfaIs0oEqdPNCGpdvuaCexAGX0CjETpWyvW2YQZPxMBKdQV1DYa9Eriw+Zwg49IwP0FZQdK7017XtWuKlpkoiG4rlGVg5zVOo9Jpydu1Mopq0AZrh6DrgWYXpRmGxlAaqSZCs50Pol2cHc6ixS9pmt9jhaWG63610VO1c7F/MHtp4HyEk7m6SpaC+K03MCcwebmeOQRJlxPoAI/6GpBQLiAB4L5OrMwKRRo81zHLsUajIvwGa2qRqm1Pgvk5QMEhPh9TxCT0JHT7bXqNKta45q+7oQLppakvtexkeAwK12/j6hsKcGUfN8khdU2nMziwaYORk3i9EMPhfkQO8tex7ZC3GlpgzekTgHPt8Ba0U4bMGoBBACHJgZdIToFa9uWO5n5rt33c8ua1eZENEzIVDWk5JalIOuXfVSSE7mpwHcDeAZAD5s/6EfXG4bw0LoIpJt1qdlIUxtevcAwzsCiKSMJDD8oivTMwpd65ZFhR8LsoxrD/QTiIlSx5L6JN5FMaTHSM8TABwYpPTnyP4BoofLLhMvG8JcKdBFKvG4iWfXfZjAb2n0Zp6o0bMBIS+EkJyfA0gMOrHWA6nHatToqIyzMiZR3dZl9aA1f+M9wuhShB2DGnh/bo5w+X2xXT9NtYtxVgLVJsL6XULOsLkIrcxHnkXvVWCgDLDiUYPVLIeSFg7quibwPhpJLqbA/Dq4XBdjHhpQvQ03ux6hJs7KDkK0342mQKuwEwfQ9lUdY6O3SKJs0/vOFKRFu8k906vrSJ2Y3v2jOai86EdH3vd/k5dJ+xMC+wyOg3ZxUKqy/aYou3YsHCIEmq7jEsbnQD6TvI5BXXnZz4MsuQeZgcDcc24cERyA1uVwiEF4dy52nhADQgQQXLz/Jarg6EweVX8vZtEbxy1E9BQAXwvgM5h5K/nqVTjg2lXM/A4i8prg/yEi+u3jGPNtY1gsINmtE/Ti9XckN1ygxQhmF0VrD6Hh6sy9h4SJ5GZO9hGr8BXrNTp0D86IhZwLn6lBMQ/ObBIzy3kCyHwOchkCCB4AUw52Hm1gOBCqJmCUOdjyXSGpRCbmOMZ0zKL4eWHsCworNa7ede1nbKEr9aRjrZDCQvG3NrWZ1DWEyQUxKEp8kLY5XUGqEBBYyALnNI9TzUDFGMEotD4Hza53sGFo+zh9YvzZoCvWpLpVhvsCcKG3vTG2LApmnwH5RI6Rl0KmyMvYwYDqeZ915Dw4G0v7EpeJkq234zjs89TwxYpxjVp6olFM77ohcXhSZ4b1XLRNikCxCh+m19TnQr1vMrj2ukSdBmGNnbROcR5oBNoLxZoY1GpL9pWLIbXiXxBJoa4vACf3RJhckPFYVJ300QsgkJPoxIFBjtAEjtcvMAv0yd17B5IoHIgGhJj7qINK21UYy/sjMwa8q745YnkpgBGkeSQAvIGZn30Ta1dtEVEB4E1E9O0A3gssW6Tp8HKqhoWIvhrAV0Em5ReY+QX6+b7c7GWyeJH3un/id8xoAThiFL5j5KQPapoY7BLhGq2kUc5uSVQkIXqaPCQnEEBKFABiWCIKhyXSWsBzW04BVoZTA9MuRF7ygApJwX7vSKAEiSx2mSTLHZGyqdJ6GG2pseMcfeKNpgapXUg6JzAScZA8i8vAxUTyNBwi80qopJZn0Egny0FZGZsLYr4JFOMu0au5ELZxNwmrK3TjiueQRnIuQ9CeVAhtbFDKCSki0tItX5ON0I7Pg9buFKZSU4kxdD7mncLkgjCisgJudlWjAu2ioNET1fMIF0ZJxm3MQPlRAtmm7DhKa3x0hcsQALLalqK7vmZ4nBe4UIt/O6ZdlzeJcxVCZ/ySCIR0PkMxlt+2jdSXTC6A6m20LocPVmlfwrWyGF70oUK3ts+yW7IJ4lB5vbVDEraEoNFIkofsPZa6w+aobQAD7Qkk75l5V7jqgGtXfQXEVfoqAM+DwGhPP/QAl8ipGRZt2/yFAD6GmedEdJd+vhI3ey+5WYekDgxHQJ6wvgD04SOi2EPMEunxJlZDQ1oHk76ON7vL0DCQJb8hXoiKOMSHrrUIRR+OwF0uqQkMIoIni5gA4i7Ut/YNDGAeZB9EBGfbM8dIy7Yj7hSKjGvnPPTyQDru3nksGh1nWHzRhxE5oB2twVXJIl0RmimkSFUhlMXrQXxDIpWGgPFGbLMDoGtR0zYSWRozzWVwdXesXnsdqxex3IGN2RyM2LGgcw44LxCIhCDHjEAeKKagbASmc3H5gaiIXQa3fQXt+IJASPW2VKIbQaCQhqHQ/nBxblkNg+/PP0ICrS52A7B7tpS6K+LQ9dPyHiGfdIwxLRzuXbe0XUvbgF0ea8NipFGug9oGodyQguN8rBRjlmgUhMAMHhWom4Ayy4FRjrpleMqQMYPVGWrZwYXuHvdJqq/nUKk75UiNC8TJYmG0dI7hCVCBGXv6kmdOmPkviehOff3Nx3ms04xYngPgW5WDDWZ+v36+Kjd7qdzMveRI8jNygwO5T9ydhXAbQAzdoXiwTzylECR3Q9pHyJE+XHrjeyIwOHpPPmHABBBaZngSOMsMyuI5pcaFNaLh5Ls0cmtZDIx9FvfHcs7xfAji7WpCNV24qUdkAASeiR6zDUQPmORklnUloFRBQ3HqYiKrZBqElrKmLLppqvgdYHkCJRS0XaFoLwekbUNibgWI7XdSqFImoe0blUX6dRItmMKVSNKiQJ0KRjfumO/S2pNQx4iEbYW00EaoCl67PGxfBdWWW/FiMA1eMyYfC009GnkbI7lYwGvXwxLonDvNCc00Cs+kuapdayVo8KJjleTG7BqGUopnQ7khxcWQcwjWlggSSTSBkel9NmtC5xQxULEgBSZtcp/vFl1EByuN15l7v43HN5T5iFhge43nLAuJh/lNkEhFuT/UAPhuZv43x3HM0zQsjwXwaUT0YgAzAM9n5t/HitzsRbEci3gyBx9MSBTwPACenOK4un/dp7GywJpUVI9V9tFhuE61zGLE0aKLGqIhc4QWDtbUrk0yl8vOZdmDspd3tnjztzpPliiVSAU9OCHWwiTJ426HoW8wzGil7Cg7pik+S2IrxGZimxIkOeudMtdICAAxx+CcGr0Q25QYbEYeQKuQkiW5zQCpcraiO4bAU2n/rmURmFGW43jJaxNPH+fbZtWbRYEYEdI5tddWUyTEAeuMoIl0MyoKV1EDKXjlkVbZc8/AWPJfzn8k1OskJxTZWcZoW2huahFxZzCpvyxEeqMswJ8pHBwKW/OE4i0QmNG0YUejw1admMX7cPE+Tg3BXvd9YCGvrPqcH1vgwieTvD8C+RoAnwLg45n5XQBARI8G8F+J6HnM/JKjPuCxGpa9uNl67AsAPhHAxwN4hZ7sytxsInoWgGcBwMMfIay7m73OPS8/MFithz0k6c1uYXoduAcvQQcfQ3P0FX58cMCoTa/dRIfU/TyxZUSFVJi5gykS2GwRTsjIRQUJIEJnIA+E5EIJ/3QpLtDlLuQ3i0UEFFoxqqRKKxt10UU+7kVACDektqLckAaZQVlxALjIOoWpzQpj7ioT9lpMcrsMnGmurG367UVcFgvy4jSqYm7gwGHJ9XJ6XvYxEVp2HZPXxpQQNKxLhOVx4CGKvO5o1ogdtyXRHSM5i7acFKWaYZBtjB5ttSLJe3gxJEpEiGy+OC8p95hl3OV6Es1kkQnX2yyBZuWzhXs+ieriZ9jdgKzyOJwNfc4HoRufpjwDwN9m5vvtA2Z+JxF9OYBXA7i1DMte3Gwieg6A/66c7N8jogDgDhyAm60FRi8DgMc/4Qk3fYV3u7nFq+9wXJNmwQMzRZ2MK25/sw/OquNeJUJLv08htEUxhekTS2S5HgBghfpkPz4yeeTLTnF369yIoqoV9vF5DmdU4EQcUUcBJZIi1cSzjrmeuCaN67o6AwInJbTVMFrrIiozTL7LX0XygOU2rFg0yQGJYu8eD4MblzkBrbjQ8Q7wGgVYPUasxUiLVGl5CxMy4oLCT5xAePA5ML8uNHPWbtw6z0x51z5I58GiOlnTZCStZqqtLopxDsRdkSsj6Q7sXJ8ersy4eQAKNbwStTFC2GlMljk9yyDdg8hZ0+EM3CqGJU+Nigkz30dER7Oc5oKcJhT2SgCfBeA3iOixkErQ+3ET3OzjkmUK2R4ay4HYd60m/k/6PjvMw5k+/MsYdY1+budM8Xcd/GHuaqRaG8Rir4mQJYa3dXkHfykbSCiiqriZEeDgjb5NBMv79GjPVozXNnHpYwSFx1TRUrXZ3xZIoD0fYRzJC0zimCLMxRGQRqvR3a7znXzBkN/1mEkJJbZHCwYkarEOBIAazSKOVz7LEXwOl5VAUAKC1QfF/fTrXJCSHry2hCEHNoPppIdXSo+OK5/CIkaZFHYZapa5sQzOXizMW0PfHlIYaNtbInu/18pmR7ekZiKnaVh+EMAPEtE9kJN7pkYvB+Vmn4js9dDY+1vtYdoNUmuVbdZyZ1iYECG/kCgUI2hzUosQki4GzJaI5QgVeYXhHDS3lLCBbDxVDBG5Gx9l0cj5IocDd7AOoPUuYrWoTepJYp7EyTLUdv5Klggt96I1O2fv5HwN8lx1Lm1fZmAcEVyac6KkuBFaE2WLVC0U0cZzaGt4XfIhTC7IpkmnY4OqpMBUacwaFfFoDZbE5yx0nYoV9osti6yoU6OjlHzRwPWi+GX3ziLU9WCQWyRi+T+I6NqSzwlAueTzQ8upGRZmrgB8+S7fHYSbPcghZRl0sdQbZdkwpYKakUk9fIBjQn5ZesxgLyMOLB7TbIrlt5YZ8sA7I0TnR7EDg7N1d9TYtH6EOjAQulqI9OiLEKEp0L0gzWXS2y5YtKc1UiSRXcq445SVlcByMXyyNXw4yE8WiRNaTwNoNGaQmS2pkHV1SG1Wwrdz+akZMg7S6VhZbE2x1uUOxSxKMMjYEbGtAnfdzsLMt0Tynpn9/lsdrdw2lfeDHE52w8WBnZDZzRSaLVsS1fa33za7/SbNexFRVGptywAVcAB8Ptnx+/R3e0Uiq+iMveDPNJI1qq13BKIMzgnjsAksHRQ07xFAcAk0xUmOxlqmGNvM1uDptnUdZTrLul5yLJ0UiNSQUdfQsqtX8qgpi91608iNl0QijrrrdYusongscivQjU9DBsNym0tqFFZJ8h9k+72Ot+O3zLsarm6Tm39Il5IRGD3q9n5yszR1k/1ybIGB0No8dBuKYjZqu0aAbIzELoKCwlFoGZ4An42EDMFtRwJQ41S1Ad4BWTaSbgyOovFqXQ5SYxbIA+R7/bMW4a3dCCieBsV6KxVInqQMhuUE5OK6eM1XbmwdC1SwqjFIlcTF9Qk+cH2r953J+TUZb/r94jbpsVbF1veiSe9HoU6LPA+672XbHtaA7jWGVbfbD35cLHRNpQWhaQIyl8HnWoipEGGjzLA6+dG8CQhMcMQYZRIF1XC7tvTZk6iwy3gfbMJ8y9SxnLgMhuWYxYwKIAp7UVkvykESoMuUaHq8ZXLlxlY0HPtta9+nY7bfmnzg+lbPWO0nV27sPP+9iBF70aMPQ+U+C/pgtzHsZvTMMUhp8E3gSLJYtp+6lWglheXmSX+rlFq+KgnlNNiPZ1IYCCe/0NctIYNhOUZZpmgXP1tmaFZROMBOJb+K3Mxv9jIYqxiTZcc3A3MzNTi3u1JLI5pUiS/+v5fRNTEjlM5dz5C3KdQ2yMGEewzJQToZDMsRyEGV636/XaZ0l0FVt7IsnkMayeyXDH+wyEGM7nEfC+iM3IPtOuwmjFuGbnziMhiWFcXR8tzDYYzKMlkGFQH93MjtKLsZy/2gw0GORxYdGbsv0+dgt3v1QSM8GJbd5LY0LItV8nvJfvfFKnDWccvtakxWkYvrk2MjPQyyU5Y9L1dubHVLMNzC0M9xQKlD8n653DaGxZ6H20kJp97hg1mGaOb05dx0vC/x4izIXo5kbCh7AAr6XsLMCLdGS5cTl9vGsHjnbiujMsj+skiZfrDKUVbAp79LGWO7wV5niSF2GnnIIWJZLreNYbldZIhQVpPBoCyXVbzyvViHKeOsK9JcLmnu5bRkmVE9yWeIw6m3MTyTMhiWQW4puZ3yLTdbs7TX9uem492/TGTRMFsuq7dkAvPK+ztp2Q3yOlHHjHkwLLvIYFgGuWXktL3jo5BlCjH9bL8OA7b9bsYlLYDdS5bBxgdRylc3t08lkb9XDuWko33GYFh2k8GwDDLIEUoahRwnFd2MS9ou6Cj3v0xJn7ZhP0tGBYDQjdvBsCyTwbAMcsvIqn3QTkIWowxgdRjqZmUv5XnUijWFHPej7O/WQ+6o5MwWB3NAaI5lnaxbXgbDMsgtLcsWmlqU41R0t4NYvsXyLMDO/m0HOeejrhc5U8ZkQQYobLm4/Tc5HiGixxPRG4joTUT0RiL6hOS7FxLRO4jo7UT0Oac1xkHOllxcn8T+WfaXym4Mpr1+s8r26d8ivHV+bYJz0/GZTXIfRKyh6G7NPQ/6dxi5uD6Jf2dVLMeyyt9hhIj+LRG9RXXlq4nog5PvzqSuPM2I5dsBfDMz/xIRPVXfP5GIPhLAlwH4KMia979KRI89C8sTD3L6cha817MwhqOUs6y8z7TwiUUs38HM/xoAiOifA/hGAM8+y7ry1CIWSA+3DX19DsB79PUXAvhJZp4z87sAvAPAJyz5/SCDDDLIKQojhHalv0MdhTldr36KbpW4M6srTzNi+RoAv0JE3wkxcJ+snz8MwBuS7e7Vz3YIET0LwLMA4JGPfOSxDXSQQQYZZIccrI7lDiJ6Y/L+Zcz8slV/TEQvBvAMAFcBfKZ+vLKuPGk5VsNCRL8K4CFLvvp6AE8C8Dxm/hki+lIAPwDgs9G1/UplKWKrF+ZlAHD33XffJmVzgwwyyK0gzIxQr8wKu5+Z797ty710JTP/D2b+egBfT0QvBPBVAL4JB9CVJy3HaliY+bN3+46IfgTAc/XtTwP4fn19L4BHJJs+HB1MNsgggwxyZuSocix76coF+QkAvwAxLGdWV55mjuU9AD5DX38WgD/X168C8GVENCKiRwF4DIDfO4XxDTLIIIPsLnxirLDHJG+/AMDb9PWZ1ZWnmWP5JwD+ExFlAGbQXAkzv5WIXgHgTwA0AP7ZWWA5DDLIIIP05cRaunwrET0OQADwlwCeDZxtXXlqhoWZfwvAx+3y3YsBvPhkRzTIIIMMsrrI0sTHvx4LMz99j+/OpK4cKu8HGWSQQW5GmIeWLrvIYFgGGWSQQW5GmA9do3K7ymBYBhlkkEFuQhhDd+PdZDAsgwwyyCA3I8NCX7vKYFgGGWSQQW5KBsOymwyGZZBBBhnkJmUwLMuFTmN50eMQIroO4O2nPY4FuQPA/ac9iCVyFsc1jGk1Gca0uuw1rr/BzHceZudE9Mt6jFXkfmZ+ymGOdyvJ7WRY3rhXL57TkLM4JuBsjmsY02oyjGl1OavjejDIabZ0GWSQQQYZ5DaUwbAMMsgggwxypHI7GZaV1zY4QTmLYwLO5riGMa0mw5hWl7M6rttebpscyyCDDDLIIGdDbqeIZZBBBhlkkDMgg2EZZJBBBhnkSOW2MCxE9BQiejsRvYOIvu4Ux/FuIvpjInqTrW9NRBeJ6DVE9Of6/4VjHsMPEtH7ieie5LNdx0BEL9R5ezsRfc4JjulFRPS/dK7eRERPPeExPYKIfp2I/pSI3kpEz9XPT22u9hjTac9VSUS/R0Rv1nF9s35+mnO125hOda4GUWHmW/oPgAfwFwAeDaAA8GYAH3lKY3k3gDsWPvt2AF+nr78OwLcd8xg+HcATANyz3xgAfKTO1wjAo3Qe/QmN6UUAnr9k25Ma00MBPEFfrwP4Mz32qc3VHmM67bkiAGv6OgfwuwA+8ZTnarcxnepcDX/ydztELJ8A4B3M/E5mrgD8JIAvPOUxpfKFAH5YX/8wgKcd58GY+XUAPrDiGL4QwE8y85yZ3wXgHZD5PIkx7SYnNab3MvMf6uvrAP4UwMNwinO1x5h2k5OaK2bmG/o21z/G6c7VbmPaTU5krgYRuR0My8MA/HXy/l7s/TAepzCAVxPRHxDRs/SzD2Lm9wKiOADcdQrj2m0Mpz13X0VEb1GozGCUEx8TEX0IgI+FeL1nYq4WxgSc8lwRkSeiNwF4P4DXMPOpz9UuYwLOyH31YJbbwbDQks9Oi0P9Kcz8BACfC+CfEdGnn9I4VpXTnLv/CuBDATwewHsB/IfTGBMRrQH4GQBfw8zX9tp0yWfHMq4lYzr1uWLmlpkfD+DhAD6BiD56j81PZFy7jOnU52qQ28Ow3AvgEcn7hwN4z2kMhJnfo/+/H8DPQkLt9xHRQwFA/3//KQxttzGc2twx8/tUMQQA34cOljixMRFRDlHgP87M/10/PtW5WjamszBXJsx8BcBvAHgKzsh9lY7pLM3Vg1luB8Py+wAeQ0SPIqICwJcBeNVJD4KIpkS0bq8BPBnAPTqWZ+pmzwTwP056bHuM4VUAvoyIRkT0KACPAfB7JzEgU0gqXwSZqxMbExERgB8A8KfM/F3JV6c2V7uN6QzM1Z1EdF5fjwF8NoC34XTnaumYTnuuBlE5bfbAUfwBeCqEQfMXAL7+lMbwaAjr5M0A3mrjAHAJwK8B+HP9/+Ixj+PlEAighnhp/2ivMQD4ep23twP43BMc048C+GMAb4E89A894TF9KgQKeQuAN+nfU09zrvYY02nP1ccA+CM9/j0AvnG/e/sE5mq3MZ3qXA1/8je0dBlkkEEGGeRI5XaAwgYZZJBBBjlDMhiWQQYZZJBBjlQGwzLIIIMMMsiRymBYBhlkkEEGOVIZDMsggwwyyCBHKoNhGWSQQQYZ5EhlMCwPAiGiG/tvdaj9/yIRnde/f3oTv38iEf38Abe/SkS/uMv3/42Ivvig47gVRefik5P3zyOivyKil57muAZ5cMtgWAY5tDDzU1naapwHcGDDcpPym8z81P03u3khouw4939E8kQA0bAw80sAfOOpjWaQQTAYlgetENHjiegN2gX2Z60LLBH9BhF9my6i9GdE9Gn6+YSIXqHb/xQR/S4R3a3fvZuI7gDwrQA+VBdY+o7FSISIXkpEX6mvn0JEbyOi3wLwd5NtptqV9veJ6I+IaN8lEEjkpUT0J0T0C0g6SBPRxxHRa7Xj9K8kva0+Xs/ld3Ss9+jnX0lEP01EPwfpVL10PCSddb9DP38LEf3f+vlDieh1Ogf32PztMu4n6/H/UI+5pp9/o+73HiJ6mbZ6ARH9cz3HtxDRT5J0QH42gOfp8XY91iCDnKicdun/8Hf8fwBuLPnsLQA+Q1//GwD/UV//BoD/oK+fCuBX9fXzAXyvvv5oAA2Au/X9uwHcAeBD0F/M64kAfj55/1IAXwmghLQwfwyk6+wrbDsA/w7Al+vr85BWPdOFsS/u9+8CeA1k0bcPBnAFwBdD1uj4bQB36nZ/D8AP6ut7AHyyvv5WG7eO715oe5LdxgPgWQC+QT8fAXgjZAGpf4munY8HsL7LNbkDwOvs3AB8Lbq2JGlrlB8F8Hf09XsAjGws+v+LsLCwlZ7DS0/7vhv+Hrx/t0KoP8gRCxGdgyim1+pHPwzgp5NNrNPvH0CMBSB9rP4TADDzPUT0lkMM4cMBvIuZ/1zH82MQRQ1I884vIKLn6/sSwCMhi17tJp8O4OXM3AJ4DxH9T/38cRAj+Bp1+j2A92rzwnVm/m3d7icAfH6yv9cwsy1Mttt4ngzgY5JczjmIofx9AD9I0qX4lcz8pl3G/ImQVQ1fr2MrAPyOfveZRPQCABMAFyG9534O4gz8OBG9EsAr95iPQQY5VRkMyyDLZK7/t+jukWXrWewnDfpwa5m83q1JHQF4OjO//YDHWrY/AvBWZv6k3ofJ2uy7yOZ+41F46quZ+Vd2HFTW4fk8AD9KRN/BzD+yy9hew8x/f+G3JYD/AokG/5qIXoRu3j4PYkS/AMC/JqKP2uc8BhnkVGTIsTwIhZmvAricYPJfAeC1e/wEAH4LwJcCABF9JIC/uWSb65C12k3+EsBHkrQqPwfgSfr52wA8iog+VN+nyvVXAHx1klf42BVO6XWQluhecyifqZ+/HcCdRPRJuq+ciD6KmS8DuE5En6jbfdke+95tPL8C4DkamYCIHqv5mL8B4P3M/H2QFvhP2GW/bwDwKUT0Yfr7CRE9Fp0RuV9zLl+s3zsAj2DmXwfwAggst4adcz7IIKcuQ8Ty4JAJEd2bvP8uyPoZ30NEEwDvBPAP99nHfwHwwwqBWbvyq+kGzPwAEb1eE+G/xMz/ioheodv+uf4OzDwjWbr5F4jofojRshUJ/y2A/wjgLarM340+TLVMfhbAZ0Hapf8Z1Egyc6VQ1X9Ww5bpvt8Kad3/fUS0CckrXd252z3H8/0QmPAP9fP7IGu+PxHAvyKiGsANAM9YtlNmvo+EyPByIhrpx9/AzH9GRN+n5/JuCLQGCIz3Y3oeBOAlzHxFSQb/n5IKvpqZf3OfuRpkkGOXoW3+ICsJEXkAuRqFD4Wsv/FYZq5OYSxPhCSs9zM4e+1jjZlv6Ouvg6zb8dyjGeHpihqsu5n5q057LIM8OGWIWAZZVSYAfl2hHwLwnNMwKioVgI8mol/km69l+TwieiHkGfhLCJPqlhcieh6Egvwzpz2WQR68MkQsgwxyzEJEvwuhJKfyFcz8x6cxnkEGOW4ZDMsggwwyyCBHKgMrbJBBBhlkkCOVwbAMMsgggwxypDIYlkEGGWSQQY5UBsMyyCCDDDLIkcr/D3+DVuTRlMImAAAAAElFTkSuQmCC\n", + "text/plain": [ + "
" + ] + }, + "metadata": { + "needs_background": "light" + }, + "output_type": "display_data" + } + ], + "source": [ + "ds_chunk.sst[0].plot()" + ] + }, + { + "cell_type": "code", + "execution_count": 20, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "" + ] + }, + "execution_count": 20, + "metadata": {}, + "output_type": "execute_result" + }, + { + "data": { + "image/png": "iVBORw0KGgoAAAANSUhEUgAAAYUAAAEXCAYAAABCjVgAAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjMuMiwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy8vihELAAAACXBIWXMAAAsTAAALEwEAmpwYAAB+2UlEQVR4nO29d5glVZn4/3mrburuezv3hJ48A8yQM4iKogjMuCprxnVZZOXHqqsYFgOyrrgmzHFdv+iuy2JAXXVFlygmEJGcYWBmGCbPdO57u/umqvf3x6mqvt3T4fZMz3RPcz7PU8+tcKrqrXOrznvO+57zHlFVLBaLxWIBcGZaAIvFYrHMHqxSsFgsFkuEVQoWi8ViibBKwWKxWCwRVilYLBaLJcIqBYvFYrFEWKVQgYgsFZGciLgzLYvFciAQkeUioiISm2lZLLOT57VSEJHNIvKKcFtVt6hqWlW9mZRrIkTk3SJyn4gUROS/xjh+iYhsCJTbzSLSXnEsKSLfFpHdItItIr8SkUUVxz8pIo+KSFlErqpCluUi8jsRGRSRpyrzUgxXisgWEekXketFpH6Ca/2ViNwpIr0isktEviMimVGy/2dwrV0i8oFg/5nBs+ZEZCAo8HIVy1IReZOI3BXI+ftR9z1CRH4pIh1BntwiIqsnee73BzL0BTIlK441i8gvAlmeE5G/meRaZwd5Nxjk5bJRefg5EekKls+LiEx0vUORifJzjLQniMj9QX7dLyInHERRnxc8r5XCIcoO4FPAf44+ICIvBT4DnA80A88CP6pI8l7gDOA4oB3oBb5RcXwD8CHg/6qU5UfAg0ALcCXwPyLSFhz7O+BC4EXBvWpG3Ws0DcFztQNHAouBL1Qcvwo4HFgGvAz4kIisVdU7AkWeBo4O0jaG+1R1C9ANfBW4eoz7NgI3AKuB+cA9wC/HE1JEzgM+ApwNLAdWAp+oSPJvQDG41luBfxeRoxkDEWkFfg58DPN/3Qf8uCLJpcBfA8dj/rNXAf8wnmyHIlXkZ2XaBOa/+T7QBFwL/DLYb5kuVPV5uQDXAT4wBOQwheFyQIFYkOb3mILqriDNrzAF4A+AfuBeYHnFNdcAt2EKofXAmw6g/J8C/mvUvi8C/1ax3R48z6pg+9+Bz1cc/ytg/RjX/j5w1ST3PwIoAJmKfXcA7wjW/wf4YMWxFwJ5oLbK53sd8GjF9nbg3IrtTwLXjzpnxP83xjUvAX4/yX2bg2u0jHP8h8BnKrbPBnYF63UYhXDEqPfs6nGudSlwV8V2XfA+rgm27wIurTj+duDuKvPv4eCdDRcFzhrjHW8A/gPYGeTxpwAXSGIqDcdUXLMtkG/eNL7H4+bnGGnPDWSUin1bgLXTJY9d9PnbUlDVCzEv1KvV1Cg/P07SCzA13kXAKuDPwPcwhceTwMcBRKQOoxB+CMwD3gJ8a4Ja4rcCU8lYyyP7+FgSLJXbAMcEv/8BvEhE2kWkFlOTvWkf73U0sElVsxX7Hma4tj6WLElMbb8aXgI8DiAiTRgF9/A495pOXoIplLqCe79YRHorjh89hhzzRaQFoyg9VX16PDmD//fFY11LVQeAjRXpx7pXVc+sqsfrcAvqA5hKygNjJL0WKAOHASdiCt5LVLWAacW8pSLtm4A/qOqe0RcJ82mC5cWjz5ngGcP8HCvtIxpog4BHODDvwfOW561SmALfU9WNqtqHKUA3qupvVLUM/BTzIYFp2m9W1e+pallVHwB+BrxhrIuq6rtUtXGc5bh9lPVG4E0icpyI1AD/gqkV1gbHn8Yowu2Yls6RwL/u473SQN+ofX1A6Ae4Cbgk8Ds0AB8O9tcyCSJyDnBRIH94r/D6Y91rWhCRxRjzzwfCfap6p6o2ViQb/dzhemaMY3vJGfy/d45zrdHpx7pXeip+haAw/hTwGlXtH3VsPrAOeJ+qDgSF/VcwFSEwFZxKpfA3wb69CPNpguXOsc4b5xlh7P920vy17D9WKUzO7or1oTG2wwJrGXB6Ze0IUxNfcFCkBFT1dkzL5WfAc8BmIAtsC5L8O5DCmMDqMDXBqloKIvJ4hfP2TIxJYrTjuD64Hxifx48wJrjHgd8F+7eNcg4/Puo+L8AUPG+oqHHnKq4/1r32m8AXcivwLVX90QRJRz93uJ4d49hkck6Wfqx75UbVlMdFRJYAPwEuGtV6CVkGxIGdFe/s/8O0dAF+C9SIyOmBA/wE4BfV3HsKTJSfk6UN00/be2CxSmE6Q8RuxTStK2tHaVV951iJxfQCyo2zPD7WOdWgqv+mqoer6jyMcogBjwWHj8f4IboD88A3gNMCh+dk1z1ah523d2AK+pVS0UMouP7jQXpfVT+uqstVdXGwfzuwXSucw6paaVo5EeP0/ftAwYX37sHYvI8f6177S2CeuhW4QVU/PUnyx8eQY3dgbnoaiInI4aOOjyfniGsFJshVFenHuldVzxy0FP8X+Kqqjqf4t2L8Qq0V72x9+J+oqo9RKm/BtBJ+PcpcWHm/Myd4n8OKxFhMlJ9jpT1uVEvpOKbpPbAEzLRTYyYX4G5GOvKWs7ej+ZKK4yOcu8ArgA3BegZTO78QU/uKA6cCR06zzDFMbf+zGCdmqkLeFMZ/IMDSQP5KJ973MIqiIZDvo5hCOjweD67xw+BZU4A7Sf59MUj3Woxjsi041owp4AQ4CqOYLp3gWsdgWmFvHuf41cAfML1O1mCUxNpRaUb8fxX73UDGdwB/DNbjwbF6TI+jb1aZ/2uBXcEzNWFq01dXHL8e00Kqw/S86gOOHudabcHx1wcyfY4KR3Ig75MYf1Y7pvB7R8Xx3zNOh4BAhh+MsX9EHmF683wtyAcn+M9eWpH+9CCvHwPOPwDf4IT5OSptAvONvRfjn3p3sJ2Ybrmez8uMCzCjD2+6bm4JCrPLx/hgfk+VSiHYXo3pztkBdAUv+AnTLPNVgYyVy1XBsUaM420g+NA+S0WhznDPqT3BM98JnFZx/L/GuPbbJpBleZBHQxhH5isqjh0R7BsMPtwPTPJc38P0BqvsMfN4xfEkxiTVj1Eee11v9P9Xsf9tYzzXfwXHLgq2B0bde2lw/EyMyabyeh8IZOgP5E5WHGvG1NAHgnfrb0admwPOHPUOPRXk4e8Z2ZtNgM9jerN1B+uVPW82AueMk58a5H3lM505Oo8wFYR/x5gY+zBdjC8Yda0Nwf0PSOE7SX7eBHy0YvtE4P4gvx4ATjyQZcTzcZEgoy0WyyFE4BT/qaqeMdOyWOYWVilYLBaLJeL57mi2WCyWGScI77FHRB4b57iIyNfFhLB5REROOlCyWKVgsVgsM89/YZzu47EOM/DzcMxI+H8/UIJYpWCxWCwzjKr+EePMH4/zgf9Ww91Ao4gsPBCyzJnwuQ3Nri5YHJ9pMSwWyyHA048WOlW1bfKU43Pey+q0q7u6gMr3P1J4HBP7K+QaVb1mCrdbhBlXErIt2LdzCteoijmjFBYsjvPvNyybaTEsFsshwNkrnn5uf6/R1e1xzy1Lq0rrLnwmr6qn7MftxgptckB6Cc0ZpeCpQ69XS8opVZU+JdWlq4Y4HiUO3Xl54sy+6SP2JT/zOn5LcTr/7/1lIjmnwmx6pulkovzJ+7PHGqCAj3+wbrcNWFKxvRgTRn/amTNKQRFKGmO8/2i0sghfvIk+rKkUlhOlnS0KY38L/5RTniZJRpL3934NK2UdL/+mUriOTnswC9QpyTlOoTdWZafyujOhIMZ7n/bnfR+dV6Pzo6RjF1lj5fGBzhNFKR28+bhuAN4tItdjRpn3qeq0m45gDikFP2hdhS9NXEYWYHk/XnUrYrprzuH1JvtYplqDnOylr/Y59qewT0xDXiUcc43iqPwJlcXolthY+TRZDXKsSsF4+TdWvlVb0E2HAhhd8MWlPGbaymea6Hmmi2rfp2rTTfo9BM9cmR9h/pZ08v+jMk1cDkzhPV0tBRH5EWa+i1YR2YYJbBkHUNVvYyIgvxIzunwQuHhabjwGc0YpwMjaf0ljkWIYSxlUUyDsb814dA14LDPTZIXI6MJgdEEAI59log9yvOfZ14I9Nc0fWqpCjry6JBxvWFEE394I5TBGoTEuwfnjVQymWhHYHxNQ5X8ayj6hyWSMFkGoKKqt6Owvo/Nn9Ls0Vmtvqtcs4ZKS0ojnLWlshCIYW1GMvHdqVIUwLmXyBN/KNOaXonjTNPhXVd8yyXEF/nFabjYJc0YpqAoldcetEUylFhW+8FMpLEfXciuvM9YHMxW7aWXrZ6yCIKwlTkWhTfRsUy3sk7J/H0Zi1PQARVWSUqagQgqPvLrgDOdjZcEwOh8nqh2Ol3+T5Vvl/zdd/gDYu8Az95rYfFT5ju/VGh6jtTBWwTvWsWpbQuO9VxO962N9T2N9L/hjKwYYVggmz2IUKhWrP/JaBWf4PkmnRF5jw4piml0A/oHx9c4oc0Yp+GM654eJ45FyyuT92KQfwFgF5kQFZV5dEnhjv+gQ3TeUo4Q74uMfXUilnFLFBzBGITFOrbfygx39DKFs4z1bUpSCDufhVAv6sGBPysihLwU1whZV9yr8Q9Iy8jUsiR+cF8rggUIekx9hvkxUkIYtxb0UwOhaZPBeRM8xTr6BKXSnYqYaqyIy1v85UYGXdEqkpEzJG86jjDu017NEcjkVrYngWSrfv6lQKWtcTD4l8EZ8C/kKJVzE3ete4TnRdnBuaoz3MGwZZv0kwIj/OcyfPq8GgJyXMml8874VgnsmnTLx4J1LBvmQrPhPqmpVVokCnlUKs5eixthWbKbBHSIuZdpiWUoaI+MOGXMSblSYRgWzxlnk9lHEjV7cBqdMQoSsb1748CXOOJD1oV/NHOH1UgSGP4oibvQxjPURhh9Hf/DCt7nZEbWrrKZISYkWZ5B+P0nGzdPhDU9VkPVTe7eEfGh0B6OPN+/H9voIw0ItkmeM4YoNUqbPj0XKwTyvOa+gPmmJUcIn6w9fNyFCUY0i6dcECTwanDIdntDmeiOUQFIckmKuFSqNrO+RcVwK6pPT8l5Ko1jRLE+Kkh/17Y1nSggLyriUo44HIwrqYD3vx01eCKQYzrMwv7J+csR7Ep5TeX6libJSgUdpgpp7xilE18xInpRboqNcH6XPejX0eTXkvBS5ciIq4OpjeZrig+BAxsmTckpsKzabezqQ9WuiCkT4OyKPcKsyi41ON9pEF76r9VIk4wz/N2ElosuvJY5HVlNkJB/kwbACCd+phMiISkP4PoXXSYlHUV2TXz4m4Lln/uuUUyJFiT6vhp6Smbyvv2wUQzloKRT9GAtSZmK2pJRIx/IU/LhZiFPyXQoaj5TFdGBbCgcAEXk/ZkJ1BR7FOFBqgR9jwvxuBt6kZqKV8a+DUvJdOv00rfHccA1jVJM6LmUa3UHieMTFo8sfOTvkdm+k/XaRm2W7lyHueSMK+yzJqOaYckoj7pENajFQUYMLvrmwAFuZ6IgK8ZRTZp50068J+v1k9JEuifeM+LD6KhRNZQ02Hnxc89xB9ni15IO/tcPLkPVrxnTK5f04GTdPSkpsDfJkdE0469eYQi2omWYkT1ZT0Xq9U4iUYZ5YJNN2b2TLqVKBDrdYhD4fuvwMLc7giOeprI2GLZgGp2z2eRAmDWWtLJizvpEvV06RjgUFFCWyXoqSM/w+ZNwhU0A7ZbaWmkg5JbJeDVk/Rd6P01nOkPNSpN08rUEFoz3eQ4rhQjhFKTJ1pCjRJv1knMK4rY2MU2BTsY2sn6KjnKGrlCFXNpWMUBGUfZe6WIGkU6bgx3huqJmkUzY1YMcn7eaDZ/bIOOZ/Cd+/THCshBsporwfI4FHvVuM8r9eiiRFI8UOsHxxdR1Z+ncsBYH69i08vGVJVMnp8DLm3XaJFEMRl6K6JNT8n1vL5ls7c/kGwMRxr4b1W9vNs6jL8amtPFFoH/FOd5Qz5MopChpndz5Db7GG2liJ2liBsu8Sc7woX1viA+S8ZJV3nhgFSnMwoOiMKgURWQRcBhylqkMi8hPM/LBHAber6tUi8hHgIwzP8TsmrvgsTPRGBV1Y+Pd6tZESAIZrfgIdZTOzX6XiyGvcFCBB4f0we5sc9nJie+YjLalLxslHtdTwumFBk/VT5MopFiZ6yfqpqNafcYb2UiyjbbehQmhwjK09rKCEBc6OclP0YWbcoagmm5Lh64YFZtarCWQ053aUjfIICxmTtobucpq8xqIPLldOkI4VWZTooTmWY0l8eHKsON6IlkrY+irisrHcbJ5T8lH+hz6QTIViyWpqhJLNOAVjNgpMfqHSyvo10X/UWUpHNcKS75KO5U2tOl4ebi0Ezx4+f8GPR2aZ0MQU970Rz5vzknQX69jqN5Fw2qiLFeiMp4ev75TMb6xEm2tmCw2VX6jIwkK4soXYHuthR7mJ5mSO7liWbcVmnhtqjvKxOTEQrbfEB6L7he9amD8ldU0+BHmxMrknMvHgD7cMS7h0+bUjxvCcumLMmGtVUd++ha88eS593eezOLGcjJM3rXGnxOtWPLhX+oe3LOH4pWYg7o8eOx8wkzrsCw1OmT4fVsb3RN9HaGrKOHk6yxlIwaCXYFuugXw5hitK3PVoTg0y6CXYPthIayo30W2qRlFrPjpAxDDzwJYwLYQdwBWY7lkA12ImH5lQKXjq0FnOmBcD2FWoJx0r8rsdh5GKlenO1eI4iv6lAXUh1Q2xQaXjDA9n0MVP+lAbmJBacpQ9l4H+FHX1eRbU95Mvx3Edn2XpHnYM1lMbK9GaytGe7ItqpLsKDfSXU1Et7/GuhXQ804KbF8oLTIFxxN8/xMbPvYrEiizHLdgZNXe7i3VcOv/31DsFuvxasv5wS6TbS0fKptEdHFGzTuCRJUnGHaKjXE/WS7Gp2BYV5GCa0sfVbh1Ru2x2c1GB2ZY087lnvRryGqe7nI5styFJKZGMm0Kls5SO9lcqQoBSeW9TTmVrJS4eKSmxw28i4+Rpi/XT4gzSFiuQkEGS4pD1vRGtorCW2+8naYvngB46vDQ7Sk1Rz5MGd4g+aiKHYti6KXnGrBjalTPOcCEbPn/GydNRztAcy5lWXgwKGmd+KktXsZay71LwY3SV6gAoOHFaY1maXdMixYcWZ5DtXmZMZ2+vZ2rI3V56hMlr/eB8BspJin6MhFOmLlYg7viReaM1nquoYMTIuHma47mokjPaJAmwtdQUrYdKICUl1hcWcv2mk3n4VZ/k8J9+yuRreoiy5/Dwqz7JyTddyf3rxp+J9EW3fTh6/2POIgbLSXYl6kk6Zb524vhTWocKAeCqY345brqJWL1k4jFaLwdWf/Ir+HEQH4qtZVqW9NLdncZxlVRNkXw5Rl2iSFvNAC2JwX2SYy8UvLmnE2Z+PgUReS/wacxMSreq6ltFpFdVGyvS9Khq03jXAKhPL9IXNb6e8g7TDL7N/ynn1VyI09IE5TJeVzfqedzm/xQgOqYDg9zc813WLXgXxEwh4S1pw93ehbdzF7eWrjfpT/0EXm2cfGuCvmUuA0sVd9EgC5v78HyH3sEafF8Y2lVH/dMuTetLiK+kduaQYhl6+vBzA2ihgMTiOMsXU1xYT+cxKfpOLHHsYVs5rmH7iJph1k/tVciFBXvKMTXUrJ+MCp2sX8OzhTZyXoqS7xB3fL54/I8BuPKR1wGQjuVpi2WjVkplC2llfA9ZTUU137wfi1ofYQsqNFFUtioa3UEWuVn2eLV0eBm6vTTZQKmErZaMaxRAaFqoZHTrAojWRzv4w9p3+NyVNv7QBNZRro98Dlk/FbUiChon7eZpcIfG7cESd7wRjskwTeX5SadEe7w3yoNGd3CEuabSDxGapCrzw+RfPmqlZL0adgSFeWWrtLLVG/6/S+JdLHKzI0xxYQtlu5eJ5AA4denmvfIa4IiffRL1ZYRSGI8X3fZhOvrTZGrzlD2HZMwjk8xz+8u+PO45B5PX3/Uu7n94JfE+l1jQyPLj4NUoXo1Ck8kb9QVyMTLPuNR2Kvdc90/372fYCY49Lq6/vHHS6c0BWLVk137f72Ax0+ajJkz0vxWY6SF/KiJ/O4XzL8WEkWXp0qXc9NzXx03rtjRz065vRdu3DF0XrZ8bvwCJmY9YyyVu3bb3dW659+MArDv8QyT6m2l+ygESJPoacNdvob7Nwd+8zchVV4PX3RMpoLE4L30RQ8e2UmiCRH2BlelOWuM5Mu5QZIrqLGfYVWoAjM15QbKfBneIkusS973I/BXSXU6TdEq0xrJk/RRdpQyXP/xmWuJZPn3cjQC84/4Laa/vpaQuHeVMZEJpi2Ujx/ZW39jYM5KnzTXztJdwg6E0w71SQh9DXuNsD85tdE0tLHyGuHhsKzezudBGyXdIuwUWJnqNyUY82mL9ZL2RPYhChdfvJ+kK7ONhywmGFVmlWRBnuEZurmkK4c5yJirQS75DjlSkNJvigyQDM9DweIHhTyJUCJVKNlQexnxlWklhayEk9FeVcMlLPFLy0XHxolYfQFsiR3usZ4SJbIQJjZF+mbCzQ4eXNuZJhlsKYT51lOvZuvFEdpSaePea3464/9Ov/xgvuf2DAPzx7C8wFmf/7gPUxkzjvSZZpOw5FEsxWusGgv0zz+ZtC1mTPoPeNSk2724l7wladnC74jhFIZYTdCiJV6OIQu02h+anStQ+sHmaJBC8SXo9HorMtPnoFcCzqtoBICI/B14I7BaRhaq6MwgPu2esk4Mog9cAnHLKKXs1eW4Zuo5z4xfgzp8HjsO6lZczdEQbqe393Pzop1nbeil+Xz9OOo3X14e4k/fVvumZz+/701bgNDaQ6ikRG0xS8IVfPnICLzvqKZJOmZb4AK3xHK2x7Ihaa1iAd5QzUUHTFjOFdtZPReaPjDts8w5ND++4/0IKfoz6WJlrNp9JPjDzHNe6gyPqdlMqxWiODdtaU05phF8jRTlqOYS12oybj+7XFuuPWhi9Xi1ZL8XOYiNbhpop+jG25RpYUJel1i3yQO8SNnc1U3guw+qTnqM1OcDpDZuiawLkJTbSkV2hiEKHf2ULAz9v5Akc4XEpk3HyHJnaEflvQj9E2EoI8zb0NaSkHLXM8hoj6ZQiZ3DBj5F0ypQcZ7jLo1OCmPHJhErOXCcwVblDNLqDbC218NRQO0kpEQ/60C9OdJP1a6L0YavBKNoaSl6MVKwU+Qa2ek2RL6vNzUb9+Z/IL2JzoY1cOcG5jY9FaUrqRv9TJa++4z386sxv0JWr4/Hzr9rr+LoF70LLZTZ+ZSXz2vpZ3bSH5sTAhCaimWDzNhM1+k2N93JEzS4eaFzGn3evoPOpVsLGpVMCZ0Co3SEke5W6nUVS923E6+ufFhkU8Oeg+WimlcIW4AUiUosxH50N3IeZ9Pwi4Orgd9+MkRCZf9a2XoqkUtRs7AJvuFrnpNP4uRyx9oXctO3rnFdz4T4/zFS4aYzWyHk1F+IsnM9jJy6g8xiX5KndtNYNcHh9B6dkNkemo9CcsyTeRcYpsLXUNKKratj7KeWUWBnfE7Uonh6Yz9PZedQn85S8OjwV7t62nKfT8zi2eQfLakxheWRqB+vz7eQTHdE1SxpjR6nJ9Npx8zxbaItq1ztKjTw9tIA7OlaxvaMJdqRMLS0G4kHdDmh9KMfu9jZqdwzh1cRY9uBGZEEbnS9YxpMvKVNcE+OclsdZGd/DPHcw6gLc5nhkpUhKy2T9JFlN0VGKjxjQFzpbQ0UY7g/NLlFNvULnV45xqCzQw9ZNdzltzo9DS9z0ZgoJC/aCH48UdNjFddiJH4OgUO7zakhKiZyXpMkZpDWWJe/HebbYRtzxyHkpWuJZUlKmPd5DXMp0e2k2DZrIzquTO+n20qxO7KSEa/Ig+E9LGiPt5lmU6OGRoaXc37OURzcsQT0h3TawV8H/qzO/Me47ubb1UiQWQ2IxVl/6OLgOuwYHua6ixXvEzz7J06//2LjXOFgsX7yTe7csZ2upid90H8XdW5ZT3lVDqkuIDYC6oEHv12Sv0vK/TwBwc893ARD58X7LoEBxDk5JM6NKQVX/IiL/AzwAlIEHMTX/NPATEXk7RnG8cX/vdXPnNaw7/EOQzaFlU1h43aaXa6WZp9KsdCA5N34B6pnCxUmmwDUvl/b0UnvDNpbeYBTaugXv4uljj+aW15zAmuO3sKCmnyPqdhvnqNTTganp3tR5LL2FVOQQ93yH+mSe2liJmPgcVreHlsQgLYlB0m6eq14yrGdPvulK7ti2imfq20i4HhuTbayq66DbS9NRzrCr0EDBjzFQThJzPOpjebYNNdFbSLGrv57S4w00PaU037mDw3Y+CYCkknh9fROa0ADWLX0fjreUB9KLAYi3enR4Q5Gdvt8zzvQOzzi3R9vmQ7NTaH8Pe3oBge+kPOyP8YdbPqHzO2xpwaheZjGjXMOeRikpR6Yl0+Nm2N4f1tyfyC+KWluhUilpLOrSGvpznhpqZ+tgEzsHMzQkzH/0i+eO5/DPDtHw7Q5+fMa3AfjsE68kV07xnLTRHu+J/DWmk4DLowMmz7YNNbFrIEO+HKNrTz1rvprFr01w65+vGjffx2olAJHfK6xMgTF13pK7FmBWKISQN93xD/zTybfxXK6JYl+SVK+Dm8eYinYrXgJieWh4sg8tlaJnmE58nXvmoxl3NE8Xp5xyit53330zLcaEnBu/ADD+jdDxHSKuO+zUTl+EPzg4okA97+SPM7QojbrCjjMdnCWDvHj5RgAe2L2Y7IYmUnuE2j2KOsLgfBg6rEi8toTvOZywdBuLa3qqNgOs+MFniafKFHpSpNsGKD3egHgQy4FbhHybWa/phETWtLz+/KN/2qd8efHrv8j215U4beVznNCwhYyTZ2VyD21ujnopsserZUe5KerfHzrKK+MAhaYYME7w0F7fUa4n66dG9pCqGGjWFusnJSU2FedFfpC8H4+URaWTN0wftgaAyOQTji0ZPTakOTCphc737cUm1vfPoztfGynvfDlGKlamOTXIpu4WTmt/jvpYngXJPq446kaueux8btx2NK9d+hAA9/UuZ7Acp2uwjoFCglLZ3Mt1fJ587cdZ8Y0vcfgPB3G3d7HtDcsoZeCpT7wfGC7gX/HSz/CbP3w0yrNznOF612hFfm78ApyGeiSVQvN5JGVaTd7uYauuJJMHpNCdiNP/7svo33SSHUqS356mZodDZpuS6iwTHyjj3r9+LwUXySuy347fo45L6Pd/vaCqtCcv23rIOJqtUjhIRGYp19nvj2fdgnfR+7JV7DlV8Gp8Ej0uTgHyCzw2v/Py/Zb17Jd/FveOh6PeWued/HGcjj5u2vLV/b72RBz94a9QTsPiF2+lNTXImU1Pc3xqSzR+IWy5hLX3sMAdPcYjHF1bGSokHGQWmplCJ3ufVxONgg/HPYQ9kLpKGUq+E/kT4o5Prpwg6ZRZkOyjLeiWWqlkusumRTNSAcWilsbxNVvI+3Huyh3Bg92L6ByoY3Aoib/VOMm9tiJ19XmOmbcrajGcdvNH8VS4f92nWX7d1VB2aFlgujL3PdaCeEJstbGTL/2bDeA6rP/G0Wy++ENRnqxtugQ/Z/JrrEJyXznHeSNObTAA1PMPWksbYO3xHyP+zV4efWwZzY+41O7xSPaUSDyxdUSnkrGYDqVw5HFJ/e9fVzcj5mnLnjtklMJM+xSeF6xtugQtl6avNlVbS8PNT9LwxEJkdxe5M1YCcMcn9k8hnHfqJ+g8sZ5GTMFxXvoio8xcB9/zOa/mQrRsCl/nqMPxHn1qUvPQVHj8c6Y2+4qXfoYu4JtnrWZoSZkXHL2RExq2ROlCx3DJMSOYM5onS81eYwQi53Tgh0lJiVIQviNXTlFwPNObyCXqrbV1sInBcpzaWIn+UpKY+CRcj/r4EGXfpbNQR22sRHexjrpYgWPqtkfdXgFagxZGOCIaiHpdrUh10FGu5z+2vIhdf2ln1XV7aKfMTU+O3yUUQH/cirhw7r9+kpXpOL6r+IlGuo6McdiNnbBtN16fURI+e9f0R3OO88YRacIW7L4oi/A6lebQg8GFf7mEpy4/DHm4kdodDuntZeoe2Dqmr+5AMhfNR1YpHGDOS1+ExONRl9jzai7EL+TH/XDD9OFHDsa0JMkkWijgpNNIU6Op9T3xDO5hK6i742lu7pzKdK8jOft3H2DjlvnIRWmefff79/k600Vo1li3+DKypy/l8fWruXfREWhbAcdV0ukh0skixzbvYHXtbrJejXGAB6Nrh/0MI/0OWT/FtmIzJd8l5yVJU6CgcR7JLaIzn2ZDZytDu+rQuCJJDzfuk0iWyNQUyCRTJFyPoudS9FwG3Ti9xRqe6F1AvhRnfl02Uh71sTxpt0DJd+gu1vFE93x6++pwN9XgDhkHaMMWhZ6+CWu0r77jPRQ/0ArHmO6+3UfX4SWFBb/dg1+b4PEbPsGRia/w5Cc/zYpvfomVR2/n9pd9mcO+8GU2fPADI67l53I4DfX4ff3c5v+UF/ztl7n7+yaNu9xMKbnq+s+w8m8e3CdFLzHzjq9b8K5Ja+n7Q2hKm19XQ83TSeqfU9Lb8iQ2dx10haAIxSrmdTjUsErhAHKO80Zi89oo7+ngNv+nrFt8GX5h78FblWjB9EsPfQyVtfMQb6sZD9Hzt6dx7/c+sNc1psp4A5EOto14NOFHvm7l5Wh/Pyxso/f4Fna/IElsVTcFP8b2YlM0Ajjr1UQ9fjpL6SicSMGP82j/InoLpubemMyzobuF0l3NtD1UItlTYGhBDYklLrV5KKfA8WIk+5REf5xCY4ata4TiwuB/yDvUbnUptCp+QtGGEke37KQ+lufYum1RLJ6m+CDbhprIF+PEnqlh/r1lau96ZkIFfs6LPsWODxnT0+PnfwPu3TvN4T/9FM+88Z9Z/r3PU0Ocl59zNYfdfj/d/3cEABs++AGOe/9XaHs4z+2/vQIwhXZ43zPe8iVwh2u4A98OTMgdI2v+U2k53DJ03Zjv6nST81IMFBKsf2AlC+8vUXPPhv2qEO0PZjpO2/vIMgWcZAo/NxzLRvNGIYzbSqi5EPU8bg0K48N/+ikOa8hALIa3e48ZbwHc2vNd1h5veoFc9dj5+xw+YDqJnOjz50WFebjPaahH2lq46cnP7vP1B848AnUgNuST2u2SXxKnv1TDYDlJZ76WzbtbcVyf4xaZkAi9hRQ7ehsY6q2BmE9yc5Km9Uqi3+PZRTFiHiy4vxeeMOMj7hi6jnWLL2PwuMX4SfOhpx/ehb+ngxrPp/6k1QwtSDGwwCW9wyPZPUSsewBNxCg11/DIiuPoPs7n1kVrKJddSoNx0o8lSO9Umgd84rkCyUe3jNlHPnT+rlt8GSybx9Cz9az++g7O+9QnokGToznt5o+y+eLPGNNkvoAPzPv/sqzjMrQpQ+7tLdR0JaP/QWLxqKD/84/+iRfd9mFedNuH+dM5n4uuufGCYcdzqBBGm5omwlk4H4rFyH92IPwLu/MZCk/X0353mdoHNuPnC5OfdACZi4PXrKP5ABP26nBqa/EHB4nNa8PPDYxZC1/beilAVPMJY9Qsbe2mN296v1TGp7kqCDA2W5RC2ENlOprxYa0zGmfSdAn95xzJnlNNHKkjl+2kv5Bix1PzaHrCoZyC7Eofp80UEl7JoaFpgIEnm6jfJAy1wrIbupEuY5YrrTK9Rip74Kw78goKixvw4g6pjiGcriz+zt1IXQ1+Xz/qecjpx+Hu7sPfuXuvQi9UKrXP9uBv3VF1S+u89EWAGdAYhmmJLV+G35Kh6/gMnS8qj3AaT8bo9+i8mguRVBJch9JP0wc0RMXa1kvRAWO2e/ZjJ/H0R6fPHBmaOVe/4zEklcTP5fbZaT4djuYjjq3Rf7theVVpz135lHU0WwxhLStUDl5Xd7QtrjvCOeckUyPOfeaN/8yqL38ZWrvHDFY2G5RByHT2aIG9a5laKlF/25PUbV9O/6pasgNLyGzoY02xE03EwPfJHdEEJOhe7ZpRrUNJ4g3Ghr/oj0OwZSc3BYOXxuKmJz/LufELSC5cAMmE6Sk2Rm33vPRFY+73u3r4w40jFWJY0w7/f7fBhC25uUKOW3LXmm7IvUZhievibd2Gbvbg+DMA9qrVT4aOUYPWfIFNj6/mJf4H6RtK0V7fz00v+Vp0/OhfXsXg1gzPvnvsrsXhM0zUcpBUCkml2Pi1NmB6opECLP/3L+IUFqNpj/zLjqXmng04DfWTn3iAmWxyr0MRqxQOAufGLxhZMLQ0U97TMaIgPTd+AX4hj1Nby7lnfJJb/2zMQ35C2drdxPG//hhlzx130NFcR+Jxbu75LufGL6DhHkytffVhlNdv4JlrT2bzhR8Zkf74y74SrRcaIPbIpnGvvbbpEry+PmLtpnvhZF1vx2sBjFYUYQsARo5DmeyaYYVh0xfPoFzvkawvjFAIy6+7GjduxoYc9qUS/v2PjSioR9vYbxm6jrXHXsnGv2uhdkk/fUMpyp47QiHA+APaQqoxI/m9fTjz2lj1jzspr2pn+eDVe/03+0L774SOk4TFv4WB9hipZQu55f5P7Pd19wcz85r1KVimyHnpi/aq0WihiHPyMSP23Vq6nrXHXok+u23E/mff/U+s+OaXePrdH+PoX17F2b/7wKyJUHkwCWvWowvWw3/6KeLsPW/ww183Zou1rZeiyxYiTY2RvXusWr6TnB6zVyWVBf1UWlKVhe/y666mmEtEvoON/3oiLIAV/w6bX5UCjGM3HIegnseG605i3i1Jmh7u4eaHTXfXwa8WSeb6Wbf8Sdb3zxv33qHfYaqO5pBbctdG8cae+6s6Nl+4b+ajdSsv56ZNXzS99U5aDYtqyDwHpTqH1nuHn2smUWRap/ecLcy9J5plVBYMk9W0bn50bxPRma/9Is/+wow/WNo04eRzz0ueeeM/T3g8rDW/9JWfp/YRo3BHjxjXUonyGUcZW7znjzDtzDROV4J594G0NiNAao8wUBNn5weybDj/o6z93KW4zU3g+bjz51HesZNn33oFr7jmMyOuE0ZDPfqXV3Fa+3Nj3uvc+AU46XQ05mBt0yXjhocIWzOjFcfapkuiStC++hPOcd5IbPkyYLj1dV76Iu6a4d5wY+HNwXEK1tE8S7n43ovZevnhFJoT3PGL/R+lbDFEjtAxRpaf47wRt6GB7RcfzaNfnvnxGgeKI3/xCQa7annukg+OaBGESiFscYQjlcczl61tvXTMEPH7MxhushbK2mOvBIYrUGubLkFXLsJLJ0d0GpiM6XA0rzq2Tq/+3zVVpX3TYQ9YR7Nl/1nyxWf43qnfm2kx5hSV9vaw8AJTgN3m/5Tj3v8V/Jf2jXXqIc+Ff7mERzoWEn72oyMCjwiCV3PhmMpg2Xe/wJEfXG9aUwvboHvv1ms4Gr5aKpXRRKxbeTnlJc3EN+5i3dL3ge9PmP5g4Kv1KVgOElYZHFjCQstdYiKNhjXU+fcMsLW+gePdj004I9mhiFEI0Pa9Wv54wwc559Lxgw+PN8ZgyU2Cv2bZ5D6HICzKRGMVwvlMQpx0ekLT3U2bvjj+/WYA62i2WOYQlbXgtU2XICccGQ0em6vMf+0zgBkJD0S9rUKisSaxGF5X94iBiCF3/qxKU6ZbRWG5sA1nYduYvrRDAUXmpE9h7qk5i2WK3NzzXW659+PctO3rxJ7bw9LP/IX2d2UnP/EQotJU1PKrp0YcCwe7STKJ5gtR7CIdGOScF5kBlJc//GYuf/jNLL+2urESt+SuHTGZ1Vjos9sOWYUAoGqi4FazHErMuFIQkUYR+R8ReUpEnhSRM0SkWURuE5Fngt+95xW0WA4AN237OreWrj/gYcIPJqHvRGJxbi1dH5lsho5ZRHnHTrzuHtYteBf+4CBaKLBu8WWRKUfueYx1h3+Ix87KcOcXTgeo2l8gqSTrDv/QCN9NJTMdW2v/Efwql0OJGVcKwNeAm1V1DXA88CTwEeB2VT0cuD3YtlgsU2Tt8R/j1tL13DJ0XWTfD30Bv7/5wzi1tab3UCKBuC5OOs1N274e2faddBp/2078XI7GR3tY808bJ20BhNzc8128zVsmT3iIooCnTlXLocSMtmtEpB54CfA2AFUtAkUROR84K0h2LfB74MMHX0KLZW4T1taraRn5Tzwz5W6mt5au55RLvmwi3fb0zqoxINPBXHQ0z/QTrQQ6gO+JyIMi8l0RqQPmq+pOgOB37nr/LJYDyFRG/p4bv4B1K0c6km/u+S63DF0XmZ72hebHByLH81S6qs52FMHX6pZDiZn2gMSAk4D3qOpfRORrTMFUJCKXApcCLF269MBIaLE8T9DTjsHzxx7M6p+0OhoJDmZ0/trWS6uayyCM4wUmkuxcQeGQcyJXw0y3FLYB21T1L8H2/2CUxG4RWQgQ/O4Z62RVvUZVT1HVU9ra2g6KwBbLocZ4jl4wo7jPcd7IufELiO3sxd3aMWa62/5kwonE5rVFI5glFpvw2mNxsGdHO7AIXpXLocSMqjlV3SUiW0VktaquB84GngiWi4Crg9/ZEyPaYjnECCfYcdas2qsLaFjAn1dzIdrTi9TVjnud0b2F/NwAkkxOv8CHCMrcHNE8G57oPcAPROQR4ATgMxhlcI6IPAOcE2xbLJZ9QMslJGbmqR63e+jQdZETeLRfYTxuyV1rJu95HmNbCgcAVX0IGCtQ1NkHWRSLZU5S6SBet/gyzj3jk3Df46jnoWeeSHzjLrzdxkIbhv2olps7rxkR8mKy0BZzCVWZtpaCiKzFdM93ge+q6tWjjjcA3weWYsrtL6rqAYmFM+NKwWKxHDxCm340/0LFMUkm8bZum3IvI7elOYow+3xjOsYgiIgL/BvGKrINuFdEblDVJyqS/SPwhKq+WkTagPUi8oOgG/+0MhvMRxaL5SBzc893ubV0Pd1H1fDMe5ejnkf5tDX71O30g3ffjtvQgHfUcpy2Fo7+8FcmP2kOYCbZcataJuE0YIOqbgoK+euB8/e6HWRERIA00A1jzC41DVilYLE8j5l3Zycrfz6Ae/QRxP78xOQnjMOHHryDoQUpNl2yjHLdNAo4izGO5qrHKbSKyH0Vy6UVl1oEbK3Y3hbsq+SbwJHADuBR4L2qekBih1vzkcXyfMZxcDuD4H91Nft0iZcvX29+fwZnv/yzZue/TIdws58pjGjunGCSnbE80aMHjJwHPAS8HFgF3CYid6hq/+gT9xerFCyW5zE3P/xJ1i19H4XVC4i1Zvb7erf/9oppkOrQIBzRPA1sA5ZUbC/GtAgquRi4Ws1UmRtE5FlgDXDPdAhQiTUfWSzPc27a8lXE0xEjjy3V4eNUtUzCvcDhIrJCRBLABcANo9JsIeiRKSLzgdXAprEuJiL9kyxZEXl6PGFsS8FisTyvavjThSrTMsmOqpZF5N3ALZguqf+pqo+LyDuC498GPgn8l4g8ijE3fVhVO8e55EZVPXGie4rIg+Mdm1ApiMjrJjoekFfVG6tIZ7FYLHMGRSj7k/Ysqu5apgy9cdS+b1es7wDOrfJyr9+fNJO1FL6DCTExkTp8CaMexmKxzD5W/+tXWP7LXpyegVk33/Ghymwcrayqe5mVRORsoBYzd01prDQhkymFm1T17ydKICLfr0pSi8UyY6xtuoSVmTR+dw/Ma2Nt66VIKjXHAtQdXMIuqbMdEfkSUAR84J3AKydKP6EHRFX/drIbVpPGYrHMHJ99wpQB3s5dsHo5/p4OJJWaYanmAibMRTXLQZVK5ItBWIyQpcBHgX8O1iekakeziLwQWF55jqr+d9WSWiyWGeGKo27kip7h7bPWfo7se/tp/lQNq//1K6y8bicANz3z+RmS8NBlls6//AvgxyLyf8C3gP8G7gZSwKQTYFSlFETkOsyAiYcAL9itwc0sFsshwopvfIllQNeuBtqeeJr5Lat58oN2LpJ9QRVK0+Ronk5U9U/AWhG5ELgZ+Lqqnl7t+dW2FE4BjgoGTlgslkOUeNbB8ZTNF3+YF//6i2Tu3caq3Hy2vNOb/GTLCKZx8Nq0IiIxzAjo3cBrgQ+IyP8H/LOqPjLZ+dUaux4DFuyzlBaLZVaw6ns7SW403dvTG/rA94k/8iy1NQWO+Fn18zlbDD5S1XKQ+V/gDGAt8G+q+q/AO4DLROQ7k5082TiFXxFE5wOeEJF7gEJ4XFVfs+9yWyyWg412dkfrNz/8SV5024d5y5LHePeayedatoxkFvc+WqaqrwpGR98N0TiHS0TkhMlOnsx8dFA6MwfxxO8DtgcP0wz8GOPY3gy8SVV7xr+CxWKpBj+Xw21pjraPbd7BNf/v1bz7+RHtetqZpdNxXiMiD2H01pcqDwSTmk3IZF1S/6CqfwBeGa5X7tt3mffivcCTFdsfAW5X1cOB24Nti8WynzhrVqFtTdH2zY8cSyJrXYX7RJVhsw92a0JVv6GqJ6jqiao65XFk1aq5c8bYt26qNxsLEVkM/BXw3Yrd5wPhLOHXAn89HfeyWJ7v3Pzop7n50U9H25sv/hD3ffcDMyjRoYsCZXWqWg4mo+ZqmHKayXwK7wTeBawSkUqvdQb4U7VCTsJXgQ8F1wyZr6o7AVR1p4jMG0e+S4FLAZYunXRMhsVisUwbs9in8BERGS9YHpiwRe9lnDELk/kUfgjcBHyWkSacrKp2j31K9YjIq4A9qnq/iJw11fNV9RqCBzvllFNsG9hisRxUZqlS+APw6knS3DbegQmVgqr2iUgWOFZVn9sH4SbjRcBrROSVmNF29UEspd0isjBoJSwE9hyAe1ssFss+M1vHKajqxftz/qTGrmAe0IdFZNrtM6p6haouVtXlmIklfhvEUroBuChIdhEmUqvFYrHMKmbpOIX9otoRzQuBx4NxCgPhzgM4TuFq4Cci8nbMjENvPED3sVgsln1DZ635aL+oVil84oBKAajq74HfB+tdBFPPWSwWy2xEgbI/K8cp7BdVKQVV/UMwL+ipwa57VNXa+S0Wy/OW2epTCBGRJGaGteWMjG79rxOdV5WaE5E3AfdgzDhvAv4iIm/YV2EtFotlLqAqVS0zxC8xY77KGLN/uExIteajK4FTw9aBiLQBvwH+Z59EtVgsljnALHciL1bVtVM9qVqDmDPKXNQ1hXMtFotlzqGBo3m2hbmo4C4ROXaqJ1XbUrhZRG4BfhRsvxm4cao3s1gslrmD4M1uR/OLgbeJyLOY6NYCqKoeN9FJ1TqaPygir8cMNhPgGlX9xX4KbLFYLIc0M+gvqIZ9ik9X9RzNqvoz4Gf7chOLxWKZa8zi2EcAqOpzInI8cGaw6w5VfXiy86rtffQ6EXlGRPpEpF9EsiLSvz8CWywWyyGNGr9CNctMICLvBX4AzAuW74vIeyY7r9qWwueBV6vqk5OmtFgslucJs7z30duB01V1AEBEPgf8GfjGRCdVqxR2W4VgsVgswyiz3qcggFex7QX7JqRapXCfiPwYMyF05RzNP5+CgBaLxTKHEDx/ViuF72EGGoedgv4a+I/JTqpWKdQDg8C5FfsUsErBYrE8b5nNLQVV/bKI/B7TNVWAi1X1wcnOq7ZL6oTxuUXkClX9bDXXslgslrmAcSLPPqUgIvWq2i8izcDmYAmPNU82QVrVXVIn4Y2Y2dksFovlecMs7ZL6Q+BVwP0Yi06IBNsrJzp5upTCrMwZi8ViOZDMVHfTiVDVVwW/K/bl/Okao71PWSMiS0TkdyLypIg8HvSrRUSaReS2YGzEbSLSNE1yWiwWy7SgCL7vVLXMBCJyezX7RjNd0u5rS6EM/JOqHgm8APhHETkK+Ahwu6oeDtwebFssFsusQqtcDiYikgr8Ca0i0hRUsptFZDnQPtn502U++um+nKSqO4GdwXpWRJ4EFmFigJ8VJLsWMyPbh/dbSovFYpkuZqmjGfgH4H0YBXA/w5X2fuDfJju52jAXnxeRehGJi8jtItIpIn8bHlfVz0xZ7L3vsRw4EfgLMD9QGKHimDfOOZeKyH0icl9HR8f+imCxWCxTY5qaCiKyVkTWi8gGERnTMiIiZ4nIQ4Gp/Q/jiqT6tcCfcLmqrlTVFcFyvKp+czJZqm0pnKuqHxKR1wLbML2Nfgd8v8rzJ0RE0phge+8LulJVdZ6qXgNcA3DKKafMQpePxWKZy0xHS0FEXEwN/hxM+XqviNygqk9UpGkEvgWsVdUtIjJmRXmkbPoNETkGOApIVez/74nOq1YpxIPfVwI/UtXuagvuyRCROEYh/KBihPRuEVmoqjtFZCFg54O2WCyzjmnqfXQasEFVNwGIyPUYE/oTFWn+Bvi5qm4x99VJy0QR+TjGDH8UZv6bdcCdwIRKoVpH869E5CngFOD2YDrOfJXnjosYzfIfwJOq+uWKQzcAFwXrF2HmGrVYLJZZgyqo71S1YJy+91Usl1ZcahGwtWJ7W7CvkiOAJhH5vYjcLyJ/V4WIbwDOBnYFA5CPB5KTnVTtiOaPBBH2+lXVE5FBjCbbX14EXAg8KiIPBfs+ClwN/ERE3g5swZirLBaLZVYxhZZCp6qeMs6xscwuo68cA07GFPI1wJ9F5G5VfXqCew6pqi8iZRGpx1hcJhy4Ft5oUkSkFvhHYClwKcarvRr4dTXnj4eq3sn43VnP3p9rWywWywFnesxH24AlFduLgR1jpOkMwmAPiMgfMTX/iZTCfYEv4juYXkg54J7JhKnWfPQ9oAi8sELAT1V5rsViscxBBNXqlkm4FzhcRFaISAK4AGNCr+SXwJkiEgsq6acD405nEJjmP6uqvar6bYwT+6LJ4thB9Y7mVar6ZhF5C4CqDsl0eZotFovlUGUaWgqqWhaRdwO3AC7wn6r6uIi8Izj+bVV9UkRuBh4BfOC7qvrYBNdUEflfjMkJVd1crTzVKoWiiNQQZIGIrKJiXgWLxWJ53jGNg9dU9UZMD6HKfd8etf0F4AtTuOzdInKqqt47FVmqVQofB24GlojIDzAO4rdN5UYWi8Uy55idI5pDXgb8g4g8BwwQRElV1eMmOqna3ke3icgDmPhEArxXVTv3U2CLxWI5tJndQ2bX7ctJ1Ya5kOAGJ6vqr4FaETltX25osVgsc4bZGBFvmE+p6nOVC1V0EKq299G3gDOAtwTbWaoIrGSxWCxzFsWYj6pZZoajKzeCcBonT3ZStUrhdFX9R4JRzKraAySmKqHFYrHMJcyUnJMvBxMRuUJEssBxItIfLFnM4LVJo0NUqxRKgZYJex+1YbpFWSwWy/MXX6pbDiKq+llVzQBfUNX6YMmoaouqXjHZ+dX2Pvo68Atgnoh8GhNT45/3XWyLxWI59JFZ7GhW1StEZBGwjIqyXlX/ONF5kyoFEXGAZ4EPYUJPCPDXqjruaDqLxWKZ88ysE3lSRORqzOjoJwAv2K3A/imFIKDSl1T1DOCp/RXUYrFY5gYz6kSuhtcCq1V1SgONq/Up3Coir7ehLSwWi6WC2d0ldRPDc+FUTbU+hQ8AdUBZRPIMj4yrn+oNLRaLZc4wi81HwCDwkIjcTkVYIlW9bKKTqh3RnNk/2SwWi2WOoRz0nkVT5Ab2jrY6KdXOp3DSGLv7gOdUtTzVm1Z5z7XA1zBRA7+rqlcfiPtYLBbLvjLLex9dGwQyXaqq66s9byojmu/GTNbwnWD9euBpETl3qsJORsVE1usw84u+RUSOmu77WCwWy34xi30KIvJq4CFMMFNE5AQRmbTlUK1S2AycqKonq+rJwAnAY8ArgM/vg7yTEU1krapFjAKajuk/LRaL5fnCVZiytBdAVR8CVkx2UrVKYY2qPh5uqOoTGCWxaapSVkk1E1kjIpeGE2F3dHQcIFEsFotlbESrW2aIsqr2jdo3qTTVKoX1IvLvIvLSYPkWxnSUBEpTlbQKqpnIGlW9RlVPUdVT2traDoAYFovFMgGzOyDeYyLyN4ArIoeLyDeAuyY7qVql8DZgA/A+4P2Y/q9vwyiEl+2DsJNRzUTWFovFMnMoJgJcNcvM8B5MpNQC8ENM56D3TXZStV1Sh4LWwa/H8GLnpiZnVUQTWQPbMUO1/+YA3MdisVj2mVne+2gQuDJYqqbaSXZewz54sfeVoJtrOJH1k8BPKn0aFovFMiuY3b2PbhORxortJhG5ZbLzpjJH82nA78F4sUVk+dTFrJ6xJrK2WCyWWcUsbikAraraG26oao+IzJvspGp9CmN5sS0Wi+V5S7U9j2bQxOSLyNJIXpFlVKHGqm0pjPBiA5dRhRfbYrFY5jSzO8zFlcCdIvKHYPslwKWTnVRtS6HSi/0joJ8qvNgWi8Uyl5nNLQVVvRk4Cfgx8BPgZFWdHp/CvnqxLRaLZU4zu30KAEmgG1PWHyUi+zfzmoj8igkeW1Vfsy9SWiwWyyHPzPoLJkVEPge8GXic4dES+z3z2heD39cBC4DvB9tvwcRDslgslucvs1gpAH/NPsy8NqFSUNU/AIjIJ1X1JRWHfiUiE2obi8VimfPMbqUQzrw2fUqhgjYRWRkGwAtGGttgQxaL5XnNbDYfcSBnXsPEO/q9iIRRUZdTRdcmi8VimdPMbqVw4GZeU9Wbg/EJa4JdT03VTmWxWCxzilnuaA5mXksARwS71qvqpFGtJxynUDkNp6oWVPXhYCmMlcZisVieV8zu2EdnAc9gZrEMpzt4yUTnwOQthe8FF55o2N5/ACdWI6TFYrHMKWZxSwH4EnBuGNlaRI7ADD4+eaKTJlMKDcD9TKwU7JRnFovleYcwu81HQLxyqgNVfVpE4pOdNFmX1OXTIJjFYrHMPRRk5ibQqYb7ROQ/gOuC7bdiKvkTUm3sI4vFYrGMZpp8CiKyVkTWi8gGEfnIBOlOFRFPRN5QhXTvxIxmvgx4L/AE8I7JTpoxpSAiXxCRp0TkERH5xajJIK4IMme9iJw3UzJaLBbLhEyDUhARF+MMXgccBbxFRI4aJ93nMJOPVUMM+Jqqvk5VXwt8HXAnO2kmWwq3Aceo6nHA08AVAEFmXICJyroW+FaQGRaLxTKrmKYoqacBG1R1k6oWgeuB88dI9x7gZ8CeKsW7Haip2K4BfjPZSdVOxyki8rci8i/B9lIROa1KwcZEVW8Npt0EuBtYHKyfD1wfdIF9FtiAyTSLxWKZXVTfUmgVkfsqlsrBv4uArRXb24J9ESKyCHgt8O0pSJdS1Vwkqlmvneykakc0fwsTZe/lwL8CWYzGOnUKAk7E32NifoPJjLsrju2VQSFBxl4KsHTp0rGSWCwWy4Fhao7mTlU9ZZxjY/XuHN2++CrwYVX1RKqe2GdARE5S1QcARORkYGiyk6pVCqer6kki8iBEc30mJjtJRH6Dia46mitV9ZdBmiuBMvCD8LQx0o/ZAFPVa4BrAE455ZTZ3TnMYrHMPaan1NkGLKnYXgzsGJXmFOD6QCG0Aq8UkbKq/u8E130f8FMRCa+1EBNKe0KqVQqlwK6vACLSxnB87nFR1VdMdFxELgJeBZytqmH2VpNBFovFMuNM0ziFe4HDg0Cj2zE+1b+pTKCqK6J7ivwX8OtJFAKqeq+IrAFWYyrbT+13mIsKvg78ApgnIp8G7gQ+U+W5YyIia4EPA68JZnYLuQG4QESSQSYdDtyzP/eyWCyWA8I09D4KfKvvxvQqehL4iao+LiLvEJFJu5BOcu2Sqj6mqo9WoxCg+oB4PxCR+4GzMRrnr1X1yf2QFeCbmKnibguaRHer6juCzPgJpk9tGfhHVfX2814Wi8UyvUxjXCNVvRG4cdS+MZ3Kqvq26bnr2Ew2HWdzxeYeTNyM6Jiqdu/rjVX1sAmOfRr49L5e22KxWA40wsTxfw5VJmsp3I/RhQIsBXqC9UZgC7Bi3DMtFotljjObw1yIMcG8FVipqv8qIkuBBao6oTl+Qp+Cqq5Q1ZUYW9erVbVVVVswzuGfT5PsFovFcmgyi0NnY4YSnAG8JdjOYkZOT0i1juZTA5sXAKp6E/DSqUposVgsc4rZrRROV9V/BPJghhIAkw4lqLZLaqeI/DPwfcwj/i3QtY+CWiwWy6HPLJ95jX0cSlBtS+EtQBumW+r/AvMYbpJYLBbL85PZ3VLYp6EE1XZJ7caEXrVYLBZLwGx2NO/rUIKqlIKI/I4x9J2qvnyqglosFstcYTabj0TkBcDjqvpvwXZGRE5X1b9MdF61PoXLK9ZTwOsxA8ssFovl+cnMmoaq4d+Bkyq2B8bYtxfVmo9GT+H2JxH5w5TEs1gslrnG7FYKUhFTDlX1RWTSMr9a81HlyGYHOJmxo59aLBbL8wJhdpuPgE0ichmmdQDwLmDTZCdVaz6qHNlcBp4F3r4PQlosFsvcYXYrhXdgeiD9M0bS2wnmn5mIapXCkaqar9whIsmpSmixWCxzBgXxZ69WUNU9mDDcU6JapXAXezsn/jzGPovFYnneMBvNRyLyIVX9vIh8g7F7jV420fmTRUldgJkKs0ZETmQ4KGA9Vcz1abFYLHOaWagUMHMyANy3LydP1lI4D3gbZvazL1fszwIf3ZcbWiwWy1xhNrYUVPVXwe+1+3L+hEohuOi1IvJ6Vf3ZvtxgMkTkcuALQJuqdgb7rsA4sj3gMlW95UDc22KxWPaLWagU9pfJzEd/q6rfB5aLyAdGH1fVL49xWtWIyBLgHMzcDOG+ozDOkaOBduA3InKEnX3NYrHMKmZ/QLx9YrKAeHXBbxrIjFrS03D/rwAfYqS+PR+4XlULqvossAE4bRruZbFYLNOGYGIfVbMcSkxmPvp/wepvVPVPlcdE5EX7c2MReQ2wXVUfDuZoDlkE3F2xvS3YN9Y1LiXod7t06dL9EcdisVimjs7epoKIpDBm+KMx4YkAUNW/n+i8akNnf6PKfaOF+o2IPDbGcj5wJfAvY502xr4xc15Vr1HVU1T1lLa2tsnEsVgslmlFtLplhrgOE3niPOAPmA5D2clOmsyncAbwQqBtlE+hHnAnu7iqvmKc6x6Lmd85bCUsBh4QkdMwLYMlFckXAzsmu5fFYrEcVGZ/QLzDVPWNInK+ql4rIj/ETK08IZN1SU1gfAcxjB8hpB94w75KqqqPYibqAUBENgOnqGqniNwA/FBEvoxxNB8OTDjRtMViscwEs9xfUAp+e0XkGGAXsHyykybzKfwB+IOI/JeqPrffIlaBqj4uIj8BnsDEWfpH2/PIYrHMRma5UrhGRJqAjwE3YCr4Y5nsR1BtmItBEfkCezsspmWSHVVdPmr708Cnp+PaFovFckBQZrWjWVW/G6z+AVhZ7XnVOpp/ADyF8QN8AtgM3DsF+SwWi2XOMZsdzSIyX0T+Q0RuCraPEpFJo1tXqxRaVPU/gJKq/iHo0vSC/ZDXYrFYDn20ymVm+C+MY7k92H4aeN9kJ1WrFEKHxU4R+asgON7iKQposVgsc4Zwkp3Z2lIAWlX1J4APoKplTOigCanWp/ApEWkA/gkzPqGeKjSOxWKxzFlUZ7VPARgQkRaCtoqIvADom+ykaudo/nWw2ge8LLjB+/ZJTIvFYpkjzPLeRx/A9DpaJSJ/AtqoYihBtS2F8W741f0432KxWA5pZnNAPFV9QEReCqzGWLvWq2ppktOq9imMxVjhKCwWi+X5gQK+VrccRETk1GCCtNCPcDKmi/+XRKR5svP3RynMYh1psVgsB4HZ2fvo/wFFABF5CXA18N8Y8/81k508WeyjLGM/kgA1U5XUMj7npS/CaW7ipi1fnWlRLBZLlcxS85Grqt3B+puBa4JJ0n4mIg9NdvJkYS4yEx0/VDjHeSNuQwM393x38sQWi8VSLbOz95ErIrHAdHQ2wfQCAZP6kffH0TyrOcd5IwDiurgNDQCsbb0UicW4ade3ZlK0Mbklt0/TqVoslklY23TJgakQ6vT1PhKRtcDXMNGnv6uqV486/lbgw8FmDninqj48zuV+hIlZ1wkMAXcE1ziM6eqSeijwzEPPsbY1UIje8D/lHHU4AMV5dThFn1jvEOsO/xDa0zsi7aHSili39H3sWbuMxg15AG7/7RUzLJHFsv+E3+7NnZOavKvmvJoLwXWQeJx1Ky8H18HfthMtT9oBpyrM4LX9bymIiAv8G2Zq4m3AvSJyg6o+UZHsWeClqtojIuswvoHTx7qeqn5aRG4HFgK3qkZCOsB7JpNnzigFTSWQ2lqIxSgubwGg0BRn1wscdEkerwQLb0wi85M0PNoJC9sotaWJP7gRwLw0QHlhI7f96Z9n7Dkm4ryaC8m/7Fh61yiNG2ZaGotlGvF8JJnYr0usW3wZOA5+dw9OYwMctZJCWy01T3dEaeSI5bBxi4m/PB1MT0vhNGCDqm4CEJHrMdMSR0pBVe+qSH83k0SUUNW7x9j3dDXCzBmlUGgVnrp8Ce6QoEvyJJIloMgZ7VtpTgzQX04xeFiSznwtW/rrGehPsWhBD3la2L6jmeSWBEtvHpjpxxiXdUeaFsHWc2P4DWUSO/uhx7QE1x57JTc/undQ2bWtl+L39cMpR+Nu7UAHBg+ZFpHl+cO6xZeN+16uW/o+ho5aSPLOJ7gld21UefP3dODMa8Pf04HE45ROXEWoUpx5bVAssvGCBryUEjtrEU4JUh3Qd0wZZ/B4eP/0yD6FlkKriNxXsX2NqobNokXA1opj2xinFRDwduCmqoWcInNGKSSTJc46/TGe7p1HyXPJJPO8oGUz6Viegh+nKT5IzkuxINXHic3bKfmmN25TfJC/JJfTNb+O/mdaaXqwi7Wtl05rM3Z/Wbf4MnrPXEbH37eROayH1roBnn3TElb80DO1o6YM6468An/rjsg3sbbpEiSThr7+kdc68go0EePmhz95cGRf+j7bo8oyLmtbL0VSKc5LX8QtuWs57+SPU25IEcsWAej4q2XEB5SaeW2sO/IKdrxqEbUdPm5xAQB1t3Sghy1h+5kpmuYvY9cLoeVhoX/dAFrK097Sx66uBmLJEsmaAmc27eG5XBPPTofwU+tu2qmqp4xzrOopiEXkZRil8OKq7zxFZlQpiMh7gHdjGnP/p6ofCvZfgXlwD7hMVSedQi4dK9Ce7GMwnWRJbQ8t8SwrEx1k/RRZr4a4lGlwh2iP9/Bkvp3WZJaSxohLmZMat3Lb4Bp61ghND4LU1x/Ix66adQveRd9LVlE6Zzl9rxrgFcufpuDHeLp3HoVWn+xx80j2lIh3DaIJJzqHWAwtldj+hmV4qWWUT84hj68g0QuL/vtJWDSfta2XogND3DJ03YF9CH92xwGwHDjWtl5qvqVyOaoYrFvwLrRc5ubOa1i39H1IfT3+HmPeOS99EaxeTqwvz6Y3NhIbgEKLIgvy7DltIWu+tpvaDp89pwpek4cMuOhfH0lNukBdqpPas4ZoL8c59oU76CnWsSDVR8GPsThtWtRNiQGW1XTRnBgwntf9RpHpGZhW1RTEInIc8F1gnap2TceNx2LGlEKg8c4HjlPVgojMC/YfBVyAmdCnHfiNiBwx2exrrvikY3mOzOwk6ZRISZm8xiOF0BbLEhdjSFyc6KbZzVHSGJuKbeS8JAOFBG2n7qL0m0ZKKxoP5KMDwcfR1jTC7LNu5eX4O3cjqSRaKjH08mPoOsahsKTEe4+6g4w7RN6PUx/Lc+dhMboWJyg+m6F2W4pCK5Rrm1hwl1LTWeTZ1yxnxfFb2dLZzCtXPsGdtStpT/ez4cxFDHbVcuQHdx7wZwS4advXD8p9LLOHdQveRXlV+4STuJ/jvJHY4kVoXZK+vz6ezuOh9WHYdXaZeQt7qS13s7Shl9ZUjqd753HWUc/wg8zp1K53WHDsLlbWd1H0YyScMktrulmR6iDr1ZDXGLlyipb4AF2lOlbX7mY981mQNC3mtliW9njv9D3s9HRJvRc4XERWANsx5d/fVCYQkaXAz4ELq/UN7Csz2VJ4J3C1qhYAVHVPsP984Ppg/7MisgHjiPnzZBcs+HGOTO0gr3G6y2kA4lIm4+bJuENkvRoy7hDN5Mj6Zuxdrpzi99sPo/xIA9sX17LS9am751nWLb4Mv7cPZ14bWpdEOnrwF7Vxy/2f2KeHXbfycjRTg1+bwH12uEBet+BdUFuL9vSipRKSSgJQPm0NO18YI3FkH29Y8QjNsRzd5TR9npH7xQs2sSvfwGBrJ4/Nbye2NYUsyLNjXQynL4nfUKY1Ncgxh+1keaqLwrzgr26FHal6+tYeGcmF66A9vdzceQ1rj/8YMljgpmc+v0/PaXn+sm7Bu9D5LUhtLQOLahg4+UhSvUr9xkHOO/nj3HL/J+hcdxiZbUV8V+hdGMctQd8KIXlEH32HCWe2b2XQS1AfH2JZTTdHpnawrKadI1M7+H374WwfbOOV8zaTlBLpWJ5UUNHL+3FTaSpnWJjopaQxjqs1ZvpljR3sKDXRFsuS1zglnUhdTYFp6pKqqmUReTdm7gMX+M9gWuJ3BMe/jZlGswX4logAlCcwR+0XM6kUjgDOFJFPA3ngclW9F+N0qfScbwv2TUhMfBrcITrKGVJOibiU2VFqBCDj5oPfIVJSIuWaLmkpp0Q2kaKhJk9ik9L6wx78dDTbKE5zEwCDKxqpSZisOveMT+Ju7+KmLV+tyvdwxlu+RGbzYFRrkmKZHW86nGIDtDzh4R/ZTKK/TOexixhY6hvrooKmPd568h0UNM4RNbsAaI7liEuZ1pgxfS1PdbG92ETtyhIPphaxoD5Hvhxj+eoeBstxXtC4KVKGAD3FOlpTOQbLcXozDvN+s52Ol7cTH1AaH3I5L30RsmopQGTjtVgm4ryaC3EazBjXoROWsev0BE7Q49MpwVCLALU0/34LZ772i9T3ldh+ZopSg9JyVCclFY5r7KKsDsfWb2dFsoMHcst4YWYD7bEeAOI1puA/tnkHi0/oY1Gih7zGWJ00lauSxsi4QwCsTJi6Za9XS6M7SDyYPqDRHYxk7vVqpy8DpmnwmqreCNw4at+3K9YvAS6ZlptNwgFVCiLyG2DBGIeuDO7dhJnB7VTgJyKykqk5XS4lGK3X0p6IzEOm1pCnLZY124EZqS3WT4szSL+fZEm8hw4vzcrkHl4+/2mee1cn5Xe4xByPP21ZifNIGo1BsclnzXFbiIlP57eW0/gnU/tY23QJuA5nrf0cNRu70FSc/KJ6YkMe8WfNi6lNGTiqkU2vq+Ptf/VnnhtqoadYR2dvmbMWbOLwt+6mpDE251tYnupi/eB8eop1HFZnzj+57lmyfg1ZL8XKpNmXcYYoaYysnyLvx1me7ODUuk2c2fQ0mwttrKnZQUc5Q3u8l5K6tMd76BCPdKyJlvgAOwoNtNf20/mabralF1F4UY7Y/WnKqTayb5pHTScs+OW0uOEsc5x1iy/DWbwQv7EOpyvLnpMSeMcMUPKMf8vvTKKZMqX6PD3nt1AcLLO0vZuzGneRdMqk3QIArfFc9O22xfr5+5Y7SYlHXl3aXI9V8UGeKmVYkOznmLrttMd7yPo1tLlZUo45r16K7AkK+xIuq+KdFHFJ4NHvJ8k4BTq8NB3lejLO0PRlwqwc0Lx/HFCloKqvGO+YiLwT+HkwsOIeEfGBVqp0ugTXv4YgwNPSY+q1Pd5Dt2fMRuEfn/WNMkhJiTgeDU6ZlHhkHMj6JZ7IL2JFsoOXpJ8ir3E2FebRctggi47q4YH+pfQWazi5aQsl3yX3oU62DTWx7duHUcwIuSWm+eieupB595cip5M/vwmvNs62s2p41evuZlGih+ZYjuNrttDm5kgs8ujyzQuc1zjLEsP9qDOZocjfkXKGB9lkJE+Hl4nMXiHhB3J8agtHpbaztdRCSsqRQgjzIu3maY1laY3n2F5s4qT5Hrv+up/6eIGdjaam17GjhXg2ib+wBacmxdrjP4Zfm+DWP39swv/5+cyZr/0iye4ipboYydseAODW0vUzLNWB57z0RciKxfQd08zAAgcvUY+e2s+a1k5aUzk682nWx+exqq2TZXXdtMQHuLtrOdmCsfenY/moI0hcPNpi/WS9GlJSoohLg5TZ4yXp8l16vVqyfg09pVqOTO1gVbyTLr+WJTHThbzPj9GvCeqdAimpcD0qpMSj6JSplyIJt4+8H4/KiOlgOgavzTZm0nz0v8DLgd+LyBFAAujETArxQxH5MsbRfDhwz2QXc8Wn0R0cUXvI+klSTolFbh/9fpKspoABkqJkfWFNPMvK2JP0+TEagnPa3Bwpp8S2YjNFP8bJTVsiP0VKSpCBj52/kIuP/DPt8V6eLbSRdErct2459z+zDByHfzj5L/yx83B+vPznkQIC6PCMEanfN36D8GUPlVYp3sPKQEF0eGnyftzIFOtnU2keWc+Yto5KbSeOx52DR9DoDpJyStQ7Bfr9JCkpsSzRET13EZesJDk7/Tgl3KjpvDzZQSltel/9X+lY6uMF8m1xOla0sDVVT/P6OjIP7oLaBOsWvGtGQoOsW/o+dHBwxAj1yv7sp//dl2n+7WZwnIPe7XXtsVci+RKDL1tIOZUk0efhpNNIMmG6A9fVzkkn++l/92Uaf/4w3smr2XCJQ83TDurAvJfsoCGRZ1FtLy3xAZbVdHNcw/aoFbAyuYez6x8HzHufckqsipmYbRvLzeT9OHmNE8czBbgIKadMCtNhpNtLc1L6OdYkdtHglGlzs4RGhZR4FNUlJR5JUQpq9jc4ZTKOS0aLZH2Trs3NjjAl7RcKeHNPKYjOkKYTkQTwn8AJmDCvl6vqb4NjVwJ/j+mq+j5VnXSgxjHHJfTXN7bQ58ei2kJ+lEOpiEvejzHPHTQ1CzF9oZOibC3XRfsBEnhR+hIuLc4gKfHY49XS4Rm/RUpK5DVORvKknDJZP0le45ESKuGScQrRfcJrhwV+aPPs8DIsifdE9wqvmdUUO0pNkTlsSbyLOB47yk20xfrpKNfTUc5wfGoL9U6BLr+WjFNgkVukqBrlRfih9GuCTcU28hqP/C25cooHepfQX0iRL8fIPthKvA+S/TDvrm40EUMfejIq8G7a9a2o0NOWBiQ7hDe/AXl0A1ooRLXk8041Dvlb7v34hP9b6Jy8+eFPmi6JgBYKuMuXsv1VC4kPQN2uMoVGl6ZfPYEkE/h9WZzli+k+tY30tgKJDbvRbC6Sr1pCpzowHPZkYduYAwFHyLz0fRRWL2Dby5LEj+5jcEuGmp0OfhyW3prD3WoUe89Zy2lYn500D2Y761ZebgaJrVrKc69pZmhJGbe+RPKRGtQF8WCo3ccpCurAuWc+SMGP0Z7sI+cliTs+y5MdrEzuib6NeqdAg1Omz4+xsdRKyimRkTxLYgMsjWXYUs5SUKHVdej0fPo1QYeXps3NUS9FFroxSvh0BhWG8HvOB8oBIONAWmLs9IaHL4ffwpqlO+/fX0dtQ127vuCof6gq7a33XbXf9ztYzJhSmG6OOS6hP/u/VmBYGRRxqZci/Zog78dYFR9ku5egzSmwtVxHKmhWhulDO+ZopVLEpc0pRDWQ7V6GOF7UIgFT0IdkJE8pcC2HDi+AOF6kYEKl0eKYY6E5qVJJbC01Rb0mwDjKM2Kc5ptK84xJTEytqVIBZRzTKunya2lxBqNrZ5wCW0tNpJwSeT/OI0NL2TjQRl2sQNIpc+eulfTnaijvqkHry1BwSD8To/XRIjUPPQfAwGkrqL1j/bhKQY5bPWLwkbO7B4pmXctl8qesItkxiNMzQGlJM/H120yPld1d6PwWBpfVI2Wle02cYhOoC97KIdLpIQaHksQfrKOmE+bdsYfta+eRW6aoqyy4Cxp/txGvqxtJJvFOXk18Ry+l9kbie7JoIgbbdyOxGH5uACddR88rDsMpKrG8T92THUYxLGxDOnogFoOaVKQsKjsUrDvyCvacOY8Xv/Meuot1DHoJHt25EK/k4myuITYA9c8pjU+a/83pyqL9/abPvuvAUB4yddz05Gf36V0/mJz98s+S2NxF7+kL6T7SYeVZm9nc1Yz/aD0tj/m4JaWYcXjF5XfSVaojHSvyyoaHyDgF8n6M23NHszjRzZk1G0fU4isrX5Ucm6gh5+dJSoycFtlWHuliDL/N8FrJcWJXJ0RIi1EcBfUpqpIQoRiUdyuW7JoepXDkpZMnBG69/xOHjFKYMyOaXTE1g6Q4bCqbArnNKdDhJ03BT4ynShnyGo9qHJUvZGjmyeCR9Y1CCJVKg5TZWq4z6Zxy9MKH5P0YHeV6SuqS9+Nk/VTUVQ5gh5RJOSVWJvawyM2yx6uNlEbYGqnENI2HfxvdwcjsY0xgsCQ+PHYl5ZTpKGVYGQtHLw9/SD/pPZWuUh2PdrfzTytvZVW8k+1eAymnxIpkB4sT3WS9Gvq8GtYuepK443FH6ype0LLZXOAF0FWq4+bHjiG2J47XXqD5nfPpz9Vw2PwO9gykmVfXyVNb1+DsTuKlPVYdvpNcOc72Hc2s/P4CsosT1O4uU7M9S8fxCRb/ZpDe0xfSe5jD0FtXoA7UbG0mNgjFJvBSinhKqSkoAOIe7fX9dMXqKLywAK5P1+uEuHbSFDxn/9I4g22HU9vh03OEQ+0eaHIFCZr3xXl15FevxvGUrqNc6nYqAwuFcp2grhA/fiHiLaThWZ/6DQmcnSZ/pa0Ff+sOM3IcIFNH96lt5JZA2i3QVDfIimQHg2Vj6htoS7C9o4k9a4T8m8r0bW0A6jjivxvIt6Uo1Tk03mEU7HmnfgJ5djvS1kK5ua6qmFvrDv8QFIqmZZRJ43f3VNVLbN3iy/YyZ1UOYAxbTaFiZGcH3uqleJk42/56Edk1ZV524mMcUbebDbtfSDwPXcc6oHDL33+e73SfwVtb/szxiSIlfJqcWkpaJqsmttjGcjNtbo68H4sqU3k/RsLxItMtQM7PU8InGWynxKPVdUYU7B2eS1LKkUJoduKUgiBEcZxIEZTwieNQwCfjBN/bxMOdps4cqVRXMmdaCkcfl9D/u7GFDj/J1lJTZI5JOeURpiCAeqdAEZcOLx2ZaSqVRGhyqbxWeI2sn9zr3ltLLWS9FG2xbGTWeWRwCU/lFlD0XFpTOVoSg5xat4m4lEc4i8PeROH+lJRoi/WPUDyhv2BJbICECNu9BAk8FseUOA47PfOBhMfuHDiCzlKap3ILuH/9cgDi6SLHLdrBmvQujqjZxQ0dJ9CWzPKyhqfIuEPsKDXxbL6N1niOX+w4noW1WXYOZjizbSNH1OyKHIFxKUeOuo5yhoyT50dbT+UtS+6lPd7Dlzady/ZtJiDhdS+7JjKrbS218OjgEv64cxVdvWmWz+/k8PoOCn6M+php/WwbaqLoufQVUwwUExzW2EVnvpZ8Oc5Z858h7ng0uEPk1eRL5H8pZyKH+6ODS3h2oIWi57Klr5GY69O1qwEZcKlf3gtAzPWZV5eLTGaNKdMpYUdvA87d9fgJyM/3ieUc3CGo2wk1XR7ukE9ucZzscjj2rGdYXNPDrnwDTYkBkkHBdnjNbjJunh2lRnYVGog7Pmk3z6+2HkPZcyh7LgO7TQXjiO/lcZ7ZGimFYkOC2ke2QbkMiQTalEF6spRWzCO+o9f4VlwHCkVIJqBQxO/uofDio0j95Rm0ZDomlE9bQ7xrkFJLLYmd/XiNtcak5QTOrboa/HQKeXY7OjCEHLYUHIe+oxppeMzY+f10iu6j6xicJyw8ZyuH13eQjhXJlYeD1n37ZDMafvO2hSyNjT31Ss7Pk9MyRTXfU0gCjzbXVMDMdwBZf7hiB1DQkYMA0mL+97CwB6Jrh4V+1vdIiIw4p4RPWhJ0+XmKOo3mo9p2fcHq/6+qtLc+9K+2pXCw8RE6/CR5P8aSoNdNpUJocwqEfXzCrmqrYt3GXMSwP6DNzdHvmZc3qymeK7axKt4Z3Sc09wCRLbS7nGZZooOjE7ujNI8MLqHWLRKTGN879XsAXP7wmzm2disldYmLR0pKkWnJDKoxXU0pQ6+UOTqxOyrs29xBQMj6RIoq6xcAj/c9+yY2drTy9Os/xsbNhxkzkzvEA71LkAHzsTxz8cha6EUmojgX33sxL2rYwLZiMz995kQa64a4Z+1nAHj9Xe9iYaIXgJ92nsrv/3IM2lDi9MM2sy3XwOJ0H02JATr603xnw4t4+FWf5GOPXQUx8zF3eBkaGaTFGSQfi1PyHc5pf4qn6hfQlszyaHc7AGfNf4YVqQ4Or9nNzmIjXSVTaJ6S2cy2YjNPZhdy8/YjecOyB8m4Q7Q7edpi/ZHvpS3WT69XS8YZ4tjarcaJrjE6G9JsGJjHixdsAiDu+OTKCQp+jKRTplAToyUxSNrNk3RK5FpS3N24nNpYidpYibI67BrIUPJcuoaS5AcTnLRiMzHxObPJDCptig+yq1BPU9z8j52BgkpJOjpW8h1e1r6Bku+wKddKd+0QHf1pNry5Do2tRl1FykI867CQxdQ+sBmAvqMayTc30/ZAjt5T5lNodBicB+5JfZQebyDZCYXWZaQ6YEHfMooNCfKtcdQBb2WS/pWCeClKGUVjtSS6HeY96JFrdyk0wIJ7kyTX76LUXEd2WYq+FcJgWyuFJsgv8CBd5PJTb+WZofnsyjfQUchQ6xa57vRhZ/+b//wO/nv5+JMwpp0U+HkQaHb8qBbf4qSM+cf1yfoeWd/4D7K+B/gkxYkK9DgOyUAhFLQcbYeFfEIkaiE0O3G6/RIJEZLikNMyC9y0abX40K97V+r2h7nY+2jOtBTWHJfS79ywOFIEQFQTCZ1QYXMz9CeE5qWwL3No0onjkdVUNOhrTcIMHqv0NTQ45eg6HV6aVbFuMg5s9xI8UWgn4wzx4z2n8+Mzvj1Czs8+8UpWJ3eSckqsz7dzfM1zbC218Gyhjd/uPoJdvfVkavMsb+jhZc1PcU7dU2QcuHFgFX/oWc0rmp+IRjY/mV241/UPBG/+8zt4qqsNgIdftXcgvdff9S6KnsuvzvxGVdfalmvgiMY9PN61kLjr8apFj3J67caodZTVVNSfvKQxft5xMn95aiUvO+ZJTs48Z0aoO0MjnPOVdJTrjXLFjFkpBS2LMEzIX/pWMuglWFzTQ9zxSUqJFakOMk6erJ8i4+TZUWqiz6vhxm1H01I7QGtygObEAAuSJo5Oe7yXHaVGnhsyraJlNV08PTA/WO9mRbKDrJci66fIlVPkvCT95RT9pRpTWXA8WhKDdBVrKfsuRT/GzsEMu3rrKQ7GSdSWOLF9O/2lJDv661na0Mui2l56inU0JQb4w5bD8HyHYlcKajxq0gWG+lPEa0skEmVSiRJ1iSKe79A7WENLeoCBYoLevjq8koMb9/E9IbEpRWF5gUULeujO1eI4ypq2PayoM+azNTU7+EPPajb1t+A6Rtn/8ewvjMhvf9cR1b1IQEnLdPl50hKL/AZhiyAtMXJaJikOcZwR6wBJiVHQkTGvc8F25TlG4VQ6l819sr5Hnx+jiMtpy56bhpbCQj3jsLdXlfaWRz99yLQU5oxSOOa4hP7g1/MjM1FoIkrgRb1yQqdyuL/BKdPhJ+nw0tE4hqymSElpxIjIsP9zWGsPe/ZU9k7KazzyK1xyxB2s++N7ueklX5tU7ssffjNfPP7HXPnI63gqt4CnOuZx0oJtzE9lObXO1HDzGueB3DIe7F5MvhRnddMe5qeyfPH4Hx/obD1gXHzvxewaqmeglOAjK2/ixcn+6APP+rDQNb1Gwk4CHV6G54ptxAP/THu8hzY3F6RPRiNXK6kc19FdTpsQJ26O54pt5DVGSspsLxqvRMl3WJ7qIi5ldhYbKWicpJTIeUke613Iic3bSYoxz8Qdj9ZYlmeG5keO5rZkNjIhLUj2cXqtsaU/kV9E1k/x3FALA+Ukg16CWBAb4cjMTjP+JeihA9ASz1Lw45GvJzSVZZw8neUMJd+lq1QXXau3kKIxmac+PsSuIRPIsei5HNO4k7jj01WspTOfpjWVi8x0u/INkVnOdXyOatxFOlZk62ATh9XticJDZP0UHeUMuXKKHQUze+HjXQujlmRIYedK4lK90aGkZQpaHmHWCU1HIUlxImVRqRhKY0xgUFB/hFIYbWbq9ksVaY1zeloczTUL9YzD/r6qtLc89hmrFA42Jxyf0BtvbB3hJA59CuFIxrZYPxnJM88dJOMQOZ7zfnzMWuciN8vGcjO39B7L4TXGNLQyuYej4l0UVLhtYA0A717z2wP6bGFNPeb6NKaGuP1lXz6g95tpvvnUy1mZ3ENKSlFX4H5NkPWTdJTrjdmooqVQ75hupUVcNhVNiybslVXvFNjuNbCpMM90e3RMwRiGQ6kseLtKxi6+O29+62LmumHEzQXJPgp+PPJrhEqlq1hrRujGiixK9ET3b4/3kHJK7Cg10VHO0FXKkA5CruS8FGk3z+KEseE/mzeBGdNuITLZbS60kZQScccjF4xR6SrWUh/Lk3YLtMZz5DVGVykTyQBE5jGA5waa2TNgfEDHte5gWU03rbFspGBWpDoi31DYogLoLKXZMtRMb7GG1lSOB3YvJhnz6MnW8swbR5oid2xvZ4E79QFhJS2T0+JeBX5B/ahgh+EWQrJiX1xikVMaRvoSKpWC2U5ESis8p3XR9ulRCiurVApPHDpKYe74FFTp8Fz6/SSbSvOigHjNsRw7So081LeUDd9bTfa8HK857DGWJ4dHEbfHe8yAGcfUNp8qLuDJfDs3bjuaTDK/VyH8+rvexSmNm9lVaOBrJ/7ogD/bj8/4NhffezFA5J+Yy1Qq2c8+8Upel3mIeilS7xajbr29Xi0JPPLEoi63YAr7rG+c/pUjV7O+MeU0J03rIuPmyXop4lKOQiwU3Dg9pVrqYgUKfiwqXGtjBTblWok7flToh73LWuJZSr4TmaHC9y0uZUoao1EGaY/30OzmyMZ7WRLvoterjboaR6FZEnE6yxkaXGMy6ywZ2XNeEjyilkR7so90zIRwyTh58hqn4MdJ1wStgEI9g+UkyUSZnmIdtbESqViZVKxE2XejZw9NcN1emoyTj+KDPTq4hPX989g9kKGrN43j+jzzxm+w4gef5dm37j3165WPvI6/a4IWpzyl1gKYgj0eFPYFLZOWhPEZyLCSSMuwYzs0CZXwQYsjrhX6EjKOG7UQKs8NyWmZ9kU7GDuaztSZiz6FOaMUEonjOX6pmdjozPESnTH27tNu/iirm0xsoUEvEdnHrzpm7PQ/e+HBH937fFAGY3HFUTfy2SdeScbJ0x7vYUm8h5LvclJyl+nzHrQEQt9BKajBbys2k/NSUW+ZMHTyjmCcRly8EYHUsl4NDe5Q1Ap4emA+LYlBtg42URsr0JgwoULCQryzbFoTpaCgDcOIgOkVlde4idDrFIj7Hhk3H3VYCE1d4XiR0MTV4A5F5rEwum8Y0TMc3Z5ySkbB+DWReSfplIZ7OsWKpGNFklKio5ChY6iOVKxEa2q4g0RoSks5JVa6e9hUnBfd557OpWx5ej5OwUHbCmy8wLQKxlIIIUlRunyjVFqc1Ajl0OOb+zY5YwehSzspSkFhP7o1QIWZyRw3LYjQ0ZwLFEO43RyYn3JaZpsnNDhD/CW/mI5yhscGFvFMv2lF3j5peM0pYJXC3GS0jdQyu7jiqBvH3P/zjScCZhxHWNMNa+hxKVPyXUpBN8ye0nChlHRKtMd7o5hTJY1F40ty5RRbhpppTgyQlBKH1e2JfAiV5qbQBANmvEJPqZaGIApvqDhSTika1Jj34+woG/9FGDKl5BmlFPo4QtNURvK0J3tGOMmb3RyN7iAd5Xq6gxHxJY3x3FDLiG69oRkpGS+xJr2LFXVxcuVEpCgagp5pAJsK8+gsZ9hVqI+OL6zNUnd0kSc3LqqqLv3p437OvVuWR4Mmc1qkSWKR3yCs/Yfk/LzpkVRBTotRutGO5LC1UNAyCCN8BmOZhS7ffg79pZqo4ra8imfYZ1RHhGCZK1ilYDlked2qB/nu06ZdGA7wa3QHyaupVZc0RkHjtMSz5MopCmrMQ03xQbJeakQBnnHzkc8htOln/RQFP07SKUUKIFkRpLDBHaLTT5OO5WmNG7NUXuNRkLdsKRWFKEkFTupK521kBnKHqOzlHwZzTDklUpSiUCc7Sk1k3Dxx8cjETA+p7YON7B4wZ+f+2MbCu/Pc/tsrOO3mj/KShRtpiWdJu/Fo4ikYbnWELEj2U/JdFiZ6WZHq4Lauozl9zSY6/mU5vHXy/yHvx2iLFSgqgE+PP1jR7dTcs8cfjBzBpWA9KTHiEosK97CADxVKUmJRawGMEmhyhouskpbp8Qd5uFjH7f2n0FWqoz6WP7itattSsFhmBw9vWUIRl4zTFo38TjmlKNhaxh2iWeM0x3KmQE6EY0HMGBEgCj8eKoMl8S46yqYHT17jkd8h66ciZZCSMs2xXOSPWJEyrY1txWYA1g/Oj2rdoaLYUWrcy5Ebdzy6Shl2FRo4vGY37fGeqIUTKriSxugoZyhpjPWDprvrKZnNrE6YeQQ6yhlqYyVaagdMT7e1w/kzUev31Xe8hxe2bKQtluW4mi3clTuC3fkMT+UW8LMXfov77r+Q7YONPPu26gq8sHNGQYWCQocKi2NOxfHh2nSLk4rMPjktQuBoDruphv6AXIXZqLJFAEYZ/KkQ5/b+E8h5Sb524o94eVWSHgCsUrBYZp7vPn0mpwYWiPZYz3AMKj9Pyi1F4x0y8XzklC5pjIwzFIUjz/vxKKJuOMAxgRk70O8nR4RigOGCzwQTNH6JjnKGncXGqPAv+S7Lakz/fjMdrJkaMu540ViFBck+2mJZ/tR3GIPlJD8+49u8+c/vYEmtcWAfW7uV9YWF0XmtsSydpTQdhQyb+5q4+YmjOXLZTt7WfhfHp7ZAk/FvvPqO91Q1TgTYK933Rp0bjlSuhovvvZgL24bH/iTwSInHtrJLSvIjRigbR7AZvJbV4d5CBUwIi51+2YxsVo+M40bKAYoU/DzbPdMD7ca+U9k62HRQxuhMiALTM0fzrMIqBcshxZWPvI4XZXpIiUcKj3q3GHVXDeNDhd2MITDBOCWWuKbQjSZdCXwQMDzCPVwPQ3PE8ZjnDpIUMy4lJGxdxMWDOEHroxwNfAtNQjtKTbTHe8l6KXYWG0m7hcg/UjmF1lQKt6N/eRVbeprY1NpGxh2iLZbljp4jWL9n3j7lJ+ytJKZCZz5NXuO0jBOOOutDNmopCPheEBRP6PPdKOZYGCgv60NSGKE4tpWFe/OruKnzWGBmOnqMjYJan4LFMmN886mXk46lRtjEN5ab2VFq4qmhdtJungZ3KJpcKOuniPsmnEhKTAsiLPRHB2ajYuBUqBDC8Q8FlRERdNvcLOuLCwGIi0fej7Oz2EjOS/LH7OHUxwssSPUdkO7Kj59/FWB6zN1Vt4pfnfkN3nzYtN+marb0NXJt7MWc0/I4qxM7mRcohxGT3TAc1TQs/IEoEF5RIeMMRzAN5zcJWxIPFZbw1FA7bcnslFoxBwVrPrJYZo7TazfR5hSCqJgO28pCR7meZ/Omq2HOM7b/jsApnHJKlNQl4wyR1zi9pdrIfJSSkvEXBNO0hpXZMApu1k+aaLKBMgGi1sj6wkKeyLazqb+FP53zuYOfEcyeHnOjw568+o730JrKsbbpsWhCnJDK0NkwPLp4LDr8JA/nl7Ct2EzJd2fn6H3F9j6yWGaSsHtnn+9RJEaHl2ZHyYSkyJUTJJ0yuwoN5NxUFFE1LmVwh3vcRK0MxyiGtoTxB4R+hXD0+9NDC+gq1UWTxWwYmBeFxy6rU1UIk+cjlaaoi++9mJbEIC+vfzyaHKcy2sA8dzCa/Ck0320stZL1ayipS0c5w1XH/HKmHqU6bEth+hCRE4BvAynMDGvvUtV7gmNXAG8HPOAyVb1lpuS0zB5eueKxaCDb/dllfO/U7/HKUWlOvulKUrEy+XIs+p1Xl4tMOsfWbSMuHt1eOgrvEI4/uLXjSDZ2tFIcjEPZQRIe2h9n8zsvn5HnPdSp7Boa/m9gzHq7Cg10F+uIOR71sTxxx58dzuMpoVYpTDOfBz6hqjeJyCuD7bNE5CjgAuBozBzNvxGRI1Sne3YMy6HIeAPZQu5fN/FUmhPx/iP3+VTLJEz2vx2SKODPPfORM3mSA4YC9cF6A7AjWD8fuF5VC6r6LLABOG0G5LNYLJaJUa1uOYSYyZbC+4BbROSLGOX0wmD/IuDuinTbgn17ISKXApcCLF269IAJarFYLGNyiBX41XBAlYKI/AZYMMahK4Gzgfer6s9E5E3AfwCvYOzwhWPmvKpeA1wDcMopp8y9f8discxeVFFv7lm1D6hSUNVXjHdMRP4beG+w+VMgnONvG7CkIulihk1LFovFMnuYgyOaZ9KnsAN4abD+cuCZYP0G4AIRSYrICuBw4J4ZkM9isVgmxvoUppX/D/iaiMSAPIFvQFUfF5GfAE9guqr+o+15ZLFYZh2qc7L30YwpBVW9Ezh5nGOfBva9b6HFYrEcDA6xVkA12BHNFovFsk9YR7PFYrFYQmzobIvFYrGMwIbOtlgsFguYhoLaloLFYrFYgKC7qW0pWCwWiyVgLrYUROdIlyoRyQLrZ1qOUbQCnTMtxBjMRrmsTNVhZaqeieRapqpt+3NxEbk5uEc1dKrq2v2538FiLimF+1T1lJmWo5LZKBPMTrmsTNVhZaqe2SrXbGcmw1xYLBaLZZZhlYLFYrFYIuaSUrhmpgUYg9koE8xOuaxM1WFlqp7ZKtesZs74FCwWi8Wy/8ylloLFYrFY9hOrFCwWi8USMSeUgoisFZH1IrJBRD4yg3JsFpFHReQhEbkv2NcsIreJyDPBb9MBluE/RWSPiDxWsW9cGUTkiiDf1ovIeQdRpqtEZHuQVw+JyCsPskxLROR3IvKkiDwuIu8N9s9YXk0g00znVUpE7hGRhwO5PhHsn8m8Gk+mGc2rOYGqHtIL4AIbgZVAAngYOGqGZNkMtI7a93ngI8H6R4DPHWAZXgKcBDw2mQzAUUF+JYEVQT66B0mmq4DLx0h7sGRaCJwUrGeAp4N7z1heTSDTTOeVAOlgPQ78BXjBDOfVeDLNaF7NhWUutBROAzao6iZVLQLXA+fPsEyVnA9cG6xfC/z1gbyZqv4R6K5ShvOB61W1oKrPAhsw+XkwZBqPgyXTTlV9IFjPAk8Ci5jBvJpApvE4WHmlqpoLNuPBosxsXo0n03gclLyaC8wFpbAI2FqxvY2JP6QDiQK3isj9InJpsG++qu4E89ED82ZArvFkmOm8e7eIPBKYl0LTw0GXSUSWAydiapuzIq9GyQQznFci4orIQ8Ae4DZVnfG8GkcmmCXv1aHKXFAKMsa+mepn+yJVPQlYB/yjiLxkhuSolpnMu38HVgEnADuBL82ETCKSBn4GvE9V+ydKOsa+AyLXGDLNeF6pqqeqJwCLgdNE5JgJkh8UucaRacbz6lBnLiiFbcCSiu3FwI6ZEERVdwS/e4BfYJqnu0VkIUDwu2cGRBtPhhnLO1XdHXzUPvAdhpvyB00mEYljCt8fqOrPg90zmldjyTQb8ipEVXuB3wNrmSXvVaVMsymvDlXmglK4FzhcRFaISAK4ALjhYAshInUikgnXgXOBxwJZLgqSXQT88mDLNoEMNwAXiEhSRFYAhwP3HAyBwsIk4LWYvDpoMomIAP8BPKmqX644NGN5NZ5MsyCv2kSkMVivAV4BPMXM5tWYMs10Xs0JZtrTPR0L8EpMT42NwJUzJMNKTO+Gh4HHQzmAFuB24Jngt/kAy/EjTLO5hKkdvX0iGYArg3xbD6w7iDJdBzwKPIL5YBceZJlejDEfPAI8FCyvnMm8mkCmmc6r44AHg/s/BvzLZO/2Qcir8WSa0byaC4sNc2GxWCyWiLlgPrJYLBbLNGGVgsVisVgirFKwWCwWS4RVChaLxWKJsErBYrFYLBFWKVgsFoslwiqF5wEikps81X5d/0YRaQyWd+3D+WeJyK+nmL5PRG4c5/h/icgbpirHoUiQFy+s2H6/iGwRkW/OpFyWQxerFCz7jaq+Uk2ogUZgykphH7lDVV85ebJ9R0RiB/L608RZQKQUVPUrwL/MmDSWQx6rFJ6niMgJInJ3EE3yF2E0SRH5vYh8LpjA5GkROTPYXysiPwnS/1hE/iIipwTHNotIK3A1sCqY3OQLo1sAIvJNEXlbsL5WRJ4SkTuB11WkqQuiW94rIg+KyKRh0MXwTRF5QkT+j4pItCJysoj8IYhce0tFrJ5Tg2f5cyDrY8H+t4nIT0XkV5iIt2PKIyZC5xeC/Y+IyD8E+xeKyB+DPHgszL9x5D43uP8DwT3Twf5/Ca77mIhcE4S/QEQuC57xERG5Xkwk1XcA7w/uN+69LJaqmekh1XY58AuQG2PfI8BLg/V/Bb4arP8e+FKw/krgN8H65cD/C9aPAcrAKcH2ZqAVWM7IiXTOAn5dsf1N4G1AChPG+HBM9MqfhOmAzwB/G6w3YsKX1I2SffR1XwfchplwqR3oBd6AibF/F9AWpHsz8J/B+mPAC4P1q0O5A/m2EYRsGE8e4FLgn4P9SeA+zOQt/8RwiBMXyIzzn7QCfwyfDfgww6EaKsNFXAe8OljfASRDWYLfqxg1qUzwDN+c6ffOLofmcig0jy3TjIg0YAqVPwS7rgV+WpEkjBh6P6agBxOX52sAqvqYiDyyHyKsAZ5V1WcCeb6PKWTBBBJ8jYhcHmyngKWYCWfG4yXAj1TVA3aIyG+D/asxCuy2oLLtAjuDQGoZVb0rSPdD4FUV17tNVcNJgcaT51zguArfRQNGyd0L/KeYaKf/q6oPjSPzCzCzgf0pkC0B/Dk49jIR+RBQCzRjYmn9CqPIfyAi/wv87wT5YbHsM1YpWMaiEPx6DL8jY8Wjn4wyI02UqYr18YJuCfB6VV0/xXuNdT0BHlfVM0bsnHye7IHJ5AlMOu9R1Vv2uqmZR+OvgOtE5Auq+t/jyHabqr5l1Lkp4FuYVthWEbmK4Xz7K4wCfA3wMRE5epLnsFimjPUpPA9R1T6gp8IGfSHwhwlOAbgTeBOAiBwFHDtGmixmbuGQ54CjxIQrbgDODvY/BawQkVXBdmXBeAvwngo7+olVPNIfMWGR3cBn8LJg/3qgTUTOCK4VF5GjVbUHyIrIC4J0F0xw7fHkuQV4Z9AiQESOCPwPy4A9qvodTBjsk8a57t3Ai0TksOD8WhE5gmEF0Bn4GN4QHHeAJar6O+BDGFNWmr3z3GLZL2xL4flBrYhsq9j+Mib+/bdFpBbYBFw8yTW+BVwbmI3CkMV9lQlUtUtE/hQ4bW9S1Q+KyE+CtM8E56GqeTHTlf6fiHRiFE44k9cnga8CjwQF8WZGmnbG4hfAyzEhk58mUHCqWgzMO18PlFIsuPbjmPDd3xGRAYwfpW/vy04oz3cxprUHgv0dmDmKzwI+KCIlIAf83VgXVdUOMU73H4lIMtj9z6r6tIh8J3iWzRhzFBjT1/eD5xDgK6raGzjE/ydwgL9HVe+YJK8slgmxobMtVSEiLhAPCvRVmPj5R6hqcQZkOQvjXJ1MWUx0jbQGE7+LyEcwcfffOz0SziyBsjlFVd8907JYDj1sS8FSLbXA7wJziQDvnAmFEFAEjhGRG3Xfxyr8lYhcgfkGnsP02DnkEZH3Y7qp/mymZbEcmtiWgsVygBGRv2C6rVZyoao+OhPyWCwTYZWCxWKxWCJs7yOLxWKxRFilYLFYLJYIqxQsFovFEmGVgsVisVgi/n96ZaRBJFcILAAAAABJRU5ErkJggg==\n", + "text/plain": [ + "
" + ] + }, + "metadata": { + "needs_background": "light" + }, + "output_type": "display_data" + } + ], + "source": [ + "ds_chunk.ice[-1].plot()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "The data look good!\n", + "Now let's try a random chunk from the middle." + ] + }, + { + "cell_type": "code", + "execution_count": 21, + "metadata": {}, + "outputs": [], + "source": [ + "chunk_number = 500\n", + "for input_file in recipe.inputs_for_chunk(chunk_number):\n", + " recipe.cache_input(input_file)" + ] + }, + { + "cell_type": "code", + "execution_count": 22, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 720, lon: 1440, time: 20, zlev: 1)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n",
+       "  * lon      (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n",
+       "  * time     (time) datetime64[ns] 2009-01-17T12:00:00 ... 2009-02-05T12:00:00\n",
+       "  * zlev     (zlev) float32 0.0\n",
+       "Data variables:\n",
+       "    anom     (time, zlev, lat, lon) float32 nan nan nan nan ... 0.08 0.08 0.08\n",
+       "    err      (time, zlev, lat, lon) float32 nan nan nan nan ... 0.3 0.3 0.3 0.3\n",
+       "    ice      (time, zlev, lat, lon) float32 nan nan nan nan ... 1.0 1.0 1.0 1.0\n",
+       "    sst      (time, zlev, lat, lon) float32 nan nan nan ... -1.72 -1.72 -1.72\n",
+       "Attributes:\n",
+       "    title:                      NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...\n",
+       "    Description:                Reynolds, et al.(2007) Daily High-resolution ...\n",
+       "    source:                     ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n",
+       "    id:                         oisst-avhrr-v02r01.20090117.nc\n",
+       "    naming_authority:           gov.noaa.ncei\n",
+       "    summary:                    NOAAs 1/4-degree Daily Optimum Interpolation ...\n",
+       "    cdm_data_type:              Grid\n",
+       "    history:                    Final file created using preliminary as first...\n",
+       "    date_modified:              2020-05-08T19:05:13Z\n",
+       "    date_created:               2020-05-08T19:05:13Z\n",
+       "    product_version:            Version v02r01\n",
+       "    processing_level:           NOAA Level 4\n",
+       "    institution:                NOAA/National Centers for Environmental Infor...\n",
+       "    creator_url:                https://www.ncei.noaa.gov/\n",
+       "    creator_email:              oisst-help@noaa.gov\n",
+       "    keywords:                   Earth Science > Oceans > Ocean Temperature > ...\n",
+       "    keywords_vocabulary:        Global Change Master Directory (GCMD) Earth S...\n",
+       "    platform:                   Ships, buoys, Argo floats, MetOp-A, MetOp-B\n",
+       "    platform_vocabulary:        Global Change Master Directory (GCMD) Platfor...\n",
+       "    instrument:                 Earth Remote Sensing Instruments > Passive Re...\n",
+       "    instrument_vocabulary:      Global Change Master Directory (GCMD) Instrum...\n",
+       "    standard_name_vocabulary:   CF Standard Name Table (v40, 25 January 2017)\n",
+       "    geospatial_lat_min:         -90.0\n",
+       "    geospatial_lat_max:         90.0\n",
+       "    geospatial_lon_min:         0.0\n",
+       "    geospatial_lon_max:         360.0\n",
+       "    geospatial_lat_units:       degrees_north\n",
+       "    geospatial_lat_resolution:  0.25\n",
+       "    geospatial_lon_units:       degrees_east\n",
+       "    geospatial_lon_resolution:  0.25\n",
+       "    time_coverage_start:        2009-01-17T00:00:00Z\n",
+       "    time_coverage_end:          2009-01-17T23:59:59Z\n",
+       "    metadata_link:              https://doi.org/10.25921/RE9P-PT57\n",
+       "    ncei_template_version:      NCEI_NetCDF_Grid_Template_v2.0\n",
+       "    comment:                    Data was converted from NetCDF-3 to NetCDF-4 ...\n",
+       "    sensor:                     Thermometer, AVHRR\n",
+       "    Conventions:                CF-1.6, ACDD-1.3\n",
+       "    references:                 Reynolds, et al.(2007) Daily High-Resolution-...
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 720, lon: 1440, time: 20, zlev: 1)\n", + "Coordinates:\n", + " * lat (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n", + " * lon (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n", + " * time (time) datetime64[ns] 2009-01-17T12:00:00 ... 2009-02-05T12:00:00\n", + " * zlev (zlev) float32 0.0\n", + "Data variables:\n", + " anom (time, zlev, lat, lon) float32 nan nan nan nan ... 0.08 0.08 0.08\n", + " err (time, zlev, lat, lon) float32 nan nan nan nan ... 0.3 0.3 0.3 0.3\n", + " ice (time, zlev, lat, lon) float32 nan nan nan nan ... 1.0 1.0 1.0 1.0\n", + " sst (time, zlev, lat, lon) float32 nan nan nan ... -1.72 -1.72 -1.72\n", + "Attributes:\n", + " title: NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...\n", + " Description: Reynolds, et al.(2007) Daily High-resolution ...\n", + " source: ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n", + " id: oisst-avhrr-v02r01.20090117.nc\n", + " naming_authority: gov.noaa.ncei\n", + " summary: NOAAs 1/4-degree Daily Optimum Interpolation ...\n", + " cdm_data_type: Grid\n", + " history: Final file created using preliminary as first...\n", + " date_modified: 2020-05-08T19:05:13Z\n", + " date_created: 2020-05-08T19:05:13Z\n", + " product_version: Version v02r01\n", + " processing_level: NOAA Level 4\n", + " institution: NOAA/National Centers for Environmental Infor...\n", + " creator_url: https://www.ncei.noaa.gov/\n", + " creator_email: oisst-help@noaa.gov\n", + " keywords: Earth Science > Oceans > Ocean Temperature > ...\n", + " keywords_vocabulary: Global Change Master Directory (GCMD) Earth S...\n", + " platform: Ships, buoys, Argo floats, MetOp-A, MetOp-B\n", + " platform_vocabulary: Global Change Master Directory (GCMD) Platfor...\n", + " instrument: Earth Remote Sensing Instruments > Passive Re...\n", + " instrument_vocabulary: Global Change Master Directory (GCMD) Instrum...\n", + " standard_name_vocabulary: CF Standard Name Table (v40, 25 January 2017)\n", + " geospatial_lat_min: -90.0\n", + " geospatial_lat_max: 90.0\n", + " geospatial_lon_min: 0.0\n", + " geospatial_lon_max: 360.0\n", + " geospatial_lat_units: degrees_north\n", + " geospatial_lat_resolution: 0.25\n", + " geospatial_lon_units: degrees_east\n", + " geospatial_lon_resolution: 0.25\n", + " time_coverage_start: 2009-01-17T00:00:00Z\n", + " time_coverage_end: 2009-01-17T23:59:59Z\n", + " metadata_link: https://doi.org/10.25921/RE9P-PT57\n", + " ncei_template_version: NCEI_NetCDF_Grid_Template_v2.0\n", + " comment: Data was converted from NetCDF-3 to NetCDF-4 ...\n", + " sensor: Thermometer, AVHRR\n", + " Conventions: CF-1.6, ACDD-1.3\n", + " references: Reynolds, et al.(2007) Daily High-Resolution-..." + ] + }, + "execution_count": 22, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "ds_chunk = recipe.open_chunk(chunk_number)\n", + "ds_chunk" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Step 7: Try writing data\n", + "\n", + "Now that we can see our chunks opening correctly, we are ready to try writing data to our target.\n", + "\n", + "First we need to prepare the target." + ] + }, + { + "cell_type": "code", + "execution_count": 23, + "metadata": {}, + "outputs": [], + "source": [ + "recipe.prepare_target()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "We should now see a Zarr group at the target location.\n", + "Only the coordinates have been written, not the data variables." + ] + }, + { + "cell_type": "code", + "execution_count": 24, + "metadata": {}, + "outputs": [ + { + "name": "stdout", + "output_type": "stream", + "text": [ + "/\n", + " ├── anom (14372, 1, 720, 1440) float32\n", + " ├── err (14372, 1, 720, 1440) float32\n", + " ├── ice (14372, 1, 720, 1440) float32\n", + " ├── lat (720,) float32\n", + " ├── lon (1440,) float32\n", + " ├── sst (14372, 1, 720, 1440) float32\n", + " ├── time (14372,) int64\n", + " └── zlev (1,) float32\n" + ] + } + ], + "source": [ + "import zarr\n", + "zgroup = zarr.open(target_dir.name)\n", + "print(zgroup.tree())" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Let's examine one of the data variables." + ] + }, + { + "cell_type": "code", + "execution_count": 25, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
Name/sst
Typezarr.core.Array
Data typefloat32
Shape(14372, 1, 720, 1440)
Chunk shape(20, 1, 720, 1440)
OrderC
Read-onlyFalse
CompressorBlosc(cname='lz4', clevel=5, shuffle=SHUFFLE, blocksize=0)
Store typezarr.storage.DirectoryStore
No. bytes59603558400 (55.5G)
No. bytes stored611
Storage ratio97550832.1
Chunks initialized0/719
" + ], + "text/plain": [ + "Name : /sst\n", + "Type : zarr.core.Array\n", + "Data type : float32\n", + "Shape : (14372, 1, 720, 1440)\n", + "Chunk shape : (20, 1, 720, 1440)\n", + "Order : C\n", + "Read-only : False\n", + "Compressor : Blosc(cname='lz4', clevel=5, shuffle=SHUFFLE, blocksize=0)\n", + "Store type : zarr.storage.DirectoryStore\n", + "No. bytes : 59603558400 (55.5G)\n", + "No. bytes stored : 611\n", + "Storage ratio : 97550832.1\n", + "Chunks initialized : 0/719" + ] + }, + "execution_count": 25, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "zgroup['sst'].info" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "Now let's write the first chunk." + ] + }, + { + "cell_type": "code", + "execution_count": 26, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
Name/sst
Typezarr.core.Array
Data typefloat32
Shape(14372, 1, 720, 1440)
Chunk shape(20, 1, 720, 1440)
OrderC
Read-onlyFalse
CompressorBlosc(cname='lz4', clevel=5, shuffle=SHUFFLE, blocksize=0)
Store typezarr.storage.DirectoryStore
No. bytes59603558400 (55.5G)
No. bytes stored39402334 (37.6M)
Storage ratio1512.7
Chunks initialized1/719
" + ], + "text/plain": [ + "Name : /sst\n", + "Type : zarr.core.Array\n", + "Data type : float32\n", + "Shape : (14372, 1, 720, 1440)\n", + "Chunk shape : (20, 1, 720, 1440)\n", + "Order : C\n", + "Read-only : False\n", + "Compressor : Blosc(cname='lz4', clevel=5, shuffle=SHUFFLE, blocksize=0)\n", + "Store type : zarr.storage.DirectoryStore\n", + "No. bytes : 59603558400 (55.5G)\n", + "No. bytes stored : 39402334 (37.6M)\n", + "Storage ratio : 1512.7\n", + "Chunks initialized : 1/719" + ] + }, + "execution_count": 26, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "recipe.store_chunk(all_chunks[0])\n", + "zgroup['sst'].info" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "We can see that one of the chunks has been written! 🎉" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "We can also open the dataset with xarray" + ] + }, + { + "cell_type": "code", + "execution_count": 27, + "metadata": {}, + "outputs": [ + { + "data": { + "text/html": [ + "
\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "
<xarray.Dataset>\n",
+       "Dimensions:  (lat: 720, lon: 1440, time: 14372, zlev: 1)\n",
+       "Coordinates:\n",
+       "  * lat      (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n",
+       "  * lon      (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n",
+       "  * time     (time) datetime64[ns] 1981-09-01T12:00:00 ... NaT\n",
+       "  * zlev     (zlev) float32 0.0\n",
+       "Data variables:\n",
+       "    anom     (time, zlev, lat, lon) float32 dask.array<chunksize=(20, 1, 720, 1440), meta=np.ndarray>\n",
+       "    err      (time, zlev, lat, lon) float32 dask.array<chunksize=(20, 1, 720, 1440), meta=np.ndarray>\n",
+       "    ice      (time, zlev, lat, lon) float32 dask.array<chunksize=(20, 1, 720, 1440), meta=np.ndarray>\n",
+       "    sst      (time, zlev, lat, lon) float32 dask.array<chunksize=(20, 1, 720, 1440), meta=np.ndarray>\n",
+       "Attributes:\n",
+       "    Conventions:                CF-1.6, ACDD-1.3\n",
+       "    cdm_data_type:              Grid\n",
+       "    comment:                    Data was converted from NetCDF-3 to NetCDF-4 ...\n",
+       "    creator_email:              oisst-help@noaa.gov\n",
+       "    creator_url:                https://www.ncei.noaa.gov/\n",
+       "    date_created:               2020-05-08T19:05:13Z\n",
+       "    date_modified:              2020-05-08T19:05:13Z\n",
+       "    geospatial_lat_max:         90.0\n",
+       "    geospatial_lat_min:         -90.0\n",
+       "    geospatial_lat_resolution:  0.25\n",
+       "    geospatial_lat_units:       degrees_north\n",
+       "    geospatial_lon_max:         360.0\n",
+       "    geospatial_lon_min:         0.0\n",
+       "    geospatial_lon_resolution:  0.25\n",
+       "    geospatial_lon_units:       degrees_east\n",
+       "    history:                    Final file created using preliminary as first...\n",
+       "    id:                         oisst-avhrr-v02r01.19810901.nc\n",
+       "    institution:                NOAA/National Centers for Environmental Infor...\n",
+       "    instrument:                 Earth Remote Sensing Instruments > Passive Re...\n",
+       "    instrument_vocabulary:      Global Change Master Directory (GCMD) Instrum...\n",
+       "    keywords:                   Earth Science > Oceans > Ocean Temperature > ...\n",
+       "    keywords_vocabulary:        Global Change Master Directory (GCMD) Earth S...\n",
+       "    metadata_link:              https://doi.org/10.25921/RE9P-PT57\n",
+       "    naming_authority:           gov.noaa.ncei\n",
+       "    ncei_template_version:      NCEI_NetCDF_Grid_Template_v2.0\n",
+       "    platform:                   Ships, buoys, Argo floats, MetOp-A, MetOp-B\n",
+       "    platform_vocabulary:        Global Change Master Directory (GCMD) Platfor...\n",
+       "    processing_level:           NOAA Level 4\n",
+       "    product_version:            Version v02r01\n",
+       "    references:                 Reynolds, et al.(2007) Daily High-Resolution-...\n",
+       "    sensor:                     Thermometer, AVHRR\n",
+       "    source:                     ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n",
+       "    standard_name_vocabulary:   CF Standard Name Table (v40, 25 January 2017)\n",
+       "    summary:                    NOAAs 1/4-degree Daily Optimum Interpolation ...\n",
+       "    time_coverage_end:          1981-09-01T23:59:59Z\n",
+       "    time_coverage_start:        1981-09-01T00:00:00Z\n",
+       "    title:                      NOAA/NCEI 1/4 Degree Daily Optimum Interpolat...
" + ], + "text/plain": [ + "\n", + "Dimensions: (lat: 720, lon: 1440, time: 14372, zlev: 1)\n", + "Coordinates:\n", + " * lat (lat) float32 -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88\n", + " * lon (lon) float32 0.125 0.375 0.625 0.875 ... 359.1 359.4 359.6 359.9\n", + " * time (time) datetime64[ns] 1981-09-01T12:00:00 ... NaT\n", + " * zlev (zlev) float32 0.0\n", + "Data variables:\n", + " anom (time, zlev, lat, lon) float32 dask.array\n", + " err (time, zlev, lat, lon) float32 dask.array\n", + " ice (time, zlev, lat, lon) float32 dask.array\n", + " sst (time, zlev, lat, lon) float32 dask.array\n", + "Attributes:\n", + " Conventions: CF-1.6, ACDD-1.3\n", + " cdm_data_type: Grid\n", + " comment: Data was converted from NetCDF-3 to NetCDF-4 ...\n", + " creator_email: oisst-help@noaa.gov\n", + " creator_url: https://www.ncei.noaa.gov/\n", + " date_created: 2020-05-08T19:05:13Z\n", + " date_modified: 2020-05-08T19:05:13Z\n", + " geospatial_lat_max: 90.0\n", + " geospatial_lat_min: -90.0\n", + " geospatial_lat_resolution: 0.25\n", + " geospatial_lat_units: degrees_north\n", + " geospatial_lon_max: 360.0\n", + " geospatial_lon_min: 0.0\n", + " geospatial_lon_resolution: 0.25\n", + " geospatial_lon_units: degrees_east\n", + " history: Final file created using preliminary as first...\n", + " id: oisst-avhrr-v02r01.19810901.nc\n", + " institution: NOAA/National Centers for Environmental Infor...\n", + " instrument: Earth Remote Sensing Instruments > Passive Re...\n", + " instrument_vocabulary: Global Change Master Directory (GCMD) Instrum...\n", + " keywords: Earth Science > Oceans > Ocean Temperature > ...\n", + " keywords_vocabulary: Global Change Master Directory (GCMD) Earth S...\n", + " metadata_link: https://doi.org/10.25921/RE9P-PT57\n", + " naming_authority: gov.noaa.ncei\n", + " ncei_template_version: NCEI_NetCDF_Grid_Template_v2.0\n", + " platform: Ships, buoys, Argo floats, MetOp-A, MetOp-B\n", + " platform_vocabulary: Global Change Master Directory (GCMD) Platfor...\n", + " processing_level: NOAA Level 4\n", + " product_version: Version v02r01\n", + " references: Reynolds, et al.(2007) Daily High-Resolution-...\n", + " sensor: Thermometer, AVHRR\n", + " source: ICOADS, NCEP_GTS, GSFC_ICE, NCEP_ICE, Pathfin...\n", + " standard_name_vocabulary: CF Standard Name Table (v40, 25 January 2017)\n", + " summary: NOAAs 1/4-degree Daily Optimum Interpolation ...\n", + " time_coverage_end: 1981-09-01T23:59:59Z\n", + " time_coverage_start: 1981-09-01T00:00:00Z\n", + " title: NOAA/NCEI 1/4 Degree Daily Optimum Interpolat..." + ] + }, + "execution_count": 27, + "metadata": {}, + "output_type": "execute_result" + } + ], + "source": [ + "ds = xr.open_zarr(target_dir.name)\n", + "ds" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "There should be data at the beginning..." + ] + }, + { + "cell_type": "code", + "execution_count": 28, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "" + ] + }, + "execution_count": 28, + "metadata": {}, + "output_type": "execute_result" + }, + { + "data": { + "image/png": "iVBORw0KGgoAAAANSUhEUgAAAZYAAAEXCAYAAACOFGLrAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjMuMiwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy8vihELAAAACXBIWXMAAAsTAAALEwEAmpwYAAD/rklEQVR4nOy9eZxkWVUn/j33LRGRkZXVVdV7021DS6/Y0Gw9zS4Cg6iIyqCgjuDCDCMuoCguP0HUAdHBwVF77LEFZcANRkURZBdEZG22ptnX3quqq3KJjIj33r3n98e55777XkZmRlZG1kaezyc+EfHW++5776zfcw4xM3Zpl3Zpl3Zpl2ZF5kQPYJd2aZd2aZdOL9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLLu0S7u0S7s0U9oVLBER0UVEtEJEyYkeyy7t0k4QEV1MRExE6Ykeyy6dvvQNLViI6CtE9Dj9z8xfY+Z5ZrYnclwbERE9l4g+TERjInr1hPU/TkRf8ALyLUR0frSuQ0T/m4juIqJ7iOgfiOiCaP1vENEniagiohdPMZaLiehdRLRKRJ+J55KEfoWIvkZES0T0l0S0sMnxnkFEXyWiARH9HRHtj9Y9jYj+zZ/r3dHyR/prXfH7cfR/xSsLE/f1+19KRH9PRAf9nPwzEV22yTifR0R3EtEiEf0pEXWidfuJ6G/9WL5KRM/Y5Fjf5udu1c/lN7Xm8LeJ6LD/vJyIaKPjnYq00XxO2PYBRPQRP18fIaIHHMeh7tKU9A0tWE5Ruh3AbwL40/YKIno0gP8O4LsB7AfwZQB/EW3yMwCuA3A1gPMBHAXwv6L1XwDwCwDeNOVY/gLATQAOAPgVAK8norP8uv8M4IcBPNyfq9c6V3vsVwH4Y7/POQBWAfxRtMk9AP4ngJfF+zHze70yMA/gKr/4DF3GzF9bb1/dFsAbAVzmz/tBAH+/wTj/I4AXAvg2ABcDuA+AX482+UMAhT/WDwK43l/bpGOdCeD/Afj/IPfrwwD+Ktrk2QCeAuD+kHv2nQD+y3pjOxVpivmMt80h9+b/AtgH4M8A/L1fvksnEzHzN+QHwGsAOABDACsQhnoxAAaQ+m3eDWHi/+a3+QcIE30tgCUAHwJwcXTMywG8DcLIPgvgaTs4/t8E8OrWst8F8IfR//P99Vzi/18P4OXR+u8A8NkJx/6/AF68yfkvBTAGsCda9l4A/9X/fj2AF0TrHgZgBGBuneP9dwCvi/5fAmHQe1rb/TiAd69zjMb9m7B+3X2jbfb7YxxYZ/3rAPz36P+3AbjT/+77MV/aes5ets6xng3g36L/ff88Xu7//xuAZ0frfwzAv0/5fHzcP7P6YQCPmfCM7wVwI4A7ANzmn6sEQAeieNwvOuZZfnxnz/A5Xnc+J2z7BD9GipZ9DcATZzWe3c9sPt+wFgsz/zDkofwuFs325ets+gMQLfoCCLN7P4BXQRjQLQBeBABE1IcIldcBOBvA0wH80Qba6h8R0dF1Pp84xssi/4n/A8D9/PeNAB5OROcT0RxEo37zMZ7rKgBfYublaNnHUVsNk8bSAXDfDY73cf3DzF+EZ9LHOL5jpUdBGNthACCiRxDR0Wh9Y5z+9zlEdAAyVsvMn2utD8+Av7+PmHQsZh4A+GK0/aRzTXye2sTM9+fakns+RNH56IRN/wxABeCbAVwDYd4/zsxjiDX19GjbpwH4F2a+u30QnacNPo9o77PBNep8Ttr2E+wliqdPYMo52aXjR9+wgmUL9Cpm/iIzL0KY8BeZ+e3MXAH4G8jLCIib4ivM/Cpmrpj5owDeAOCpkw7KzP+Nmc9Y53P1MY71nwA8jYiuJqIegF+DaKdzfv3nIML0NojFdQWAlxzjueYBLLaWLQLY43+/GcCP+zjMXgC/6JfPYTJtdrwdJyK6F8SV9Xxdxsz/ysxnRJu1x6m/90xYp+vDNfj7+6/rHKu9/aRzzW8lzuIZ+m8CeDIzL7XWnQPg2wH8LDMPvMD4PYgyBYiSFAuWZ/hla0jnaYPPv07ab51rBCbf9xP+jOzSdLQrWDanu6Lfwwn/5/3vbwJwbaylQSyCc4/LKAEw8zsgFtQbAHwVwFcALAO41W9yPYAuxJ3Xh2ikU1ksRHRzFBB/JMS90g7GL/jzARID+guIO/FmAO/yy29tBdxv9ss3O96Oko8NvRXAHzHzX2ywaXuc+nt5wjpdv941bLb9pHOttDT2dYmILgTw1wB+pGVFKX0TgAzAHdEz+8cQixsA3gmgR0TXelDBAwD87TTn3gJtNJ+bbavbH5dnZJemp290wTLL0s5fh7gJYi1tnpmfM2ljEnTWyjqfmyftMw0x8x8y832Z+WyIgEkBfMqvvj8kLnOPd3X8LwAP9UHkzY57FdcB8fdChMV9iCjWFu/vl4OZHTO/iJkvZuZ7+eW3AbiNo4A7M6sb42a/v87PfSCus0kMcaZERPsgQuWNzPxbm2zeGKf/fZd3nX0OQEpE922tX+9+tq+5D3G33jxp/SbHapC3WP8OwP9k5vWUh69D4mRnRs/sgt4TZnYQwfR0iLXyjy3XZ3y+R27wPKsyMok2ms9J217dstiuxpRzskvHj77RBctdEBTKLOgfAVxKRD9MRJn/PISIrpi0MTP/14i5tj/r+oyJKCWiLiTAmhBRl3xOgv99Pw9TvQjADQBeycxH/O4fAvCfiWgvEWUA/huA25n5kN8/88c2EAbZpXVyerwG/DEAL/LbfQ/kJX+DP9Z+IrrEj+VKAK8A8BLPrCbRawF8l2dQfYiL7v8pIyOixI8tBWD8ObP15qk1Z+vuSwKB/mcA72PmF05xuD8H8GNEdKUXSL8K4NV+TgYQK/AlRNQnoodDEHqvWedYfwvgfkT0fX58vwaJIXwmOtfziegCEtj4z+m5/NjfTevDwv8UwGc2iB2Cme+ACNT/QUQLRGT8PXt0tNnrAHw/xPqe6Abzx3rvBs+zKiOTaN35nEDvBmAB/DQJdP65fvk71xvXLp0gOpaI/+nygbz0X4OgX34ek1FhPx5t30BiAXgcgC9E/y+DQHUPAjgMeeAfMOMxv9iPMf682K87AxLMHAC4E8BLASTRvopou9tf878CeGi0/tUTjv3MDcZysZ+jISQ4/Lho3aV+2SrELff8Ka7tGf5+DCCw0v3RumdOGNurJ4xnDSpso30B/Ij/P0ATRXWRX/9IiPspPt7zIUrJEgTI0YnW7YdYCgN/Lc9o7bsC4JGtZ+gzfg7fjSbKkAC8HIIyvMf/jhFRXwTw+HXmkv3cx9f0yPYcQVBh10PcpYsQ+PgPtI71BX/+fIfew43m880Afjn6fw2Aj/j5+iiAa3aSR+x+ju1D/mbt0i7t0ilEHmjwN8x83Ykeyy7tUpt2Bcsu7dIu7dIuzZS+0WMsu7RLu7RLuzRj2hUsu7RLu7RLuzRT2hUsu7RLu7RLuzRTOm1KZx8480y+6KKLTvQwdmmXdukUoI/ddNMhZj5r8y3XpwupxyOsh55v0iEU/8zMT9zO+U4lOm0Ey0UXXYR3v/d9J3oYu7RLu3QK0Bnzc1/d7jFGcPg+nDfVtn+Mr26ahHw60WkjWAiAmbKCktsFwu3SLp20NO17vB4dr/ebACTTjvUbjOecNoJlK7TRg7srdE4tmoYJzeqeHq/nRs/zjfgsbleozOoY0xAByKc92UnbOnBn6LQWLHrLt/J+GvrGfKFPRZr2nY63O9Z7u9m5ZiUM4vO0z3m6P5ft653m9p7IKRGL5bRr6DkTOq0Ey3q3eNLyjR7Ik0W4bCd5Na7Tp8fZQrX1bdOJOOc0dCz3disa8HYEzDTC63g+lxs9fzt5X/XIZpNzOOZN3+1JyuXMRk5bcIV9g9FpJViA6R5GYO3DNYv3db0X8Vhewu1WRJi0/yzHt9kx11uv52Lmxu9paL1xrjf69Y66XQYdn2+jcwDTn2cr1tdGx9yqMDjW52y9+7pdagsVPSxF52O/0LTu/Hrv9nrLtku7Fsv6dBoJFlrzMLaJWR5YN+FlIjSZxFaZz0YvaMxEj/VYs2ZQk863VeZwLEwp3mer+291Htv3dBbUPvtOnGOnqH2fZ1nOadIzNM3xJ93P+D2OBQp8YWzS9LtwHc39Jr3fO0FbCt5/g9FpJFg2pmN51k6US4yIZvrST3vOaSiek/Y+J2Pdue0w/lhIbzQ72xUu68UWJh3zWK2VY9nuWGhHlQWanM9N1Hy/YwGzmQdje0S7Fss6dMIFCxE9D8CPQ96jTwJ4FqR97V9BSnx/BcDTuO4pss6B1rdUYlJtRp/DeJdJL/Qk4bJdTX0n6FiRMOu91JOOJ5BuQkIyj2uuiKgxV9u95kmadcyIHMs422eZZFXExJh8X/WaJ83Iegwqdr+0x7GZUrKZ4GofcxZCZSOKj79/zxzuWV7F/j3NTtJHV1abYzwG6+eM+fW6U29Mw9EIxBxcYc1xrN1eXWU7xfsJQDaDg/tePO+BNLZLAbyemV9ERPuxVT54ktAJFSxEdAGAnwZwJTMPieivIf22rwTwDmZ+GRG9EMALUfdMP8ZzNV1h23kctutKmGSRbKb9b2Q9HYtQ2apLSc5D4SU1aLoUOWwznZW3HlOPxxfiFJidBRe7QjcSnhu5U6c/13TbbTVGdKzUFhyb0aRtzpifw3A0ArMIVctr1wPA4mAY7pkuW1kdHuPIhYaluMISI3Ob+Puk7rK2wNlpY4JmF7wfA3gsM6/4BnT/SkRvBvC9mDEfPF50wi0WyBh6RFRCLJXbAfwSgMf49X8GaYC04YSSNJeRP3GTwsh8rhlkvXqS9tNmHqqhN7aBMLxjoYSa57WOGwx5PSY6DWxWBcY0x5uWGDJ3jnlNwFTXTxrXei699YSKLjckY540v22BGO8DbB6kje9j2yJoCxWaMHZCfe8a7v8J28U0yUJejzYDBACT7+26Fpgfb0JAFxWYDEarA5C+J+zQmd+77niGo1E9Nm4y7ISA+bnemn329tcum7TdVmj/njmMVgco/SzO9boAgOXVIazjBqOfpBxMup/bpVm4wlhu4or/m/kPQxoRPsYvn4oPnix0QgULM99GRL8L6bQ3BPBWZn4rEZ3D0jYVzHwHEZ093QGdvCwuykZSU0UDf+wAVwG2ApIUMCk4ydb4b5lMvYwdkkhYsUnhQCDwGsaU+LeZ/BhoQidephTkqsaOluRZcry+NREzkYnadiRUYjpjfg6Lg2FYN4kprcewYzIE2BZj1m9eZ59Jlti0+RnWrbUsJgkq8i44dYtt5VVvb6vCMwx7gpJCzgIkIpYMNQRMbQ2153Njprbe2q24vybd95hGqwOAIc9e9GyvRyurwzUMWi2EXm97QuJYqTvXR9f/XhwMkScEApAaQuoVtsoxHPOOMzfC7IL3vgX4RwB8M4A/ZOYPHDMfPAnoRLvC9kGk8r0hrXL/hoh+aAv7PxvAswHgwgsvrJk4kbz4xQBUFYAtQeyQnndflHd+ETDSxp2NAVzVYC5VPo/ElbJeNVMkMJHgYaod+3FchgEUVhgnwSAliBBjBkwCNiJQqBrXJ/TLY/eMMpNY+3Let7xeDCNm4G1mtDgYBs1ydTgKFpiNXEwbCRkdV3y9k2gSxDbez/jxqxBiXis4YkrMlBaPvxnqlokF7GbvfZtxNqwQohqBFC0DJUH7Ja/Ch/+Ggvas7qLN9OS21TWJtgrJNlQLBkC0++5cH4AXMP557k4QEKPVAdi/IwCangAyx2ypz5LuWV4NSlxKEKXSOcCkSP1yxxGyLPCG2RV0F8Ey9VycSUQfjv7fwMw36B9mtgAeQERnAPhbIrrfzAZ6AuhEu8IeB+DLzHwQAIjo/wF4GIC7iOg8L6XPg/RoX0P+xtwAAA964DXhzaNyCDM4DLIVYAzYpAA7VLfeDCKD9NwrAADlXV8Gd/pinQAAGREqzoKMCJTKMVJD0KgMMcO62rfMzA1GlyeEFN5ysrZ+oK0DWRFYqjEyGbDpgKlmX465oQWZwChozToQTRUwVoa1HPm4MxJGnBjaUCuO91dKDDXcCs7HW7i1fS0gSZg/WxgykRuQ1p1LdYdtdF3tc+lv3c/ydEJGTxOjiBoun9ja9ZbsJBeq7pd4K0bnJLYQY6Hdth43Ythtt+Kk47QFNUMsTMfAaBjFNzZgrqvDUXgWxQIAKidzubffX3e/40mrQ3HNMTMKC2SG5LmKtiFmsH++AMBQAgMG2aLp0dgmbcFiOcTMD95sI2Y+SkTvBvBETMkHT0Y60YLlawD+AxHNQVxh3wbgwwAGAH4EwMv8999veiRbITl6K6gcg1wVhAlVY8CkYhmkOUAG5d1fQXb2xQAAKlYDk3Rz++Sh81YEszAj61iERTUGsUNejoAkBWc92KQDAEjsWATHeCTCLO2CM9EGnXetkLMgW4KTHJx2AiNxThiaAcPAhRdf16v7aT0GBdTMZFLwPGa6GvBkAHmiGrc/bsRU19O0GRoTap6/LWxkciFMmRGuSc9BzoLYwbBDalI4SkQQRAxyIyHZOI1HquncxMvjOdD94/lou/ZUuFA0Jyr2nVq63jxtx2Jk2w3GTGuF+DTr2vBuFZLxPSAwjDF6qMY9Sg0Qm2KsFlbLDSaWTAKQganG8u5UBdJqjOzcSyZc2YmjPBGlKigtANiIIgjmsNwQ5L2tRqDxQHaOrLHtENEWaoVteBw6C0DphUoPonD/NoA3Yqt88CShEx1j+QARvR7ARwFUAG6CWCDzAP6aiH4MInz+06YHIxIBklRgdgjckx2oHMrDZAxcdy+YTO0SYwbnPWH2SSbWhBcAqY+DUDkCZ11xazkHGCPHtSUSb3lQOQQVohVytkdccbYAACTMoGpUa4quAo1XYJIUrrsXDsCokpc8SwwyiFVjjNweAxdemvUC8pO03VhLbySPcR1IXVkdiisBNYNsxw7UrRMzUo73YQemJIzBsJV5bDGvEFcKQka0f7lWwLXcTpsJyfXK1myUGxEfT481act2kLoxLhW5LbdZTERN5NxGRuEkAZonom23VwUFwRCMVZetAZkk3C+1mjQWxKBgVYYYoHfHjlcWASAE7w2LwAczqBwF91exeAj53rryu1q/e7YZkD8WIhLGVbuqYwFbKzmmGAC2CO8lyIC7e0TpnMU4MLMYy3kA/szHWQyAv2bmfySi92OrfPAkoRNtsYCZXwTgRa3FY4j1soUDOcAzcs464eXgJAeRD1YOjoKSHMh6sr6U7clV4LQDtgVgq4arCoAImnJUn4eNP1/NKDnJwfPeVWBL0HiljqUkmeya+Ol2tb+XbIHclsjZweV9EFtZ72wdv9GgMTxjiYLVShsxydiaIEPodbth3XpInfHyUQE1AAADhp1YcSRMSjV21ehNOZJ58/Mk8+8aQoTYyRyYVK6dRWCy/w9e61qclraSrKnBfo7gsspwGxowaE3ehMjlJsx1zfEjAQ2ihlUUx5xi12E8+uCOgzBuBV4A9X22jmEpDWPXU8z1uuL2ci7cAwLgkgwpvECZMO7x0j0inJIcziQgVZgAcJoCtkB591fgemeIQubPPRoOYYZHQFURvAA7TcFC5Kb7kkjjLf6eqSLoibMO4CqY4eJsxjGjBElm/gSAayYsP4yt8sGThE64YJklNRBY1gKjZZC3RqgagYwBrRwCkgyu0wd39tQaNbsQ6BdLxjXAAPJC1gKBTQqqhjArh0DlEFyMxIW29xxw7uM2JhH3mN+XKiOWCwCXzfkHvwKSXMaojKAcgsqRnNNr+Zz3kXTmRQCSEXcb1QwlNfULNykgOy0V99wOKMhg7K83zeE6ezwSzmvsGix1Vq7JVnKtKkxUYPj9AYCrAnAOnOXhfLVQcUgBmCQJcQFgsoDYqMDmZpnck1xlbcZehdgQghCV7eVbrj9yKTWOX2vQTn1mLYrjIXG+ThtI4FgSEtXVFwsQPb4qDQQA7DAeLHuUY4K4uSGxjy+QASDw4vHSPXIdaUesH+tAxaDBjOVELlibZrQEAOjCKwu2FMWMDIpDtyI/815rrnfW1O31MB4sw5hUrFpgjaBXa45sCU5F0aRyvL4Zeoy0W9JlMp1GgoXAaa2JQ2HHrgruLVZGaMSP3EBneaERmKF3X5nxIAgJYebeijEpXN4HuuL2MqNl0GhZhJN374SRaSAfQHLxA1Dd/llZ4SzIVgEthkpcD1QMwXlPXlpvFdFwERgugjOJ3XA+B06yhra/HYEyHI3E6tAFJgVZcQFy2kHJ4sdXDY1NArLOC6CBMC0vVDjJhUmZRFyIQBCQSNLgFmST1oCGUjK6DYA0yeGyXoi7AM34kgqPthCJl4e5bwW524ylbSm0Yywh1sK1hgyanJgXgwCAWqTE1smkcamgIRJGpa7JzBBGlT+xISRhXz+V6t4Mbi69JzZYiqochefWD3S8dE9jPHIvvOAxqC2eMGBTv1NhHLXAUas8jl/uKPkYHZsUZJLgktV0A04ykKYSwAtBWHGLz4jkfu1Klkl0+ggWdYX5oD2cA3oLskxfhqoAj0fyUHb6YP8gMhmYYigvX94Dw1s4SweFIfQWJPCvQsdV/qlKw/k47QAdV1shmgfjH34kQHr+Zahu/2wtDHwQMfiyPXAASQaywrBBBNfb64P+km9DxUC2T3OYMrKkescWYF1ZHYoriAic92FWj8j1QrRZNmlwEVWOkblC4jDFQMAS7OD6B0SIF8OaQVUjGasXtCq0OcmFEQUhbZpaMlUwxQCUdkIMywd+wGRCYNry2oTMmJJ1Aqtx4DvOO1HfPcAynhb6K5RviQ4bC5WY4TM3gQHNc9fbAbLdnn6dczIcjVBYRp4AvSxK8A2WiZ7cRczVn9NkMOyfdzLozPUxXlmUcSbeUlRFxgt1U/igtgqJyI0m53W1IqDLnRWLVbcph6AZQnkn0crqUEA0tpRnNbhUuR6nJ/Lvobi5c1H4ZkyE2QTvT0c6fQQLkTAqVwVfPhkDt3QoMK3giqlK0Ni7rgAJ6uthkiRYF+z/ky3A3KktF2WKtvJCgWvXT5J7N1EkYNghveAKVLfdEoRTWO8qsUDSHGb1SLgcN7cP3On7bUQDE+uhC8qFCY2zPvbs337wtI6zeOtigyxsId2+3m68dI+82CYBlZV3k3hGpVyXIobFzsdaxJcRLEUPjAgWJZlaoBOByAgDk5sDQm0dtNFTk7LyFX0H784zgBcgkSsldo/CeDi4IIzWgxkDHj3oV1fcAhr4MaYElH5dzmUQCvE8Ip9DFsXCNFCugWmd0hhiG8dYxstHgyUxYgciiYGBSFxgg2UwGXQX9gNoZtabYlWYsVcSoqvz8+5dnUnmQTIeyOIViHCvdorUzRonMMPHjjzKhKpRDbgBYMaD+pmb5VCw6wpbj04bwULOx0iCVum1qiyHGyyBK9HO1HphdYkZj6hRq0ahiCYRhg6Al4/CAOCOtz5UOETnZ9XcXCVIFHZAVTYefrVuKLZoTCovgK0iBmqQ7z8f46V7kKwcBFUjcDYXkGucZHCUYDagydmQGS4G15zr7BELSy0zb8GkF1wxcV/79U/C9vYBWbeRUxTuhVp9fnlg7lxbGsz1Sx6j19RN1BYowVWkrjgvFF3aEXhxrP36YHCcFMlEjRiXbbnFBPJaJ++FYzmHNMk84ssj4oaLqI54i3luHwAgHRzCCh/A/FxvDfJK8zga8+9FzMrqEGnaAcroeVRqWVCrw1EoiwKIUOl4YVMcuTO8A2rVg0gg9JE1DnbCuCPEY3HkTqykCwCmq0k2LSWS3BSAIEFxA7yikgGuhBkugsoRnL7TrqotlrZ1vN0x7brCJtJpI1gAFv8pGQnW2VKsAV3tbG25WAugBLLMx04i5IhzoFT8stRfABkDLkYSnO/0Gy9U7Cogq+4HqtFfaQZYC04z2K9+HPCWRtjPFuKGGxxuXEl6r6tks7SDat9FjbwDzZ4+2YizDlxXLJhk5SBcp498//kojtyJfN+5G+6bXPgt4Ns/K8I5qQVI+I5+M5kGDBqQQG0MIVU0Vzi+Qm89KVMKz4tJRQMn04hPBG2cnQhNL0jYB/RNBHQNGfd+/1AgMQY0+PtuyhESDzkHc4P5m9UjMKMl2PmzAAjqiqmJ5IuFwXiwDGeSkGgaNGgVypqX5WMN45XFNXXBwrGjc+g9Kw9+TZSu8MxWddyMjFfmIkbt70OfR941NRvBMhoOvUXiaii7PgBaPmm4CFMORRHJuiBbeM+AC/PMrk4D2C4RNXOJdqmm00awcFXCHr4TZu8BMdmrMaq7vi4rjZHYCtAIPgrrKYEkAdIMJo+C/86C0kxcTz7Yz9aC3EjcbBpk937noEVprCSKK1ApIAEaLct3w9WCWrMHGg787lw/VJI9UbWZpqV8//n1n0j4bSZUlNLzL1t3XXHP7TJPSdZwX2ksgGMXDTsQo7ZISASPIu1i2LNauEQE110AJVko/6PMMj+jWZ5pvLLYQLLpvRS0nCdb1rXldGymjklxHt1LhcpWI4lZAIAxMONl0BkXSuwO4q6KhUsgW4LIIDEJ2HmUmLfCQAadBXF9aSyGTYrRcIjKW3uJk7kBO9jeGTDVuCF4srMuEuGSdIL7N5AXKo3APyBgDjMCpzlGq4OZKENkiwCBDsQMsIUZHvVzKChK8ghFbnkIeLQCHo9q78T2RwXa9YVNpNNGsMAYUJrBLR4GpRmo04UbDkDGBJO+QWkOyjLkD38aive/Afl137fh4e2n3w2YMSjv+cBlBR6KoOA0D24zAHKuNAeyTs18YncOEHzEyb0fCPuZ94KyDtzcPrjeXpTDYUB4acA2Ljm+3SqxpxxF6J46RuNdT355zLy0b4cDwTgLGi9L7gI7GOtjVgpDt4UAEIpB0LY3EnIbVQFWGg+WIUEbUysZHsZL1QhUjZBecAWKQ7cKAyQDzuaQnXsJqttukWcryZEs3wWyBZxfF1+f8YhGM1wM1kmSzwlSMaLV4UgSbf28OUoCCGGu18V4UAaXkvEgkvKuLwMmQXbWRQAQvsfLR0HGxwZNKq5LoM5dsoW3EiTAXh24eNO5mpqcBeBzuyL3l6LAqBoF9KWiDqkagasS7EE7XJXijZhR5j0IMLuCZSKdPoKFuX6InJWSIXlXXFhVWQsWkwRXF5ei2W0mVAAgufIx4bf70ofB41VxqTkPJa5KuGIk1g0khpOefSEwXALSDG44kHGYBOj1RUB5rTa5/JEAAHvnF+Hyfsij8AeCowRYA1j9xqG21RDTpHhDrNkPRyMgyRsuEc46npGuCiNyldR9oArJN91/2+Pt9PeE3+XBr3nGfMaa7SblfGgcSiHp6qYr7/oyQCTouyQTS2ssldaZTMgVomosUFu15CAwdJ2nOEcIQB10V2vDVYHxtq2NYIHrAnUpxlYBGZBCYUqBsOMYLJbxYFmssQCBTtdCp8mARkswo2VvFSYC01dQjH/v2VpQrw/yv2cVYyECTH4yRTpPHjp9BAsgLoS5BREuKkyqQuImarmYBG60Crhl9J7808d2mvtILbnqI28CALjRAKbbh+n2QT5uQ3lXXCJeIE0DxMzOvQRZ9F/8ygaO+RvPSpmS4njDJOp1u0C3izEZmPGKuKMU9VSOwcUQbjwUxWA0gL31s8gf/rSZjU+1/a2SWk3FkTvBaVcQg1HGOScZOmfeC9Udnxemaku43l5w2kG31wsVjHV+9FsETEtJ0cRc77Zz3QV5htk1YmT53jNRHL077MNpVywXt1pDjU0q2ftkjilZsjh6t8DR/fmVNC7CqiSUUpMv5KVosqZPRlawDpeFID0Bef+9MjkTItq1WNah00ewaE7A4uEQbCdjwM6Bx0Mgy32QvkTvO3/ymE5R3fQWmP4ecNZDcu8HiiWydA/YWlRH5IXLLroUMAnMN/+HWV1ZgLDu0rFTZ2E/hqO5UFzRrB4RyyXJYPIe3JG7BIpeFhi95QZ0n/jsEz1kAFGMas8ZYZm0ByZ0AaTn3XfifuvFNbTkC3n0FmwF7swL4/bMm2zhSxpXa/Z3eT8cuzhyp89JSuGSPeKW0/p6x3a5sq8mCtvKu5lrFKUZLkr2v7OhUrEqjjHwpuH6TnOJl44Gda7YjIjMzubunKp0+ggWW8EtHq4tFQjCC5VoLFwWoCxf10oZ/uMfwnTngkuh85gfXLuRMYAPptpPvUOWpZmUitl7AACQ3G92pX22k0m/S0L2M++VH86id+VjoLk6dvEOiW8UQ7Cz4KpE/sgfwPjtr6qZ00lKx9ozfiIlUvnbJSIsyJYidDt9IOs2Ck+Ol48CSVaXbgmJvumapOBpabx0TwPBRt6SFOSXWGKcdmDGizCDe8S1DYC6Ml43lNgVOysozKqQd9jHOU2vLy7ywoLHI7hiVAN5tkm0G2NZl04fwcIAdYQRc6H4dRviKICYxevRNJqHPXIQqEpQfwFsLUyvD+r0Qoxkl04esl/5mA+Y15qr/cx7RXP1rhYerYCHWkpd7r89chAwBsM3/v4xu0pPZrJf/TjowMUIladNCu0/BMBDdTt1Fn5EnT1nBMDBeGURplgVwaS1xDQe4g9W3f7ZDYEQ44GAX1SgIBYqAOAqmCXxBNgjd0vkxlm4wTISr8hl1z4lHK/6+Ftr68TZ+p02iXeJSyx0lrSLCptMp49ggeaneKqK+r9nLnP/6Rcm7jf6p+uDBoSqQOdxz5q4XecxP4jyQ2+U6iJJAq7KRlB/l04ySjKQFu0kU9czMwlcZx4GAJlELN1ihNW/emkz/+M0I/elD4vrb/UIsrMvRnHUK1rsACQBwSYotsk1tfIz7yWFSjVZVCHb5aqHcedNSPAGRNVYqoD7fCIaD8BH74ZbXaqtCpPIuxa926bXBzsbQDf2U+8A5d2gWErOmhUrxVpwMYIbLIfjUGfjuNzURARKdl1hk+j0ESwsbi8xe0fyIEbaydz3/9LE3YZv/H2JiWQ5kGabIsSyhzwZ1U1vAYxBOkO31y7NlpKLHwD75Y+Ke8QTw9fKclYC+aUE7tlZ2MN3BJcYGQM3w+zsk4HcFz8YAtzJyiG45buB/RdJWaJUwvkhWG5SsVpMGvqwrA5H4vqpxqJYlat1YqKWUWqVcyF2qO74/LpxICpWgzAzw0WJVxa1QFFqKIwe1k9pBvvpd8MNlgT9ORzIe1+VMHN7oK0nYBI5pu6XZTNTGoiAJNsVLJPohAsW3+P5TwDcD/J8/yiAzwL4KwAXA/gKgKcx85HJRxBiZwXtBTTcX8D6QmXltS9BsucMoCpgD9+B3lOeN9WY02ueONV2uzRbWv2blwPGYO77fn7dbeyn3gHMi5sE4wHIdQKSiDOJj1FRgIdLjUQ5yrsga2GXlrDwrJfs6HUcT7Jf/ig468BovCLrSb8iW3kAQw8Yl1IV2qS++nRRV49gh2LxEBLfhbUu3eJCCwgAzcriraKjk8rpF0fuhCmHUgqoHIuAr4pQz486XaAqxcJIEgHg6OE7XXGFeUUgXOvhO2H6UkrG9PegOnxngP8bD37gqgQPlmYytwDB7FosE+mECxYArwTwFmZ+KhHlkBoQvwzgHcz8MiJ6IYAXAvjFqY4WVTFdT6AoJXvOgJnb4+HHk7WY1Tf8LtxoFfM/+GtTnX6XZkPjt7+qBmIYcTtigzhYddNbwFUB0+tLWZ00F2RSNapLePjabWxSUHceXAwBOwQPB6BuH3NPes5xubbjTVSO4QZLnmGXMGUGaNkjbdfLLlQESC78FgBaLyytg/GslYUNksFBOXY1luoAPA7VD8jPf5zYunZMI5iVQ3AaZ9HleVfudVUGwA2A8Axo3KT40s3eMrEhQRmo46j2yMG6BqBJ4JaPBitIFdBtE+3GWNajEypYiGgBwKMAPBMAmLkAUBDRdwN4jN/szwC8G5sIFinj4uGSne5UgVfKuxtCD+96+U9h4fL7gooRBn/xm+g//Vc3PeYubZ/Gb38VAPggrG8tkGKN8NeKCYrQM/0F2QfCTJEkYHSbKCUiYYReaFGWIzlwLuAcqo+/Fen9n7Dh2FZe+xIYn71th6swWQp27qRUPNwX/h3IenB3fzVo7gLPTSTfQ0sVaTl925zffN+5KA7dKvkiQCgJY8YrzXiM9jUiA/btvkMw35+3vPOLMOOVkIBKthRBkWXBbV0nF49EqPT6Mt7xSCwYk8CtHAW1Ep41tYC6ffDqEtzqsiD7nIPJu3C+CR8AuJWjDeVzO0SnoGAhokcAuC8zv4qIzgIwz8xfnvV5TrTFch8ABwG8iojuD+AjAH4GwDnMfAcAMPMdRLR+6rWSFpW1FrAWK699yVQvu2hH8qAduf6FSPtduLLC+Iiv6+V99K5ci5LZpdnQ+O2vAhcjdJ/0HIzecoPMuS+5Q4i0WGdR3fk1jP7peomL7T2A8kNvFHRer++ZiQ3l9uN+OyEGwAw6eie4LAPjoiSRPCds7G4b/t3vAQDseByCtuxjMUHgAMIAi9FJIWz4qM/P0QVl6dsN51IbLQ60TyioqC6s4tCtIHiGTNI7HrYASifznfryReMVqQxgAZiyTna04l6zX/8k7J5zkFSjkLyoQBgA4ZuyPFgtgZyF6c7V+SrtHBafDM3DgbS8LkbBBaYBfcq7oLm6MsJ26VRyhRHRiwA8GMBlAF4FIAPwfwE8fNbnOtGCJQXwQAA/xcwfIKJXQtxeUxERPRvAswHgwjP3Ck59Apzw4O9J7OSs5/1eY7kbLMHs2Qe2FnYg5TFcWcEVFSgxoMSge5q6R04Ejf7perjhQNxVaS4CPSqxM3rrjfK/0/M5CWWjBIc9cres08oGVQmkGeyRg0j2nSUZ9PCooawHDJfgygK0cCZQjUGJtj1IAJTg8TAghezdt0kCHQA4h/HbX4XkwHkwvjyLuVTevaS/R6yV5aOgxMB4aCsAuKqEybsSJwCw/Ocvxp7//OLjM7kTyC0e9q4gaQ2BNBOGbRJQ1gkFVAO8N+6o2iJtmKX3wnUlIZI7qaC6fN8gJBl4uOzRVz0k5VDiOp0+OO3CJZkUY10+LGPRlAC1VqpBo3WFybtwy0dEWHhgjuaosMZUIekCDIDHQ2lVbAwo78IevlPGb4zsPxxs6KXYChHRqRa8/x4A1wD4KAAw8+1ENDspG9GJFiy3AriVmT/g/78eIljuIqLzvLVyHoC7J+3MzDcAuAEArrn4fLZjeTHUujj4e8+DyXzvlJZmoZopjwZwo1W4soLJU5SDIc786f8x6+v8hqfRW28EgNpnrrXcvCKQnnMRaP95YlmsLgpMNIp92YO31QJF0UjR/vbIQXBZwPQXwJ0eMDgCN1iG6e8RxucrIwMAdeZAnTlwOQYPB3Ajz7Ri18w6xGUJVywL49IOoCaBq0rYURGUErYOlJgTKlziwDYDIHUrGhMC7LBlaCss5fXziceiYiDl8g2Q7RcrphocljiK822oqxI8WBZBpgH3JAElmbTWtmWw8PR8ML6xnnMiNLRgrP92Q3k/zdwecLEsMGRjgvUU7lWnV8dZxkOx0Hz8hjrdEGsBADucZYzllBIsBTMzETEAENGO9eA4obPCzHcC+DoRaRbVtwH4NIA3AvgRv+xHAPz9FAcDW4dqMAJbh2JpEFapcDn0+z9Xb28MKMsDXr4cDFENx0iykzvr+pQlZwPj4KpsaI1qofDRu+GWJRPb9Pqg7hzM3B6BgPs2BsE/7pmOWjXqMqE0kzhLWUr5nfEI9shBuKV74JaP+gq4Tas22Xc2svMvhpk/AzS3AMpy2MXDKL/+OZS3fwXjz30suN+o10fS3yMCDgjXQcYgnes2GA1bB7YOS686MS6x7NqnILv2KWGs1GvyES7HwuituqzSRn2umNJ7XYV837nN9ghAyHdxgyVfQbiQumt6b8oS7NtnS0HYIliqoZbfaFXiJN5KRXB1eUXAmCBsqNeX99baKE/Nu7/Go1DOyY1WUa2soFpZgRssCXihKoICMCsyCU31OUnor4nojwGcQUQ/AeDtAP7PTpzoRFssAPBTAF7rEWFfAvAsiMD7ayL6MQBfA/CfpjmQK6vwYpsshUkSOG0zbFsvTFWK6VwVYOvQOWMP7Kg41TSQk57Gb3+V+L3Vd25MaHEQlqlbyyQBLsrFSALrPgG195TnYfiPf1iXPdf8hNiCyXJhniYV3zoZ0XgHS8DyEV+AdID0wLng4TLs4mHAJEj7e4BuH8neA7BH7haXUd6FWz4Ke9fXpNJCXCrIJ92xc2A7hslSUJqDK3l+XCEWcyh+CkwFDNgpksKoBuSrOoeik3oP1GLxFYGnHWt6r6skV6i7BxgOfAZ/HS8JpVV81nsjDcC/l5R4K9SY4NrS3CO2Pq8oIMV8uRaTyPurXWAh6QaUd4NFpMIjm5dkSgXerLx2hlByOnX6sZD06/4rAJcDWILEWX6Nmd+2E+c74YKFmT8GCSi1aUvZh8yM8dFlecm9cFBRUq161EhisPo3L0e1soKkm4NHZdAqi+XBrgtsxlS89y8B+ByRKFHR9BfENTFYAleF9M1JpE20PXwHkn1nC8OP8oVGb72x1lydlf2dFQ1ZXSpGasKhJygmt3K0kXCnCXS894CAA/IuKNXyLgNQrw8+WHqLyol2nHdlXd4VAeUbRQmU1QVrOHavqCuMrQiW3pUP3PnJ3oDi+nXuC/9el12xEmfi8RBmz35ZVkXVgKcg9u0fqNcHGSPCGgitK9hZiWto91ZveZIX8uRLsKgFSpm3OIsReHVJYilVCR4NUC0tIunNyX1yFq6swLaAyVOwdXCrR0MgX4V7uSKei6M3/DLyA/vhigq2mE1ZF5qRK4yILgTw5wDOhbCtG5j5lUS0H1vM51uPvAvs75j5QQB2RJjEdMIFyywpFips64dLtcbugb2+X4sL2wBAsTyYcLRd2g6N3nKDwH+1zIYmvJWFDyZ7aCoAt3S44VISjbf18vuqCtTtC9on7wJpv9kWwbefJnawh++EXTwsCoV3AVGnW8doTOKFignB4LC81UlUl9FoIOeCBOqD8PBAj9gqpsQgSXJxzx68TZjuQYFRr1cy6HiQVt12X/owuLcAKkYiDJyDW5Us9uwhT576eOkFV6C6/bPg+f2g0bIAJ6wFilGoZKDuStPrB0uDAXFhVUXTWoHEhrTMDpwDD5Zgx2PvkRiBqsILFQd2LsS2nLVI8kzefSuCJ89TmCxFMRxg8XNfxtzZ+5pxnu3O52wslgrAzzHzR30w/SNE9DZIGsax5fNNpn8noocw84e2P+SN6bTx+5Cv2xMXk2TnYH0gP+v3UA1GKI6uoBqO5XswCjGZXWtl9sTjobg1vIYK3xPHHjkIe/jOEGshXyEa8JnRapF4qj7+1trygI+j9KT3DfvSHepbF+FSgfIuTH8PzN4DoWBhGFfpNfViVENdnfUowTNkG2/dqIXkVuvAMSAC0GQp0oW9NcwYWBNjcWWF4q47YBcPC0hk5WjI0zmRZO7zYCQXfgusTxx0q5KNvhWhopSef1ndorm3B2bPfnENFiPwYEmUCbUuAYmHlDWU2C0flbnv9cHDAarbvyxuyMEyeLAUhDh5haRaHcGOCpSDIcgYlIMhbFnClWKNqFBxZYW9P/abcl19UXBcUQWFc7skqLBkqs9GxMx3MLMitZYB3ALgAgDfDcnjg/9+yjaH/K0A3k9EXySiTxDRJ4noE9s85kQ6rSwWQHyr7BxcpD2e/fOvDL/v/t2fAQAciJbt0uyp+8RnY/VvXg7j/e2mvyB1ndIMbvkIqNOTpEeFG5sE7BPbeDySuACknUF23sWguYXQT0O3gbMwe86AO3J3DV8GvJVhpPDgcCDacJqJlaNUlXVQFxLA1yxwFVAy1qMeAu339SXazZ59IUZBaQ5aPgK2Lvj2TZY2YnZkEvBgCWbvgYaQPNGU7D0ggffCbr7xRse58FtQ3f5Z6ckChMRGtT7je8bOAf5euqXDUt6+Owc3WBJ0X1WE2IkG28kYyS876pFeXols55epJUOJCTygGtTzve85L8OR66fOaNiYdiBBkoguhkCCP4BjyefbmL59m/tPTaeVYHFFBRvloZz/q9ev2ebsXYFy3KhdTXrxxl9F57wLxIfvLOBMQImZfs9DSkc1eshTedsXkew7G+zdZG51OVSjphh67F0vdvFw8N1zMQL1+lK6Z3VZhIv27gBkWXfO16YqZPtuP/jqkwPnipV1+M6QW6NCAv0F+TYJ7Eh8/QYiUKwtgntMffpsHfI0Q7L3AMZvf9UJdYkp2Qvuh+TrH5cclW1Sev5lodIwALGEBkviDlWB4qy4M6NyPY1vIJRicYWkACTdXLwLI0knUEvQZGkQJACCQqn/Vagn3dwDLWT5vue8DPhvv73t690i3PhMIvpw9P8Gny5RH45oHsAbAPwsMy/RhITVbdJx629+2ggWW9QYeVdWuPBFN57gEe1STIs3/irGR5bRvXdfgrSt8vSaOMmjQQjEj9/5577zXyHutDQL6CONuTggaMBm7wGBGFsp1WL6C3AeLVTd9XU5ti/fQt05QaHNnwE4K4LDn5eMgdmzL1hWWgDR7NkXYjqoCo9mK1AtiZbOEVNTPz/8+KrhGCZJUB49GpBvo3+6/oQn4OZ7z4S9Zx9MOdxW7GH01hthfO09Hg4kSN/p+gz6Asi74MESqL/grZlhEDhxFj2chavKRj4QW4dqNA4urGpUIMlTlIORJKl66ySef0DuweE/eAGSPMPSl+8AANz6op/AvX59Vghb2koHyUPMPAmkJEciyiBC5bXM/P/84qny+bZAb4JPaYJ0vLs3pODvVds87ho6bQQLAJSDEapRgYt/+88233iXjjupnztk3sMDK7wbS9BJ+zwcORe/e38BdjQIGdSSqe2RS5FVwqMBsPdAyPhGVYC6cxLoHSwHgcVDQX+5ZUGMJQfOBY9WfZHGusy6PXJ3jQTzrjV4F5j1bah5JLGCpNNBtToMwWRKDKrRGGwFNRZKjuQSS7JH7l4T9zmRlNz7gXBf+Hcklx5bZY/Vv3opzL6zBTKuMTKtQrBwAO7o3eIy9GV3QkAfgB2syBzlXdjBSnBb2bIGR5g8Dd6IgOJcWoUtKyRZCltWyOa6AQ2mKDGTpRgfXUGSpzB5zerueOmxtSZvk3SQnAkqjADcCOAWZn5FtErz+V6GafP5NiBm/pbWeR8I4L9s55jr0WkjWDSwtytUTk5SbXP1s59G/34PECvAJOLWMpI4GZqtaRa2D6hL/behJMZ5ocLWgoyt4xUe1eVWjsIo2gha1sQFgWSHA5BHIJnuXIDBhrwY1MgzzatxwwGAgVRpQF1Bl4AQD4hrh2mgWf8DCFp3HnJepG/I4T94AQDgwHN/Z+cmfwpStNhmNHzj76M6ek8QoiZLkXQ6cEfuhj1skV10qdy/8VCsxSwLbi7NY+HhAON7FoMAsKMCtlgEO4e020GxNPDCuUDW74ZzxaWW7GoztqKxLecc4J8166ta25FB6o8zPiqpBrOhmTX6ejiAHwbwSSL6mF/2yxCBsuV8vmnJo9AeMstjKm0oWIjoe6c4xoiZ/2lG4zl2YsYFL/rjEz2KXVqHbFmXPEeaAeMh3KoPlvuSINTpin+9kAKFoby5Jtz5goWaUKloo1Cq4/Cd8v8sYV5u5WhgZqYfNX9Sy8QLMC6LNb3UTVf6yvN46HOg/DmO3A1XVkj68w3IMnnkUkyqPZMxQdi4skK5tIR0rqjbIkMqaZ/zC/9rxrO+lhZv/FWQMVvuOVO8/w2obv+yz3qvhSUbF/J6TH9BytUrSm48qkv4APW98oJYK2TYKADvikpch15QlINRWKbzJ6kEpRc2JUyeoVyt75ONYixpN0fW70qcxTok+Qx1aaKQx7QdYuZ/RSiju4Zm1k2QiJ4f/TWQOo0HZ3X8mDablf8DMb82iiI9CsAJFyzZ+d90ooewSxuQMqOs3xN0UJqD0jL02NDS9aa/B7YYidDxFosSAXXCnC/FI9aI/vYxjVu/uCYrOwT0tWSPd2/ZxZEE/n1BSo2baM5MCOjnkgND3T5I8z48RNYNB4H5xX7+pJsHZBglTQQTW4fO2Wdh/n4PAAAMP3fzTk19oCPXvzC4hA7/wQu2biX5emqAMO3y6FCubSAxqAPPrfsfjd5yg7dGI41eM+qrAsXSwMeeCuQLcyFeovETOyrkd5bCuTpfyEWxVABBuAASWzVArQSMxqgGQyxcfB7sqMBosIzx0RXxavziH2zt2icR1ec6RSguOFlBYi5v2IkTbSZY3szMP7rRBkT0f2c4nl06DUndPQCQn3V2XStKyWu7PJZkPbcsycWx8NBcER4Pg+XB7bLp/ju4n6KSKsp0Gs3CqjIE7wX+nAWhAqDhZuPVJUnMnD8DSDO40aowu1TgxuxGwd2nQfu4JpUKGy07xNbBDpbDPHQvvmR7k7wJ3fHSn0S+Zw7V0jho2bf9+n+Z2spPzrsE+XXfh4O/9zzJCVseh/I16g7SKuImSzF3L19TrNsH3ChUOeBCcslGh5eCIDFekGiCczkYoRwMYfKsYc2wdSgHI6S92mrSfjgxMkxcdBmSbgeuKDE8eAT5nj6qwQjZ3Iz63QOg2bnCjhd9mpn/Jl5ARP8JwN+ss/0x04aChZl/aLMDTLPNqU6ff85Tcd/rXw8AuOkpT4CzjLSXIskM7vfaN6273zuvfmj4/dhPfHDHx3myUrvSATvnM7PVNWWCW8sNlsU9ZW0QAq6opDy9b4vA1iHpdMIxY604ZnTtZXreYMno+tGqwIY7XXBUxl8sI19a37vU4u6DbJ0P6nuh4RPzADRcJO2MfLYOlR3DWQuztIqkm2PhWZs3pjtWuuvlPwVXVCiWV5FkKThxGB9Z2dIxkosfAEBaTxz8veeFuIe6mNhKMrJJJHHUDpZhRwWy+brJlj1yECbvorN/Lxa/codYfWWJcknm1KECSkiQ3jmUS4MQh0m6HcneTwyqoc8VSgw4EavHeKtQ1yXewkm6HazcdhBkDgfwyMyIADM9KuxkoF/CWiEyadm2aWoHIRE9DFKzJuzDzH8+6wHtJH3kux4Ptg4P/qd3TL3PJ5/+JPQOzOOTT38SAMBZYUdsHe7312+euM/7H/toD4HkmZaPOBXp0O//XAia5nskCU4sB9+LfG5PiHW45aMAUHcU7HRRLAvT0ZpPxdJA3GlWmJq6mKrBKLie0n7XHyeyEpyBgaKFIG6xTk8y6gdL4kYrS9jhItKFvaHQJPxYQwmXlZUadZSncKPVUFJENWeTp6hWZV9n3brIoaqUMWt+xk5RO3bzmR9/itQwO7AXX/yZHwAAdPfvndp6KZZq4Vp6FJfW4rPWwWRyP2xRohwMkfV7SOe6ki2/tCoghj1zGB6UpFKNj6TdDpxzsB5Rp4LCJAYcCW3AJ0I6U1s0XmAr9Bjw8RpNkhwWKAcjdPbtwVd+8UcwKzoVLBYi+nYATwJwARH9frRqAeISmzlNJViI6DUALgHwMUhvOEAUvlNKsCh96ge/Y0NLI6ZqWGF0ZABnGdWwQjWqBDO/tHGukS0sbClTdRKVzT6upJptsbyKYmmA3llnhA5/msnuVgqY+TNqyHAUFynuPohiaYDx0RWwdUj7UpY+X+ijWB5gbq4X4L2AD9b2u0j6843GbbYo0dm3B6Y7h2ppCdYWSLoAxiZUVdagvmrbmoQH+PIteZ2Mp8cEgLTXgSuqAC9WbTtO1FPVwqjV5I/prIMbjI5rF8JPPv1JyPfUVYBNlglkd8pS8uN3vxZ7v/kCkDE4+rmvY3x0Gfken6zqrQZKTJgfOyrQPbA3CN/Fr9yBajAMwXkASLK0IWDVpWjLKswTRoXMaaSoucKCktod5axD2suBSKDYokKxrMKPxKKaUQCfZhS8Pw50O4APA3gypEuv0jKA5+3ECaedlQcDuJKZj1vm5qzoA4//VgDAtW97F2xp0VnowJbTWRE3PeUJsP7hrYZVKBNRDeXBf+91D8cj3/++Nfu5wqEYFGBv3Tz+cx+d0dWcWqSMFhDN9p6bv4zzH/UA6b0yHknuh3PIAWC0GuIn1coKysEQ46MrwWJh65B4VI/WiAKEQY8O1wmK6bwIFTsqQpKiSSTp0Q5WQkmQfKEP03JbVavDRkHJEDCO4iKSG7EcrrHwNalUS7ajImheYYxREp8KEXauduvMEqm0CZGhEKMYH10GJaZxPZ/58acAAC7/k79r7Lfy2pcgv+hSoCqQXXAJDv3LewAA2VxPXGBWrivvzgU0XDkYBSulWB7AWYejX7gTeT8HJeSVL4e8X6PA2sU8Ac20r/9bX34mydcGzoMw99+jo8MIxcayz6z6sdCpEWNh5o8D+DgRvY6ZZ1PaeROa9on+FKSk8x07OJaZ0E1PeQLKkWcYpu6XoO6pckVQOjc95QkweYL7r+POAsTtFWthxaBANaxgSweTEJxlvPPqh06Mn7Bl9M+e25mLPIlJ41EayI3dEc469J/+qxj8xW8GZm1HBYrDUmo9netiePAIkiyDHRUYHjwCV1RIujnSbgd2VGDu7H2h4ODK1+9E0s3hygqdMwTwoogjoA6Upwt9FEdXUCwPYIvJPXuSPGuUU4+Xs3VBeHUP7A2WjIsqaKsbzJYV0m4TkgsArAwo0q4BIO3lx1XrNT44fvmf/B0+9YPfgawvsSpX2CBU2rR4468iW1iQSgR5F3e//Z2oRoW4CxUgkWewvqCrKhNsHdJujsGdh1EORsi8i1LfSRWyxaBE3s/AlsHWIut3vNWj7sW1+iy79XVc5xWQBkIvT4Ibe2Z06nWQvJiIXgrgSkjmPQCAme8z6xNtlsfyDxCX1x4AnyaiDwIIDmFm3nop1B0mfWhtYZH0JWlOLYxr3/ku/NujH4XOQgfGazv/9uhH4WFe+4rp/Y99NPK+YPCdd2t1FzqwXUGuDO4eoBrKg/+2Kx6Ex99SW5jlqIKzjHJQorPQwQce/6249m3v2ulLP+Gk/vo7XvqTSLIUxfIqxkeXsXzrEVBikHVTfOG5T0P/grPQPWMPnGdMo6PL6B3Yi+LoClbvuCfESkLbgyTxWdcWgzsPBwZVDkZwh5ekRpcXEuVgKBWFl1Zh8hSpFzzVaIxqWDfiKj3Ts0Aova7BZzImLC+WVxuuquHdR0KAuFwdhcxvQEu62NppHemG5IxcV6uyLmUp0m5n5vdiPbKlDULtfq99Ez7+tG9HuVIg7aXAAHjAG/55zT7VcIzxkdtAiQkC3Gmyo8+C16KQw7uPghJCvjCHpNsJdb4U7ZX3c4yOjJD2UlTDCiZPfPKoCmsO4zOJgS0t0m4KtlwrjNE7LkH72m0GiDXaOWMelJgAullDr56Ne/p4ujFnQK8C8CIAvwepdPwsbJxKcsy0mar0uztx0jYRUQLxAd7GzN+5nQY3lBjk/QzWl6o2hpDkCaphhfdc+7AgZKpRFUzq9ciWFuwfdJMYFAOP4XfsS2GXTZy+pyQzGC9ZAAlMLus/+MTH4qFveec0l3BK0mef/b1Iezmyhb6U3xgVGB1ehC0q2MKhty9H0s2ClTK48zCqUYH+ufs9jFT87sPDi2sgxHY0FriotZL/4DXRcnWI8ZEB8oUekm4O4xmfK6sQMxguraIaFY1+9OrecqVYQ7HV4XwQGoCUAvEurLjCbtCCvVCJcyt0vPrbeIsHqOHHdX+gBZgkQefsM4/TXYJ36Qov+eATHxuWs2Vc83dvnbhPsbyK1TsPo3fWvoYQdEWF0dFVFCslOgs5kjyByRMsfnUR3X0lDlxxr5DHM7zzMAZ3DZD2Ui88OCh37XGVhRWB42S78dI4uMgabQkcw8EBBRrH0m1nmgw5gYi2VCvsZKAeM7+DiIiZvwrgxUT0XoiwmSltBjf+FwAgot9m5kaDGSL6bQD/MqNx/AykB8GC//9CbLHBzfKnb8G7H3wd9t5rD8j3mS5HFdgybCFamrOMzkInLGPLjbLX77n2YVGAz4TtKCE463DdO/8FH3j8twbhlOSTH6pHvv99eNsVDxLIpLdaJpnzpyK97xGPhLNuYmxpfGSA3tmZdPLMM5SDUbhuZx26/S7yhbpyMFuH4d1HAaAWNsPSM/MStnRIMgM7KmELsSaqUYHR4WVQQmDPnGxRYfVOcacVS6sofcJeNZLmUMXyIOQ7JN0OjPW1vJxDNtdb26d+5EKAN6CTRkWAllqNuXgrKu5YqBTHCky30ziWCqFieRV773OvmdyXaSlWph76lneGGOQkofKl5z0DgLj/ABG0y1+7KzTGq0ZFAKlUI7k/93xe7sOgGCDJbkfW78COSgyPjOAcwxYOtrQwCYV76CwjyQ1soZYKBZe1zlmSI7jK2Afs1XJx1oEshQoHCoU+HnSKucJGRGQAfJ6IngvgNgDbLcU/kaYV6Y/HWsb+7ROWbZmI6F4AvgPAbwHQkgPfDeAx/vefAXj3tOdaPTT06JD60vRlkmCuWB9qjpMlvO8RjwwILnYMykW4JJlZ48Ka1qX1+Fs+grdd8SBUwwpJlmD18OrmO53k9N7rHh5M/3c/+Do85sPvByBxFZOI+0gzqgEEhlwM5NPd1w+B3WJpgNHhZZg8QTkoMF4cI+2l6O7r+fpRDqMjoyD4tVmSLS16+7rI8hzkldQ4QW50eBHjpSHSnuSikDFIcilGqPDV2LoAEBLpZMxShyyPkvaK5VXY0RjW78POAWUUP7ESfKbEIPFJfpo1zolrZOCr1WSyVCye8RjGbWw5z5LaUPuNnmcdsx0V6JyxByu3HUQ1LDA6OsTS15fAjtE/u4/xolhoqnBRQkh7Ke7+5EEs3GsPylGFJE+QdX0b53JtH5NYqFAkOCgXl1k1dCFOknr3mYMBLCPJRAl0EAgyjcYbMvw4v2xbdOqgwpR+FsAcgJ8G8BsQd9jssNcRbRZjeQ6A/wbgklansT0A1qqsx0b/E8AvoFluYKoGN0T0bADPBoBzfA0j9cMCVRAualIDQDkoUY4qZF0xyTPPsB71gX/De659mBw3IeT9bF3XwDT0lkuu8ZZOhcW7T+3Wx++65lrk/Rz5fI7hkbpp0j9f+kD8x899FNWoCK7FNPKX28L5uk419HN8dLkBGy0HBYZHRjBGkHfjZIxqWAVUHftkVAuLcqUEJYQhRg3/PCUG46VxYFwa2GXrkPW7ISZSDUskeSLxtzxBNSwjd1sRIMUmq6vpuuVBFOsxHlpco4rSbu7jPwxKgDRKwrOFZPurBZdGVpCDCJhiaYD+fWYeO90Wvfe6h2P+nD5MnsAkhOHhlSA0TJ5gcNcA5aBE2kuxdKuU3DGJQTmqBNQy4rDt6uEhUn9fylEVOirq8ZxjD7IxXrlLQtMxFRwqJFRBdIVF1s8CujMg7mAAeBDFaByUhZ0iIoI5RUq6+HDD05j5BQBWIPGVHaPNxO3rALwZwEsh7iilZWa+Z7snJ6LvBHA3M3+EiB6z1f19o5wbAOCyuT7ryw0ICoQtByYX/NyRGQ4AxUCYxPse8UiQIfT2dT0iZTZaJFtGZgzcqYfUbpCzDrawEr/yc5P3xSoY3CWCU5Bzw8C4x4tjr2XWbonVw4MgOLoLneCuhCE4x1g9tBrcZxqwFYZkgmsMQMOVmeRJEExJboLQSPLElwcpkPVzsKtCwJcte3Sf9cH9AuQDwLaoAvIr/q2kcbdqVKEYlKIxO27kVIiAGQYLTZP+dJ4SH+dJux0gzTF+92vRecwP7tDd2zoVgxJJYbF6eNh4L5a/uogkS5D1MyReyHQWOsjmM9jSIskSOFuhs9BBMShBPsYJ1PcsyUwQKCpU2Dq5h6N6rtPInRjnKjl/75yPxdjSgoy4qxMP9Ej7vXWtFjMBpnysdKq4wpjZEtGDfHxlx5nRZjGWRSJaBvAtPtgza3o4gCcT0ZMg8LcFX3tsyw1udKrU7yqMQx44Wzp0FjriWjG1bxbwWrVPshOopEOeJ6BtBv5sYWErh5XKYWgdktl3gztu9K03fUDiTwmhHJSwhcUZ37QXJk/wrmuuRXdfV5j5QDv8MVwhjDdmFKt3rwRGwJYxWhpjcNcqTELI+lmIRxWDIlgWeT9HOaowOjqCs4y5A70QzAVEuFVDsU4NHIqBRe7HMF4StJNJCKMjQxnjqIYbq5Aa3L0c/qurzwWkVx0jsmXtcqtdN1wLtCyBHY1RLK2G+E+SpaiGUljRoQwJiUk3l2zzfhe8ugTT34PBX/wm+k//1Z28lVNROZA413jJiiXYz3H4C0cCJHjp68vI5kWpyPo5nF+W9lKQESj+6qFVJJkJ9xGQuTQ9nfukRoIFtFed60OJCJkkq/OKnGNJYjUEF7m3rfXCOjNyfw0h32MDejCm91z7sLDvtunUgxvfBODviehvAAQ3StRYbGa0KfdkZkdEHyeii5j5a7M8OTP/EqRWDbzF8vPM/ENE9Ds4hgY3lBCSTJBY6n6JKfMadjko6yS4yCWTdlMJ1nuEy8ef9u34zLu/2hAKK5WDZcaP3fOZDcfyHV//BF5/zlVQGTY8xYP3q4eGSDyTsYXD6qEhSs+kr3vnv+CmpzxBGGtpMToygi0sikEp1qExOHDffXCWsfL1JXT2dpD2UpQrvv6WZQzuXm1UK1DGXUDcbMVK2YibSZCWwd00JNuxJrAGZSKHKywqyy2BIAFktbhs4QSIkZlw/gSJh+Zy43zxf7VsnXe9lUkFWlrFeGkctHRXWCTdLIw7yWp3WL7Qx/ioz/IfnjzuUhNZmWk3xcrtKyidw9GjIxw4q4/+OXNBcdN8riSvFTYyPi6WJyFAL4JW7k/i5zkWOJQQqqIK22RZilITIeGTTCHC3fllzjISf3zAgaP8mPFRqbzw/sc+OoxF3W+zo1MOFbYfwGEAj42WMYDjL1g8nQfgZp/HEku6ncpj2XKDG2MoZPRe985/wfsf+2g8/F/fG9brA1Z6yLAkOtaaiwaH9UWohhVGS+OJlsa01sdT77oZf3X2lZhPDb7/7k9Ptc/JSk/84k1404VXS06Bc1g9vIr58+bD+rSXYvn2FQzuHsAWLlQr+I6vS2hO67SVgzJyT3KwXhRuqgLE+EIoxcB6i0BLvXBg8CYhjJck6J92UzhIPKdEVbu7IsavZBJCNdJzmjCWygeZnWUkGUIZH1nv1lRsoOi4iowaL4nVFreSyvqdIIgUbaZlT4qlVSRZFhSd5T9/Mfb85xdv405tn2ReC6zeI+i6rJMABZBnYkXYksJ24hWoEV1qoWhcRRU8RVlWowquaIJrlMifV4PxISetdCB/35U0TmoLsZDiZ8KWFgkSLN92ZE2eCUVJ09slOsUsFmbe0bhKTNMKll/f0VEAYOZ3Q9BfYObDwNYa3MxfcXlDkFz3ziYSenho6H3gFPJRpNaQiZIqHYaHhpKzslKgHJSYn89hC4thYZEbQm+LD+WpLlCU3nzx/QFI/o8GzsdLRbAEjn5lEYe+fBQAkBkTNNh/vvSBYMs451vOwt2fPSIC3BfnrIYVBp5xdyMG77z1UWdoi0WiSXWUELi0QCJ5E2Tknjq/DQCYnvjinauFVmXIB5hNiBnpOQERjnk/8wxuLIIlJO7VGeCxe0xJBR8lhKybBqGlrhyttKt1xShJkPpOhs5a2FExw86G26MkS3DHXUewL3IHd/bK2IoVb4UmBlZjXAZhWbDoHAdAR7De/JwnmQgfW1iJTRkC7ORsep1jLtinEdRCJ8RBk9o6hAd0FCulj/nUjD/E42YlDE4xVBgRXQrgegg46n5EdDWAJzPzb876XFPNCjP/CxGdA0DbWH6QmTeNe5ws9OaL7w/j4Y6q1VTDyrtABGKa5OKfRZIEJgYgCKJ5RRNtUEridKe5M3sYL2nJcpKYRDfFu665FquHhij83HS7phEgtYXFnR+7C+VYGEFnTy4xlij+UjqHhH1ewtgi6wCAaQh1Xik9IihH6isgaFDfJBRcIcqMVAjasokiCoLGnz/JknC/nWUgsqTawicWJsoU2XEABcTE1sF0U99nJJFEyiTxMb2kju05B1uUoR/JZ5/9vbjshpl7J6aid179UAFLEIV8sIbAsAwHBIsSMDBdcYfBB9PVRagAGlvYkB9GHj2W97OJgkTnDBBrp8518XXW8nBWSaRMBAigHgdE8ROtvtFZyNcWnZ1FzJMotGA4Rej/AHgBgD8GAGb+BBG9DsDMBctUopuIngbggxCX1NMAfICInjrrwewUBajxqPIvsS8uyGI+y8NfM4y4yJ2WIlF/PCAurm80+vavfBy2cJg7s4fOgiT8iVvEYrxUYGVcITeE8+59RtjHJBJkLUorQjw1SFKDcqUUhs9A4RiFY1gGhlaADoUToVOslrIvM4ZRjESZXag2XdT+ehcJgGooiDL1+QPCrEqP5hIkUe3712sKAskfV2M/ca5TmymqUKGEGq43Z0WIVD5pE5DCjWk3D+XnNdZXDoa48/2fQjkY46anPGH7N22L9OaL749iUGJ5cYyEENxLHFlrqvU7/84oxF9dgACCa0qtBlsogq4I11sMyjXWTVzLS5OaVagouULcndWwTn7mMEYX9o3d3KoM7Qj5Vsubfk4OmmPmdmHDE1c2H8CvAHiIWilEdBaAtwNYpxDPyUXf8fVPRHklte82SY1/CVyj26D62kULJZT+5QCA3Jy66K7t0uNu/hDeftVDJJ4wrDBeLDCo5AXup4lvfpagHFvYUYU8SxoMSS2PJDUYFjZYOAkB1sP6xGBQBiPfuaGwrd7DcmmM8XKBfC5DkhukSRpySZTJ6ziBFGlvbVxNLZiyBYtVN4vUqqqrHOsABZAQFVL0fEOZrpJW0x3efRRZvwNXVsi7HVSjcSiNoqVm0l4e8m1GPkv9w0/6ti31DtoOyfthcM/SGJYZ86lB6QRx1e+lwYrP5zMfU7HhXrhhFYL48e8w17ZuH0FJHZdQ11U9X3VKgFo4kvPEHt0px9F1MakwJ817cYIofNA/vG0HZ42aHUlPfjpERJfAv1jeONiRwsLTChbTcn0dxpTWzslCT/ziTeG3+v1jevwtH8HbrxJPH1sOiCbRnFi0Z8cB5fWNStVIamVZD6O2DPQSgmOJZdz6ucOw7IVFYTG0ToSxZworlcM8ECyTXmICYi4hAUaoYFEhnhAwn4qFaaxnvMywzLhnaYyFLAkwcldY2FDOx4VYj1albpMrLBy8MPF5MppQq0JFS4lMyn+ICyYCCC4gjd2Njoy8e4h8Fv8AiRcuJs9AVkrxOyt9XrKFPrr7hkHLfknvmwEAvzb8wuxuYos0fjYYVrDMSIiQGYNBZQNQJcmTOoZRVOH9KZ1DniW+6Gpd+h6IStj4OUy7KaxHX4b585amBuOV2DHKUuDGznJgNrVLTcaV9tJguYQEWSvpBXF8ZUfo1Ot5/5OQvL/Lieg2AF8GsCPJU9MKlrcQ0T8D+Av///sB/NNODOh4kXSba0qJx938IQDA2654UIBQxnkYlhlPvev0CMYfKw0HBfIsCQyocA6FA3JTWxoqIAB4ISO/c0PB3QUAPc0n8bchIfK/KRxD1+n5HHMQ8CrARtYhHUW5JC0YK+BzM3xgVymGDxsAlNTbqTACaveaxhCAZqA5+PchAsYY8kw1DQm71gNFtPQLexeZmUtDRn/SzUOds7SXNhjtTpItHA6NRaj0EgPLHBJ6EwJGowpdH9OKLVBbiXKQ+WXWyn6GFCFmfIzEn8crF5XPwFdyHrDhsiRYO3W7gfodVStREyFl7M2cFJMQHKQQ7RWveuNM52kNEQHpyQG4mIaY+UsAHkdEfYixsLzZPsdK0wbvX0BE3wdJaCQANzDz3+7UoHaa/qNvvKWaWhsyqCXw//nSBza23wq9cuEyDK3DCwef385QTyp63VkCnZa4hzB7tSqU9+q3ouf2e0aswiDeXmt4Bq3YWyttoaLrYosxzxJvNYnlInXgCEkGnyRnvMVZC5LKu2ni/BTNvWFHtdDQOlQQphe7SXUZgOASU+GlqMPQ78Wvj4WQFko0WQr2aDAAdUfHPEPW78CMSnT29dFLDBIC/mDvZXju4me3dsMies2ZV4RjqYWYGwrxwqFl5EbdjvV+Ggfr+utTJCVbFfCE0rlGdQnHjKyTSMFJH4BP86YSF8+rXjs7mXfyFo662toUoOe2WUTWOUaaEAyArN/Fl3/+h3Hv333NMc/ZZkSnWB4LER2AVDJ+BAAmon8F8BKPwp0pTa0SMfMbALxh1gM4kfTtX/n4hutjgaJFF99yyTUNt9okeuXCZbDMmPBOnNL0jIOfxpsuvBpFpCUmRN4C4SAM4jhJLTTq4wRXGTetFUAYmwoYXR+7x+J9DFEQOKNRhdwyxssFso7Ee6yDr6ZLQcgkuQmADQCwsAHppG6ZalQh7aZR3TlBQTmYBuoIeYLEUHB/qbUSrtMzYYEw150j2Tog890mfYFHW1ZwZRk09WJQSDsAAvYeo0vnxv2XAxDLMDfkQRByn/T/6866Er2kthDjKhHxPbGFC4jK4aAIYAvL4qaEFw6WBTpeji2MFzpwgPWJxbm2s0i8ILdaYknui+p4tftL3GlxTEsFStwwTJOjAc1FczsqVOREOJkC89PQXwJ4D4Dv8/9/ENKe5HGzPtG0qLDvJaLPE9EiES0R0TIRLc16MCcz5f0sWDDTkAab1Ud+upCtXGBKiujKDWGvf6lVK153fxZGJQJEhVAzeA9Mtlg0yK/blM55940IF1tpp8oa5Vd3r/THL7T2VHOQ6haLYbVKJtKcNSgdJ+8p3DhOxgyZ+n550+JxoWRM3H+9Gha+nW9dGeIJT76vn1PCqw9csf7EtuiGfZf7vCsT5lnn1/r4lMxrU1jr+hXv5lK3pWOZ33JsPYqvFvz6HOhyx4zF0qJ0alEiIP/YynFsIbW+nI9h6nxqx0h2LnRwleVNLS3OKXK+zlitPCQoB2PsPNHMUGFE9KdEdDcRfSpatp+I3uZ579uIaN82B7yfmX+Dmb/sP78J4IxtHnMiTasKvRySSLOXmReYeQ8zL2y612lEg7u10OLm0fufWfosRo5RsmjCP53ce4dHd3zoNWdeEZiSCs5aIHAQOIm3JNqk28SMrk0xg8qMCccxreOJBiy/9XyWGUnafKRN5DaR75qh6fLgz3cuQI21Em8ch6tLlpiG4NFAPyAVIGwpAkYZZjWqUA7GUk3ZC5U41qPdMevjOV/3SmqmPfC7L508WevQH+y9TI7rGf9K5bBYSt06QNxe6sZKSNxhKrj13sQwcBUw+tFlYfzR85AQYdHHUtRNqZ+E0HCXqRUU3x+Zj3ouGjExx76vkmvsw1ZbiBPmzuwh6+e48s//YUtzdqxExkz1mYJeDeCJrWXal+q+AN6BZiHgY6F3EdEPEJHxn6cBeNOmex0DTStY7mLmW3ZiAKcKPf6Wj2B5cYw7Dg+Di2EzGlqHewrJw3heem/8f91L8PPZyVUifauUZ0nQZpXp62/97iYGxiOLauhwk5quLjSYlfJytUiMD9rLfvUxupGLJ7jH1JIoLIqVMtSpAhCSYmW9L69e2PC7WCkDClDLzYRuh4o0cy7ktLjCBsanCMLSZ5vr/nJsyXuxZd1SYLwkfYOGh5YxOryI0eHluqJyKb1oAIn/jZfGwTK4Yd/mz15CUiHCMuOewmGlqudH53il4sa8NfeP71FTwOi9V8umVjBqgaVWbLyutlp8fIooFAhtWh+TBUyc8yL/XSijo8mtnYUOTJbi8j/5u03naCZEs7NYmPk9ANoV478b0o8K/vsp2xzxf4FUrC/85y8BPH8nPFDTxlg+TER/BeDv0Ox5f2LSg08QdRPjXQObWy0vHn4Bv5jfxzO+ybGGU432+rwUZRwxLDify9Ab1IloGsAtRw6mFXiPrR2dmzg2Y7k5SZqaqsdo573EFpDzrha1cDT+YQvrEUs+D8ZJS1tdBogVpLlN2tEw5LAkFI6vhTWBqJy7jwEEzTviJXElhzgLXTpiCgw6VGj28YFiUMLeuoy0myLJDb7l0RfhI+/66lRxO0HoAYVbK6yVwfcSwkrlgotN5pMa9zW2OtsCRu+DUlxkVazHGKjBYZneL7FYpAJ41qkFh7YuTrIEidH5dCEGE+cUtVGd6fEuiUMEyqY+55lE9OHo/w2+7cdGNFVfqmmJmfdsvtVsaFrBsgBgFUCcDszYgaqYJzPFEMxpqNbayPutBc75ij2XBrTU/jxBZ0+OhXvtwcodKxgvFvjO2z+5Q1dw7PSWS64BINBeDbAPo9+xywIQjdQW4n7pRUy26UKJhUQdMI6FkEKM1QWj7jRFMSnp9nF+RUKEzJcXiYXNylhKliTqoklFiBSOoW26BNFkA6N3lnzCYzPIXLcb5pCfoYFlhR2HMWZShkSqayuSrGaS46WxlISPkkGX71gJsGN19924//JNq2ur8NC5GVqOgvTN7VQAxDDu+h7VQiZervupJRUjBJuxsSbqLzcULNBhYZGQKA4akNc5VsSYBvEpiSpaWx/jSqRpmDGEfD5DNSp2HmLcINpKguQhZn7wTo5mGvL1wS5GxPtPSNl8f+JnbbSeiH6JmV86myGdvPTkOz6F15x5BYZWXBLPPrLxy/0boy/ieem9G26e+VRQPip0LDPKlRLjpQLjxR0sPbFNskUdtBXttKnVjkYVev0cw0HRcF2pNSKCYK1EXq9SdM3EuAF/BZrCSbeNGZaOMyEAY9uIA7VjA0BtqeQ8eSx1k6km/DVGLlmLBnIpyZPQkpctY7w4RuYLmobGV77svBZ2BKTBVmehI3GelRJZNw1dO2ML4NUHrsAzD0/2TsucKrwbPs+oZvpxjlFb0AtqrP6v28fbiUBvKgLiekOkFLAHaZhoP7lHlBBWRhbzqcFiaZFHcyruSZlrg1q4oIhaGDiHNPHFRnODzt4OumfMTZyLHaWdR4VtuS/VRkREfwrgagA3A6HY244YCLMCYW9a1v50In0PNEi6GSUE/G/+Cv43f8Vr+rUrQfMAVm5fkdplzHj9OVft4Oi3ToqGU7SQkmrE6tKyhUXmGYnW91Kfuy5ra7O6XCkWApPiM+pajLfRc8VdOlWgaUxA4wNNbbx2v7WD8c1EymZSpVLsilGrRTPwpU971UCLaT0tRa3pNnp8Te7U2M94aSwtmj2o4MC+bri2jUhRdypwm4KCQr5KPE+xJdMGX8QuTxFWzXWxxRPvH2+nx3Z+rhOSkkq5oWCtxBULYtK4lua7tIPh0rXS4b7XH+8KU1KEcprPMdIbUfek/xFM2ZdqA/oPzPxgZv4RZn6W//zoNo85kWYlWE7hyMHWKA5QTkO/V30Zv1d9Ofx/8fALAaWjdbPyLEHpxFWjL/FfnX3ltsY5i/1v2Hc53n7VQ+AKi3w+w3wnbQhF9cerZjv0cYxRcFvVQkfnLNaWJ7lcgBqaGu8XUwx3bW8XCyb1+wuqiSOBVJ8z5mGGqCE4AATIawwCACKEmM/B0P8xYkl7xyjF2eZavFEqPEhJlPHSGLZ0HiRQB/+TrK4YEDPrjQL5ceC8DSXWa4/vT/PTdG3peWNrpz2PahXFQmqSAFSrsp9KgmvPC5JQsy+4u/w4IwCFJr3GDcWC9Tfa2f7265Ix0302ISL6CwDvB3AZEd3qe1G9DMDjiejzAB7v/2+H3k9E22MMU9KsakZMx2VbREQXAvhzAOdCTLMbmPmVRLQfkrhzMYCvAHgaMx+ZzVC3R88+8hm8cuGybRWjfOHg83jdWVfijDO6Ps/BhYKNeZagKKVG0+vOkmfgGQenKyPzmjOvOOZxvWLPpSgc44WDz+MP9l6GszoJckOhLpfJhRHMR3BeZdi9BB55RA1UUNuN0mZscSKkUB0gBpQhUoPJxVp4nEHeDCpTYKgxgqmthW80V6GrpI7M1r3XNa+lboPtgMQ3CPO910OfkXA8GwRLMShCgcrxcBw6LSrUWas2A0CSA2XJSHspipUS86nBPYXFYlmXxlkz9gD9rq9fL6WXGCyWrmGdxPOj8S6ZWg7zHQuROsi/Ft2n1orGztpzrLEvrfWWxy0LfEKpzLO3qLJkTWY+ACTewkmyBOOlMa75u7euey93jMiAZlTShZmfvs6qLfWl2oT+DCJc7oSAsEhOzVfP8BwAZidYjpXLVgB+jpk/SkR7AHyEiN4G4JkQ/PbLiOiFEPz2L85mqNsn0YaPHeH1gcd/K84+u+8T7Qxqd+ex0w37Lo9cU8c+tpf17ytlW7xwGHnmtRL5+VVQxAJDqJnVHQsBRSDFgiOmOIAfHzO2ZPTcEoQmJFQLjRrN1MzSn3yeZjBa50rrXxkrLhvywiN2hWmcJRQ79mX8Yzis1r7KfI94Z6VjovZw166Kat046+CGzje+csEVp3kw5UqJbD4LCaAq3Ne7x3FpHL3GvVkigXIjgqlwTReX3Eupaqz7aGwrFuIx2ELLv8T5Se3E1ub9VOu1tr5UOIjQqOdYkXcmIViHRumXWqBzo/zOcSfCqVbd+E8B/DCAT2IWTGcDmtWs/M2x7MTMdzDzR/3vZQC3ALgAs8dvz5Ril8srF6aLs8QUa6TVsApZzQACnFeFw1YskFj7BKZ3h71ijyTgqQ8+1vhledPHHpMGfCetn8Rk6sB67BKrrZi129bbxIKsLaBiKLOeI84Ib1Ps8gHQiM+Urr4X7VpVTvt/uDq5L3blKJJMNW7nWxqzd3WJ+8uF3iRqpagVozk0ahlpIcxypQzXIZUOzLqlXmL3oyLBLDO+586bcZd3GSUkAqouBCr3fbFsJkDGVl/sUlssm/Gztuur7TJT1+QkxUKFiForSVYLG+kCmTSESlyPbenWZaweWp04DztNBAIlyVSfk4S+xsxv9Fn3X9XPTpxo2pIuLyeiBSLKiOgdRHSIiH5I1zPzf9/uQIjoYgDXAPgAWvhtABPx20T0bCL6MBF9+ODBg9sdwpZoK3GWNtlSeoKHFqs+lyKufzQpsL0e3bDvcsynNdIszhnYjF6x51L0EmFSwmgoYkZNphIvawsMjWHoOWPX1Yr3pev/Wqg0YyVtql1lcR2yZiwlzCmv73ar3T7x9s2Yi/XCvV1RIBYukwoiaoKfJl/GXSe1aZgKHECqJ5eDErZwKAfS8KwclBgdGaEc1cH8OPFSn4m2kjGfGrzmzLVlXn740C3++ayv485RnWyoEPHYHRYH5tv3PiY5ZvP51+PpHOs4Y3dcjNSrz+OFsU84DdWQvUDROdffSSatqLWE/3hpjOGRUWiJfdxJUWGnTqOvzxDR64jo6b5M1/cS0ffuxImmtViewMxLAL4TwK0ALoW0uJwJEdE8pMDlz/rzTEXMfINHOTz4rLPOmtVwNiXRBKWPyGJp8eIt1AP7zI8/BQCkxHtukM9nsOwT9RTH75Fj0wiHvz33qgBf1rHFWqTGadaj2L+uGnDs0ojRQjGyKg6Gtxl323rQZU2EUtPFJds0k+hixqdjXVtDbP2A8yQNumai1LrOdmVlalgxQN3XHair8wJ1gFkZpFb1Vas07aZYPTxEsVJ3TbSlxfDICMWgbBS71JbNo5EIqtGoCgpIZ0+Ozh7x6a/4ys4AJtYQW/H9ciTDfrLlK7XEmrktChtWS0WQfXo/6zlXRSa+p+373qwhhsY9zIwJxUKDlWbquW20Qo4EO/vOoMWgxPDIKDSMOzFISjrVBEsPElt5AoDv8p/v3IkTTRtj0eTnJwH4C2a+hzaBPE5LRJRBhMpro0SdmeK3Z02/NvwCXn3gCqxUbkuOylue9WQUgxLFSildD73VEr/0lBASJy+4ug42o3YQXFE66yFzYnrByufwyoXL1uQz5KY+ro5PhYoeP2ZKi6XDfFrDTOvSIS4kNLbHXI+V1zCnWrjUSXhxYD6OlcSxhPr4TeuonbzXhMzWyzNjgissobrv+5pgPSQbPy6hz652XSnDFCtG3E/loEA5KHxfGC9gvOtM22XH4xF0HcOyQy9JMF4uwpy2c0ted9aVDZDH85c/FwAZP7PULLev8bNeYvyx6jnRe2UZPt4SPxe1hbI/TxpCQ7P92/Ov+6jAUkHkmJF5RJfzfW5UUCvFxSmlO2UBIPfCW1peA9K99ETRqVQ2f7N8xFnStLPyD0T0GQAPBvAO35p4tN2Tk0inGwHcwsyviFbNGr89c3rm4VuwP5eGV/3E4Lfm7rvutl963jPwb49+FIpBiZXbV6J6VgIt1cKJ6krJs+lelFcfuAKFY3QTM5HZAvWyzSyXewobGJbs14Siarn15xz9TNCG6/hHjRCbT01kbTWhxhp0jl1ZOtY44C/L6++2C6veb7IW3ra42smB7XWxxeSC243CfYnL7LdJ2uiaYKmoMNEA/ujICOOlsXeROQy84GHrMBwUWBlXKMcWI/9frzHW9ofWYXFUNRB3OuZeUlsNrz5wRQOC/Pzlz+EFK59rjpfj34zYjdWOkVluZuvXc9ec97WoMdlWx9b+6NyqNaJwbp3nJEuQdZt1xGLhw5ZDT6BjdUfPhIiAJJvucxIQEV3qQxmf8v+vJqJf3YlzTSVYmPmFAK4D8GBmLiHlXb57Bud/OASl8Fgi+pj/PAmzx2/vCOUGyHxpk81iIbawWPr6kpT4zpOAJlKKq/Jq5d6EhHG88+qHrjneqw/U0OIjRRVpjvXLq783EypajTkmTW4cWifZ0Z7p3Lj/cvQSg/15ssYldU9hcdB3IlRBE8daFsu61W2MJooZU9v91hQwTQYWM7tJwIJ4eVtQxRbTJNLyL5pTEcrkRxngmh1fDoo6GbLyPV/8dqXzpeZXy4CwO3xkJO2uvZDQpFPLCNvEyYbtOIWOOQ7ex0z2hn2X443n3Q837r+8UTBVhU4s4GNLsLZcajdZswZYs1Noc/laga4UC/Z4XkOxzqiYp5ZziYER7eTVYrX0la/lmEulPUFlkAggM93n5KD/A+CXAJQAwMyfAPADO3GiqVxhRDQH6Zd8EYBnAzgfwGUA/nE7J2fmf8X6UOVZ4rd3hA6OLXqJwZHSrmFqAPDe6x6O+XP6EpQdVaDEIOumE9vaKnSyjEqQAMBZ586vOa7WGoshum3kVs2oaVOhpxQz4TpIi9DvJIy55XqKhVltxYjmv1K5KHaz1jqJczFUMDZzI5oTG8ddJrlg4vPE/3WO4njDZm5CtnWL3izqMlrXD5P1bmyRz2Ww4zIwzhBLYZ2vGqmW+ETMOJ4U3yNxc7mgWKgrUFo219DgZgypnru4BM7QMl65cFlwh+m59J7Gc6PlV2LhoS6s+H7EFQ+AGOnXBG3EgkbnPRQCZYaxUd8aAOS/5QA26lZZuwnZcqhRp+fe7D7uJPHJIzSmoTlm/mArjFGtt/F2aNoYy6sAfATAw/z/WyEQ420JllOdXjj4PH5r7r4TGeA7r34osn4WgozaLlcpzn0AEBhVG5VUDMpQ3FCpdns1S8bHAVJlFPH6N553Pzz5jk9hEr14+AW8Ys+l4TriIC7QjE3EjEuZUOzqkO3JF5+stePYMomPOYnaLpk6uN4WlM0qy/E8tLedVKZEl8X1zZTiysdxjAXwJVyKKlybHRQB1OHGzT7sahXE4ANtCRDfMx1XLQDrY2gXyJjiuayTUmtB0b7+SbXtNPCvve71/rat0fa1xGjB9jgSIvRTg7SXYmWlrn+nY4qFizp9Q+UCb+nVTbsQyrwkWQILi36ahERLPe4JIcLJZI1MQ4eI6BL4hHYieiqAOyZtSETTVPO8h5mfOWnFtILlEmb+fiJ6OgAw85BmFb0/xelXVj+PX+5cgszPxuvOuhKLpcNZHXEV7bvXQshHyPqC6KmGVY0oCslyURmL6EUZDgp0WxnWezPTYrTq9onrRBH6vRSDYRVKxwB1u9pJ1XFjv3gcmI2TEjW5rZ3JXict1jGZ+bReL24xbgie+Bgxw4qtkvb42vuqu60pKCYLE/nfFCpxAzFldjHEOQibisFWti0rh86eBCPrwvFU2KoFFrsl4xwcvba4xI2e657Chnug1a9VadF7ElsGtbZeWxs1NJvCs6L3PSFqFK5UqLI8drX1IWOk1j3W9fJPn3GlWAnRYPpwUETCsL7WzJgwr6NRBctAz0rRScC7ybwb0Y0ZhsS9qO9H2kvhVktYZuzLZ5XjfSxEEmc5degnAdwA4HIiug3AlyHtiSfRFQB+fINjEYA/XG/ltHelIKIeakl3CaK+LN/olBBQMoeERI03nNURvD3yBFk/F9+xz66WwnkMQHIWkjxBNawaL7USJQbvvPqheOwnPgigibZRUkajQXbLEgwNjLRqxoEmVWeOfeNNy6npSpJt18Y0aotBNlqpOPRQj4WGWi5bcdPpceNAc+1uaSOSaOI+9bJawKzpTBm5q9azqEwkDONCnELtMjVNhFRu6lbB9bwAAHtlhBrzmBmDhFlgxstFQApu1AJarqteH9d302TY5y9/Lgil+F7EykHT8qufCUmiZFw0lzWEnVoSSjo/8T3OjMHIN8BToVc4Rl4R2MaCXa5jaB325wnIMrioGmV2ABEy7ZYNx5VOIVQYpHzL44ioD8Aw8zIR3XudbX+Fmf9lo4MR0a+vt27aWXkRgLcAuJCIXgtpk/kLU+572tNvjL6I+y10ADSr6X5ttRI3mEcCxfh8zYOQ5Lk6Ia7JuD0T7W+OKonRQiF47jXgGLUVn6NN7XpcbYhuzMhj5qPxmLbbbDFC7sRos2bL2rqse7xN7G5bz4de16uqxxf7/uPv+nfzuDEjdFwzWZ3TmHRbx9LqYBoqHIe6XvG9bebbyHcvMegmRpqNESHPEsl38ggqQzXQYRhZSzH0WOfRRYgvdW+pQKqTVSPLuBXrmjROHask0ZpomZx/qZScktI5DCob8rz0mJY5CI76GXHe8nTBPaiB+Rr+7FCUNgAhbOHCs6Vxyc0AKjtBTAQ26VSfk4TeAADMPPCVTgBgYkloZv7r9jLfznhho22Upu3H8jYi+iiA/wBRX3+GmQ9Ns+83Ch0c24YLSV82k0gL1tJDSbWBETvfCtcLoq7vZlgHamsGX0SNoYDaPVIjepo+enGn+AZSRLDsApOwTEEIxT093nTh1eFF3rung8VlNUjrwPtKZQMTiJm95rzE59d+HPvzJBQ9jAP7Mcggdk/VLp6mtRELiTpWEwsO/d903cRWke6ry2PSZmJtaveCgXMN95tlKYOi9yVWqHWeATSKd8aorswY5FquhOos+zh/Rsv9JLlBOarnMq7fpd9qcRhqui7VMppP5dn423OvChBwzTWqt+Mw5sVSLAYVjNLfJXY16rMhNcBiIEfhXDh27LazXMPSdZ5iMEESWSzxePTaKCHMpzJ3ztb37W/PvQrfc+fNa+7hjtIpEGMhossBXAVgbyvTfgFAd5N9XwfgvwKwkDj7XiJ6BTP/zkb7TVvShQB8O4AHMfM/ApgjorUY2G9g0hcjN/LQ/8zSZ/GClc+hd+Yckgk1nTRnRclFWnUb7QM0+4DEzFlJGZ24SGqrRTs+xi98TK858wq8+eL7gy1jbyYv+z1LtZezjiFw0Hpr4VIzlqHlBjS1l0gmvyKQdHyq8caWQ406qt1ctTXSnJM6ptScn/pYNbPS/5NcYbEVo9vHY1iP4mvUccaKQLutQnv8uZGsc8sIsQY9rwqVugS/5muIcBHEYH3d8RzMp/V9B+R56vpnQc+rsTkdq153/BzpPKxUUltsaBlPvuNT+P67Px2O00sIF/TSYAXXikLzOPcUtlGhQeHrcZ6TUn1/a4ESKwENF6GH46vw1TEdf6JTBW58GSTD/gzUGfffBeCBAH5ik32v9NVQngLgnyDI4B/e7ITT2mh/BKmG+VgALwGwDDGrHjLl/qc9TQqGv/+xj4YrLKphhc5CB6uHhiBfUC/UBauaWnczcC3rFDGmqK4avlmjeZQhtF0+bVeJWhsxQyzHNmh97QS8xFs8mmtRW0hNJqLAgRoKG7vJmkJRqie7xvWu7UBYbz/JFRdTe94mzUG8bZu066Ru2xQWTYDBpPmMGWd7vPNpW/iIS6ub1AHs3FAjvyO09rN1rksct4ndUBojUWYcKx2lc435bVuV+/ME9xQ2shrr7+ccledZvwEE6/b151yFp951c+imCphQETuuzrAWlSfPw9Ay9mb1sjj+Vt/jZsxHE4FV+GoL6VC40is77QoEO02nAtyYmf8ewN8T0XXM/P4t7p756ihPAfAHzFwS0TrO9JqmFSzXMvMDiegmP9AjRDSbRgSnMY2XikYxvdzDjzXpy1YKOa1LsLSDupaBpdUyMPpX7Lm0wfhVIMTB5pi57d0jbW5tA61UbyfadRW9yGvzFeT46l83jXIzscarTFYtn9jdVVslup24rXQssXtFYi6xddS0etqpT/F118FoNJhyPD8xA47ddvX52/GYpmYdB5djIV0LG41lyD7xGOSauNHULYmFCjNs2USWxdcZn7+bGAwq2ziHxiSaQqgGUMi55JhZJwEK2wALrHH9TaCn3iXuph8+dEsArNRChUMi7HxqcHBcNRBxMj6OrBaZX7U49J51uwmKYTPFQucmFjo931WTrYMdbcrvZk+ngGBROgahAgB/DOmJ9XEA7yGibwKwaT3HaWelJKIENSrsLOxwPf/TgTRoT8aE4DxbjwzzzLUd2Jbvte6bhAjdboqzOgnO6iRBKKy3X1hWSNAzdtu0y9xrjkTbBaXMop1Ip0zqmYdvibRcYRRxMuXQcqtMjPrMa0EYC51Y+MQFD1X4xDGEtlWjFM9JvI3+Vquivbw+Vs3sm79r95nur/MnlqPELuJS9OKiqTXq+FutFQ3El04+Ov5mXGGtmychibm00WEaW1FXmwb89ViGautoMKywN0vq2Jp32cZWymakLrLYmtO5V2stDtarK1SvPSY9ho5Nt1d3ajx/+vxTIlB+SowP+h9HlxjRqVaEcsvEzL/PzBcw85NY6KsAvnWz/aa1WH4fwN8COJuIfgvAUwHsSI2Z04Xedc21oUkRUFdp1eSvka/9JJp9zUjijOJGmYzUwORJcH095+hn8Ad7L8NKxdif18UCY3gvAKyMK2ggVWt5xcHs2K1Ta/nAPUUTpRYzDdVMJeAfo4ia26v1EmvsCoeOS4nEFB+jmfAolkpsnUxi/HpuoZqpxdbLSrXWQoktpnauTX1NMSqOWsy/GR/Q5XH+iS7Tc8bHjo8Vk6Kkwv6pAbQETFG3s46VCnWzqWXc7aYT653pXKubzDLj3O7WmeD33/1pvO6sK1tWr8zJWZ0kwKnjuY3npHDStK2O19VtFlQR0Pekp/k93hOgHgDr22Jb5g0TgWdNp4IrbDtERL+2zqqXbLTfpoKFiAwkkeYXIGVWCMBTmPmWDXc8zrT6xc/jPdc+LDDuR33g3070kMQfHKFS1S1W+szsNkMD1vqmY0bjvNtiaBmvO+vKUNZlX57iSFGF7RWBpC4JZSBqqcynTZiuMl4Nzp/VSX3JdRcx5LqGlL86qYsVjbWXmHBNteZdX1sMhY6FSltANOejHmP8X48dM/RmYLspMNtzrPsObc3kYndkbwJ/VVdO7dJTZlm7rWKNPSGEOmuKwNJxxZZJOydH50LnOq4I0E6iNWEOgTyry54Yqmts2cKFbHftkhnnFJnoPrSfx62QwprjeJnOWyyc9bpj0vHUFq8LrSliF6itRJgq8KFcqRGTeh1JavDmi++Pb//Kx4/5WqYjOqXyWIjoHAD/HcD5zPztRHQlgOuY+cYNdhtEv7sQEMCmvH9TwcLMjoj+BzNfB2B6G/l4EzPSbho68r39qofgcTd/6IQNR3umt8tqa1ylDravdd9omZSYSWWWsTSWLPqFuSxkLAPUcKvkpo51iHXA6CUJNLYhpdIZezPCmR0RSOrzloZcYh3dM6zCcZ67+Fn81dlXop9K/4w7j45wZicNwWFllspA6/pWkqewNzOBaaugiV1oMcXH0cz92AqSbdqusMkCpP0/zuOReWnGI1Q4CZOr3SqaYAggQKfrdQ65R3mp8NB8lLj8/pnzWSilj+WiUa6+di/WlgMgjDLtpVIi3jcKg3PIswTGx2PqasyyOu2loSyKkgqakXVI/PzHgfBQXshtjopbjzRg/uoDVwRr4x4/5lhY6nOvcThdXz8X9XPfztWxXBc2bcyTn5okN6BImL7pwqvxHV//BN73iEfCWQd2jO6+DdG1WyPCKRVjAfBqSHmuX/H/PwfgryAV5icSM/+P+D8R/S6k+vyGNO2svJWIvu9kLuPiqtrU15fqTRdejTeed7/jPpZ3P/g6nxHcZJocMbam66a5fxPF5K+JGf008X5lEzTknu+LAdRadFNblDyLgx6qKnBPh2cc/HRIYouD8Lkh3yOmOS5FGWknxKWy3lfHshIJTTlWE8G03nW2YzBxLGaSBRMz5OZ8rnXdKZPSY7Utg5i5affMOM40KRak1sTezKCdKBhbGjUjZXR8Aq0rLLJO0uhbPymelhvC3Jk95P0MnYVOuNbMKyqUNOMlbbJVE85edyatt4/bN2iuzKRcnq3QMw/fEtxQGr/bmyWNxmDtoqZAbQ23rUydizacOX5HVMCWY6mWrF032TLe/9hHrxnjei0Qtk50qsCNlc70SY3inGSuANiNd1lDcwDus9lG08ZYng+gD6AiohFEVjMzL2y823EkZhSDAkmeNMo8GKLjZBbXJD28KZRvIZ/IpUw5DgJLfKW5v/7PTZ3jELTXJYvbjgyRkGds8xl4sS6doi+cxmfi2I0KnZipxvEMbdQFCEO4oGfwjIOfxmvOvCK4uSzbIIhi187ezDS0+fq8ynCbriq9rnY+icyP/mq6yNoWSHPb+ti6PI53xOvqGEAbVt0s86/3Sr91zmvXDLw1VruV9Bylj2nFwIckk7I9zgp0tu0qCu4elp48EjtwYOcw30kxLCwoESSUVm7Qvi/qAjYJwUHcQSEfBvCdSWt3EXkrRZls1klgR7MrdPvkOz7l4y7Nfi31fdDncW3Xz6Ywr+ugxdsEoZ+a4AXIjAnXBcC3L3C+Z46cQ1o/z0qw4GQSGtPQgIgOoAZh/QcAixvtQESfRA2LTACchU3iK8D0mfd7ptnuRBIDoRUshwcrAfv+J8eL3nXNtaDEoFopa42wlBLgABqMvw7exxqcMrqmgBlUFljW2Iy4cgonvdaT3KBnTSMXpZcQ5tMkJKkBdbdIZX7KCIe2buQFAF/3Bf6eu1h3HtRWuG2QgWr69xQunLed9R4Lmfj6JlksavnIsZoVfWM//Xpur9giiQWcMm2tXaXbx2CGuDRN+/gqWBMC5udzFKtlQ6DoGDITWzCM/XmCbjcNFoO4tsTyW/C1faUHi1xvbIGw5fDs5PMZiiMWSfQs20gj6Sx0QoUHkydIk2Y3y2pYAc5Xd/BCRzs06nsipWdmx3SfcVCC+nFMTWNXltEoUhoL2aZiUguZOMdJ5znx3Ti17I3Om7T6TrB6eLXhlraFXVMt/JiJ6GQq1zINPR/ixrqEiN4HERJP3WSfuHVxBeAub+lsSNP2Y3nghMWLAL46zUmOhYjoiQBeCZGSf8LMGzb70h4nxaAMPU4kCfH4CRVtyFWNpHyLg4+xWOl+5yJcftNtEy/TIKYJWpmuUxeTBt8Lxzi0UgSfvr54dTIjB0ullxCuP+PyEEd5xZ5LcUEvhWrIaoVYXgs3TYhC18jacqmr4Db/r+3hEgsCiQFFWvwEVFis4a4NvNffk+IB+QQFUretA/2T0FfN87eFkgq9hKQXy6RETtWY1a3US6Svu7rBtK87JRwsi7o1NSGfk6zBalg1ysiLAKmCFaOkVnl3XzfEF53l6Pk3od8J21rDD2Ou6mrCgFhfzzw825Io8RzFIIH1touRZfU6QTXGFqESJXUcSjPxrWVknSQkT6bdVOryOQdbADNtQTKj6MBW+d2xEDN/lIgeDcnEJwCf9Y0bJ41nwWfcL7dWLRARmPmejc61lcz7BwLQNm3fAkmYOUBE/5WZ3zrlcaYinzPzh5DukbcC+BARvZGZ10+pZW66A2Zp7k5BcZdHDdjrOPTFNQkhswaFq5PaRENsBmDjjpSxNhy7YOKktvi3uncEVtu0MhKSMv8v69dtlJtQ2MnXFufbxExBmDNHcYpaIKxFu9Xfk3zl6gJRoSNuObSsubWWiS6L3SOxQG67wmR+6v+xS07PVwv1Gu0VP05qKTRcMlSXGNHy+pQQsn7eEAbyfKZwWYJqVElla6BhVSe59DIRxUgEke6bdVOkvRTFSgmTSwtfFVzVqEKSJTAJobuvi8HdAwAG1bAKx1OhRYlBktYaPtC0tmZFCsJQFFeSShvneu6b8PK1rk0KuTn6nCu6Ta03SgidhU5oRZH20FAu2TLIl9cXa3FWKXg0E1fYMfG7YzvPTwJ4LTPf7P/vI6KnM/MfTdj8dRBr5SNoFuGD/79hnGVawfIVAD8WDehKAC8A8BsA/h+AmQoWAA8F8AVm/pI/319CWiGvP9EeXukAkPHMypdPOR5CxkXCQwVKuzmUIrlqSG7cMbFOMKyDx4BmTavLRjXpGjEDJBRXpq3dOoAIl4F1WEilnfCN+y/HCwefx/VnXN7wcRdOqvA+f7nZI12pzseouxrK+dQKabr5gCbTkDFxYOxtF2ATGbW2BL9SLEzisdW5LGgJlbWuufZxYqswRrhpflFbiJXOoZcnAW4N1DENk0s7hCRPkHZTqU7sWyLEAibNE9jSgq0IFe2WCIgFkvdz2NIi72ewuQnPUu9MKWZiCwtbiiWsFR30nCKwGAv3Wqjb/5Y2xPhiikEmT/ziTRPv/XapcIzM1BZLnnn3FdeWbhPAUd+/hMi7ekWQ2KJ2Bya5EUGaaxfW+j2gxPhE5Fp4xvM/K5pRHsvW+d2x0U8wc+ih4iuo/ATEcGgQM3+n/773sZxoWsFyuQoVf7JPE9E1zPylHQKKXQDg69H/WwFc296IiJ4NaZWMs7M8+FGDpeBmpZlsTG+/SkqmxcUDAalkrM+69TWZxEpp+/IVXslrAuAqeGI3FNCMwWhsZa/X0uK6Xno+zaJXhv2co58JzZ/Uvz/JRQEokxcGoEgoOX9cO8u1GPTawHpdCiZqCWCagfm2ZTRJg5XlNRpNtllfAMXB/7WB5HocCmNuxklqZqeJh5ZVYagtU4rgu2kvDZaDMkN5Bhzy+Sw8H1lXLBdb1i14++f0xXVjuRELSDIg62fI+7lYOef0kS6Okc/noMRgz/nzWD00hC0sTGLgrEN3XxeusCgG4u1wluWckcJjJgWUZkgqrNVaaZyzqAE2gMR/NAambkURCAZp7u/TfN1CYk1HVhgkmQE7BhlCkhmYxKAYFCJcPRBipjS9YDmTiD4c/b+BmW/wv6fidzMgQ0TEzBq8TwBsWJqLiB4O4GPMPCCiH4J4rv4nM39to/2mFSyfJaLrAfyl///9AD5HRB00UgBnRpOe9DVcz9+YGwDgsrk+x+avcg8yBsnsLfyJFAsU/a+97AGE3AbV2idlewOxRdBsE1tbCfV+cgwKfcD3ZomvYttMgktIOhSe201x/RmXN+IoMh5h+L8zfylesFJbLa8+cAX25wkSknyPxTKOs9Tjmk9NyFvITRK5N+ptarRPM0dHr78dU2q7seJrV+sk3qe5Xezqas6vrheqXVm6b6jlFllgbeSSsxxyRmKhAtRWq2aEW49MAgBn08DUk34GdgxnYwFSC6IkEQuEjMQRskjQ6G91jZGhhiCRVr4G3TN66IxKLN+xIq6wSPnp5AnKQTG7YPYEeubhW/D6c64SYeytBn1PKCFkiQnviQqeBLWSpjclgA0iN7MeQ6ldRTzJJSaV2hS2dDAQq2ZW18tE4OkV60PM/OB11k3F72ZA/wzgr4nof/vj/1dIn62N6HoA9yei+0OS5G8E8BoAj95op2ln+JkA/huAn4VMwr8C+HmIUPnWKY+xFboVwIXR/3sBuH3DPbzfVd0JJmEgSXbE/I3p7Vc9JECKATRcbypURqM68Jpw7JqJk/04lJrXYHlu6mdrsaxrdvUSgfbmRlFNAFAH1TMjBRDnU/h2twlyA9w5atbJkqC+AgXQsGhiGloX8i50mzppUASC9tewLEUI63E1XbNitcg/RWolVFtZMi+1cJlEat3F1ko7SbMZx6m3UWHUPI+Og8J90/Xx41M67UnimbMXANKsjYMrDPBKRqRo6LbVsApatwJOYnepCiAVLp29nRBPsKUDfIFKSgzyfhaed3ZyfvbxlZicdegs5KGskLb5LUcVeF8X+7/5QDjOTpAWrQSAN553vwaAIIYGx//1GnWdLtf5YUchjsLWAUkCo9fgw1UKWjCJQTWSemIzFaLMsNuoVBDR1vndsdEvAvgvAJ4DeTHfCuBPNtmnYmYmou8G8EpmvpGIfmSzE00LNx4S0R8B+Edm/mxr9co0x9gifQjAfX3bzNsA/ACAZ2y0AxEh7+chwxaR35USg/c/9tG47p3/MtNBqgvMWUZianPcwAUoJ2DQjR5mts0qvbHwqIPfTW1ZSeMzMQQztho0e34+rSHHEp+R45zVaXZzfO7iZ3Hj/ssDKklLmb/+HGkC9YyDnw4Z0pYZ9xQO82nNuNXiGlqpV7Y/Nzjoy9UoaOBXVj8PAPiDvZeFcWuMJb4uQZE1LZJJOkH7mnUemgi0teihSe4ynXM9lh5Xrcp2mZj6WKJ968K4K6hB/TwoOcuN4Hwcj2v/j5WSaiRBd/3vbJ30KNq5uM+srRFglJiQRwUArqzAjhvvhjLX1KPIdlKotKlwjK5/N9SFaL3Vp8I0FrRtSzAE3v1cJa0b7aI4i4muS5BhjM7eToBhz4JmZFZsmd8dCzGzg1gg129ht2Ui+iUAPwTgUd59tmlL22nhxk8G8DsQf9y9iegBAF7CzE/ewgCnJmauiOi5ENMtAfCncYxn4hiNL2dhHVzhQMbBJIJAyaZo7XuspBoU0NS24jwBSpJgvVBCSELpjHa/8Roptjeru/ZpZVdl8Au+Iu2ibwWrnf5Ug1fmrlYMgAAA0H4Z6g5TAZFQEyIMiHaplsGK7xuzUjHmU24wXn23JalSjnHbsGowY82Jec2ZV0QC1M8hx9WM40rCsixOWtRlMYCg7TLTOZ1U/LGNbIv30fUyB00k3BpYdAvXHMODg2tHtfBCguy6jQoAACG4Lhea1PlOpUWn10E5KKOsfR+TyA2c49DSWoSJXpMImiRPgkav3yYxQCJCKMlSUGJQDgrc9bE7MH/+PI4HxdYLALzlkmtCML4pUNZaNTEFD4GdLihvEgPK6xjYLIgBzMJgORZ+dyxERPcF8FIAVyLqHMnMGyG8vh8i5H6Mme8koosgsmBDmlZVeREEuXDUD+RjAC6ect9jImb+J2a+lJkvYebf2nQHUl+sgclN+O4siIZSDSt8+EnfNvtxtuIqqlXZwjU0o9hNV/v0m0Fs7f7YSyRQHFeCjUtiJLnkrWgQvS6j0uwDf2eoJyZkuS4/DgA37Ls8INL02Pvy1B9bYjJapkVqYNWd+uRYUiZeQQOA9msx+Ob5HPc/aw7/eP63NObrhw/d0ohZ6HDqvjLNnIZaQLSTH7GG2SvpnDSh0Ru71mpkXm35THILxjwsuKF8YLh268h8xNDWNgRem7fFVa/jGl+azEeGGmgu0fS9UCqtCJhybW0wyfCXRExnnd/GNTT6+Hz3/+s3T56cHaaNkGjNnJ3ajacCO8SqgvtR51re/9BMDwi/JyHjtkPMPNVniuNsjd8dG70KYq1UkBDGn0PiJRuN605mfgUzv9f//xoz//lmJ5pWsFTMvDjltieM9MVRLUYfrnw+Dw2BZkmC5qmryCql3RRpN6196VGwUgsCxoxVmWScOV86h4W5LEJf1bWsRqMKIx/3qCsP10JI3U2C4KKGxq/uqLaSlxnfzyL3c+ZBAIrk0hiIBPKb1oBSNzHI5zL0eyn6PsN8En3/3Z9uuPOAunVubDmoVdLO5dH1TVh2LWgUzt20UmJodxNIoHPTdnvFx20CBCi4npzlugho5UJWfTWsvDUSu2Y0z0UEUTkoGrGDslVSJZ4/ZZBaZFXWWxhDMIbW7N8QNE6YcjWqPGLKwJYV8oUe5s/bi8uedmK7jD/xizchyaUthAAh1lqDk6wSase01qG62KZXOCcoDMdCarFM8zlJqMfM7wBAzPxVZn4xpCvwGiKiZSJamvBZJqKlzU40rWD5FBE9A0BCRPclov8F4MTXpV+HNNiZZPIQpV0x+2dlAk86n5rmbWQY0HSbaKKaBNjr4npNlJV/EfKk0ctcmalmwSspU64TDGskU26kBa2eq58mod6VxmASIl9WRJhZ4d17mTGhEZRYJs2Ci6rpx5p9NaykN3shbr9BZfGaM69YM2fPOPjpcL3ndhNc0Ev9tda5MXpt8SemOEO7FhTN9XGwXkvoNNc1jxNbRvW1NgP+UpLehaKHQF3a3nEzIVYZpVqwsetU0WL6rKjLKmaS2k0SqBlkOaowOjJCOSgxXioajLgYlEHAjJcKlB5qHAu4GExgsgxkDA7/wQvat+i40uNv+QiSNZUFmhzZRO+Zzq3Gk2I3mlou8f7KD2YK5mFEz9DGn5OERr4NyueJ6LlE9D0Azp60ITPvYeaFCZ8909SInFaw/BSAqwCMAfwFpDXlz06573GjxGci60MEIDx8ST5jRIgn1UKVYoGi+QJxQFb3SXKDPEsmaM61Bl0NBU0WV9tVJl9XE/bXTgr7pUaNrYQQSoVYZox8PkzcClYF1aCyGFQ2HJOSOvbT76Xop0k4bpvJx/81TmOSZsZ0m/ZmkrQZ76MCL76mdrJk2/KIXXSxpdJ+oetkvGasJX752y40Ffi1O64JmRWk2GTOERceVcZeDStBYzkRvDWybK1CItaNC3GY+PnR47etnDohsIa8a3AcEOGlsb5qVIDtbF1D26HH3fyhYJ3oR98fFSoNBGacEGkooOzCshDTEuCDczwza0VpVq6w40Q/C6lO/NMAHgQJyP/IZjsR0SOI6Fn+95keZLAhTSVYmHmVmX+FmR/CzA/2v0fT7Hu8iAjI+1kQLm2TWmGHn/7P3zWzcyZZIrWf/APfGI8xazQuGUcduDU+lgIgxDA0ES+PtLDYzQXUbiANuOsyXV6XZJdli8vj4MoqnOQL6Lb78wTn9zKcvb+Hcy7ai3Mu2oun3nUzVsYVCt8orOc17pGvkRUXllQGPbQuNBvrRQioro8bxbGWN198f7z54vuH8WiVZc2DifNc4usPMSZqVhLW4zRjMM1y6/X65rH0fLFwsFwDJZTiPBtbiVU3SaDE1YPXq6IbB/hjUqbZfpaqUYXVQ6sofM6JPHcUlJpqVAWXGAAYQ3BOwCImkTIumm8Tu5tcYeHKCqt33gOTJFi88eRqCqtxFQAh4VkToHV9ENz6PEbPngpjvRdsGc7JZybjg1SOnuZzIomINI7yMGZeYeZbmflZzPx9zPzvm+z7IghM+Zf8ohzA/93snBuq8ET0D9gAUbdTqLBjITJ1lrP2ZGj7XePA5SwoftAdmtoUgGAhKfw47Bdpk8Ba7V+r0YZgcIQWk0XNpD51f4VyKHMZ7KDw+1Ij56OXaOdHAJB+LrHFlXVTvPPqh2L/gR7uPLgahFS7LH9MyrDn02aw1FmJFXUjAf/Plz4wlDzZu6eDlZWiUWFAM+CV4nhOO84xCZrcLOfStGB0rmNBUlsvkxMy42tcz/KKj6+dGmOGprBzjckos2/DjQGseYZi1JMmYtaQ4yQE6CVb33l3j0GSrA1uNyxr65B0xZItlgcw+XkNC+BE0Duvfqi8T2jmguk1O/h3CaYRd1GhQoaCAtmmYAEZwizh1SePMbIhPYiIvgnAjxLRn6OVkLlJQcnvAXANgI/6bW8nok2r3W/mG/pd//29AM5FLameDqkfdlKRYvLTrpR1UCEj2qELsM2doCQ3vnJqM9i4UcE7rXDbzdPQrlj9922NV4WL5qdokL7W0r0mnkm2dg8IcQAt12IZ6Hbrc+U+o9skkmzmCgurTMwY7O9ltQY4KNDr51hZKdDvpaHuWbO2E6EoLfKQuMaN32+68Opwvhgxp8LLslpd8P/1u64srAKxfe0qKOq5aSZM6vWu57aaCCkOkPC16+r1sq8WWZxEcR0woHaRArWrdlqW7ooa2eWsQOq17D0gQiS4jiJBIhDjGs4MAKYrYyoHI7iiOq75LOtRu7ZfOyaiFp2DQeLjLO2afJPiNHHQfpYC9CQKzG9E/xuSYX8fSFHJeFIZGxeULHyCJAMAEfWnOeGGgoWZ/8Uf7DeY+VHRqn8govdMc4LjRbF/VS0XFTLjpQLZfD7zvizxQ27QDNLXL2kNN62KuiaUbqPaWdyYSTXOsJ31XR2zuud3roKrcjDUDErGaKO4VHnXu0I6iqSJmIwCG8ZL49C/QoWKxnqqobRGHkV+/di1lKQGCWotM259SxHzS3spTC7jywCUKyVKV+fhtOM1cZwmtiy0y2UctK8RYXH7W26AFdbmsdTCTK9FrZpYeMTCReuJuWh57AbT63bQLPx6rtkPWi0QLXUfWytxyRK2HPJgtGmVauXGD1xjLbErSIWWjXJjtGijNJ+TMidkDIrlAdK5Lg79/s/hzJ9udKM9LhRXB2+XcwGa7qQki91drnHNck0mzEtsQSvNSoAy42SKn6xLzPz7AH6fiK5n5udscfe/JqI/BnCGL1j5owD+z2Y7TRvNPouI7hNV37w3pEnMyUNUv8xpNwclFYAydJQEhJGq+T8LetzNHwolXVS4TOqXEYKOSQ0oCMP2QqS9j66jhIDChpfMJMmaY6jbTyvVsq2RXYpisixM3BUWnYUOTJ5geGhVEvm60s9C+3kACLDZtuabpAbwjbiAuG7Z5JgCx5IAkXvGF180CaGzN4c7Om4crxZYaPxvI8Hikv1tfpQZaVHQLO+yVqhMorYAarvBJlknkxQLg1qJUFcYJc2ilQEA0uodNJEpRkIlWLXRbipEHAycLx/TtJYcyJJ34Rqv3ADDw0vYM9dtuOGOJ4VrtXVpJIFFe2EYKWjx9kptRFlMs4QYt+kkQnxtSB4N9sit7sfMv0tEj4cAti4D8GvM/LbN9ptWsDwPwLuJ6Ev+/8XwVYVPJopLWSSZL1vRzUNdJABrAqOzImGSadDWnW88pOesS7zU1O5JrhaEbif/TcO33q6pFKrlauHC3KAauoYbqZvUiZUmF41VUXO2sHDdFIgCpXGw00ZjdMwwUZ2vujIxBXCAurWMra9DLKD6upV5xcIln8u8RSSaf5IblGMbBKMy8jiIHncm1CoGMfy5VBgvrYV96q2Iqwe04zNNl9v6jCnu+khJW1NOomdBLNT1NOZ2+ff1tmsLHHWJUSJBe62sLOu4UZxRhUlzrAbjI8vYc+HZSHs75y5ej9794OsaVaCB5jOi72yw5uLeNVkszEXoxqX19RhArYC1qyYcKzEmKxgnIzGzI6KPE9FFm1UmBgAi+mYA5zDz+7wgeZtf/igiuoSZv7jR/tPWCnuLLwdwuV/0GWYeT7Pv8SKiWpurRgWyfhcJUpSDkeSNdKXcC4rZnjcWVOKmMIGDGdT1w2DtmmBtuw6S1jcLL1MrsK4cj4xB3hWklqy3QFTCA0AImCtkc26h04Cm1tqqCy2dY3hme3ya2FlG16sxDKBm0E++41Nh/ZsuvFqulxnOx3W6+7qNuFNdkZqRzWehY2LblaeFC3UcjmPAAgG+k2VUSioIhNxM1izbKLJJy6NpbxRPbM+TxvJiwRHunb+PDiJodF0bPivHnLA/EBQm07B0pEx8DKOlpM5MD25Xz3zTXuZ7lLg152LnMLjtEM669mqM/ul6dJ+0VY/J9kjjo1qaphE3SZLGO6MKJDsOrYnrFsv+uYnmI85fo5brbLt0aoiVQOcBuJmIPghgoAvXAWH9TwC/PGH5ql+3Ibx2M1TYA5lZ0QBjSNfIdbc5oUTiXzZZiiQXuLHx9ZCqUYFqVKB7xhzue/3rZ37qOF9FTXbRjLKAUGNXCwX1t7cx+rG57yDl/rUSbTNpzoZcmFqrrY8pL5MwILUKdB8yAjMtUTOWtZaUWxMQdf7YKFRLq62GtjWgFLTQqhYSk5LetE+ILS3Yj11a6ta5DMGVhFq46DEBGZNCiBXBFsZBtMaFVltc07GGuOx7TJPcVcK4TZiz9v2NIbMuQvzHVm44/gSB0v6vSKf4fiW5QJItJP4YtHbtumjUEjaSJJkYlKtDsLXg5aNTzcmsKEZwhvG1Auw6S5SYBgJU9jEN92DiIdWN46owirwas6BTJHiv9Otb2PZiZv5EeyEzf5iILt5s580sllcR0WOAif0ClG6EwNFOKBEktpJ0c1CS+MB9DudzK8gYZAtTARqOiTTgrfQfP/dRvOfahwVkWtyALLYsDJqQZY1pGCAEbGNoKgDYsq6km+QE47P5gVqLTrt13w9brh0f0HQpxNaUswxbNRm7rqOEgKpdLkWY9DMONhvetbXI9c6lwjMmPZeWoo+TA5XlqFvOsfQ4V0uop7W0WgH3Oj8mbgXddn+tJT1GHBMBImEbjXmNQA6KxdrAfPt642zydceSJ3VNLFML3EmkJUzibH+gdinVioUJsY3hHXfBJAnm1p+OHaG2EEBSd4uUZ6RWwkxiGrBitdzC9WVJULpiAaVFOGdJp4gnDEANxpqSNqqB1dts580Ey16shae16eBmJzkupC+Zc6AkgfGJi9VojKTbQdrvhbjLTpCie+KyE7LcAHABGomgZSUNrZxtDREVd0bNfKVqM4cquXkihS61DS5QI7HKUeVLhDuYXi77RH4dW1okeYasm6L0uS5thpj2UsDDgeM4S1ubtsz4/rs/jdefc1UjV0UpWCHWhBIvj/3EBwHULQd0jorBOJxDYklNDqCxI0rE4tNcH6AWMI0cj9TAWA55JSJTdfzNFsWTkF/q9msjvmIrM+RWtASOUoi3tYSNzrlakZrvFAuV2FLRa2sim+p8jHZcJ1grPvAdV5zQoL2LKyyPxjCZgFoGtx2CyVMMXvQTuNevbwr+mQlpPk4SZc+rVQLU1kYWBejb8Un5broBATSvPakRdrMgBsOdQs4wIlpG7b3LIXiewTolWj5ERD/BzI2HgIh+DCITNqTN4MYXTzXik4TiFy3p5mDnkC/04YoKSXfDDpzHRO+6RrqHxkFoQNBiMTWYZPSyJN089CKPX46Y1jAkv63mgwBo7B9vr3GbtJeiWCnBTnJ58n4GW1hk/dwDBKwv/eFqKyFiXHVZEu/LTkVAanylXQpdiRIC2TrbOyZ2LiB5VPhVwwpZP0dZFLV26q0VF83NGkbeYriouBamUTxHYzOGYsFCjeC7bge0WhT7ZlOxZdEAJCR1PxRRGPT3+mXdNX+oPTdt5Ff8Lfs1UU7tnBiZXw7CSTX/9RBfrqiCYHFOnHNZv4u7Xv5TOOcX/tfEfWZJcVViJDUYwcEFIQjUbTFEOESABEOwpUO+gcDQd3DmtcJOdFr9FoiZG4mNRPQUSNX6SfSzAP6WiH4QtSB5MEQgfc9m5zrxGVEzIw4vNACYPEXW7yLrdzF33n50Dyyge2DT2mlT03uufRiAJnqFjMHjb2kK81gDVaaUZKbhDovLz9QQZQqfxvHypC6oZygKfkveSeqtBNXU2LpQ/oOM7wnutb241W3DP6+5Di0tuXQS+EzyBAkB33Pnxi0jvvWmD0hNtH4WKj6HazYTXDQT0Epr6qv1s6BxKsItFlxJniDr1JacVms2JNeSdTxAguqclpCVHbm82taBHjsWuMGN1Br3JEGqJUn0dzzPG5GORc7dFDZaskWXrWftxEgwZdQmet70GthauKKUmOSwgMlSHLn+hRuOb7v03use7rtfNmMk9bgpPHPxnOgy3a9pmZigHOmxdyL5k6G5LJt/TkZi5r/DOtWNmfkuZn4YJC7zFf/5dWa+jpnv3OzYO+cb2oSI6HcgyIICwBcBPIuZj/p1vwTgxyAR6Z9m5n+e4ojhJTHe3DcBctxB0s2R7p9N6s37HiFw8Lh0zEZ+cWcZuX/wYxRLI27g3T91dWT2ZToY1tUw5Cy4Ter9gxDxKJkcQGehE2opxS1w2+UuVIBUo0qYcKTVxtfmLKPbTQMgYH5KyOa33vQBvO8Rj0SZrO3a5yyjGJTBXZN2Eawmtcj0P1ALozgXJjDsiMGyb3TVhnCr4HV2jMS7xYyNmnJ5C0cz2eOkzlhotN0wsVUSx18mUVwGSPZ1jfseU9stuBY4UEPP05aAIrPW0lGKnwG2DiZPa6SYc3BFidFojHxhDqt3H8HyLz0LF730Vetc0fZIn81gtWAtmCRGdRl4wajX14JWx/PQVso2ikcdK51irrDvjf4aiAUy8QKI6KPM/EBmfheAd220zaR1J0ywQHDRv+S7p/02pMjZLxLRlZDWnFcBOB/A24noUmbesAwrkWbdijkPSFvW7oG9MFk6M6ECyMtqBzbESdrtUQHRxAAE33AcbKzhqS7kGWQtP3ON4lGLqO7/0h5LZ74DYyS+YhKDYqUIzJkS8m6vLBzPeeZZjSqxYBrauYewBmEpUOY0Yu4mITz25k3drIEe/q/vXbNMsshRB7YTX1ctYeTz3RA/yfo5ykEBMsYngDo/bpkfXcYeFKHFHfN+JvEmH5+SOZR9Ogsd3wOdGs2ugDrJMevn3l2XhUZaAQae1IFmFfQxlJuMCa2JYyulDj7XsTRF4OmcrGWGJM+FwpR9gUndXsEPjdhP5AbTmEKoTxfmXwVl6tF2ZTinjrkcjFAsr+6Itq9UtwpYy/CDJenjp7GFG6O+glBHGwSgsSQR4CZLZ54AerJaI+tQDBGuIFbId6+z7RVEtAYVFhFBYvATadrWxATgBwHch5lf4ttTnsvMH5xm/0nEzG+N/v47gKf6398N4C89vPnLRPQFiB/w/Rsfrzb5pdR4JRn4xvu8hwNgBg9VEBjqz07WbzSkvnJl7opmCS8MTB2s9P/VemgcJ3LNxKig2G0AAHk/9y9W7epQ4VINK+TzPijQCjdptdy4LE5IvIQINf2d9lI85sMb3oqp6LGf+CDedsWD5JjdWmNuu4aksKLm99RCkD2oQSG1adfAlg5kHUySNYAUWVZrtKFKQXAVCkTc+LbAcszavRkzfZMQrEMQMnrMuKNhjP4DIlepWVtgUinefj3S5ydmv4oe1HGE5dFzV7fEblqjk4SF5tXo9sXyqp+HnQW9ALUlAjRLAgmqzYGctyyzlsUdxZco8UF/o/e7fg8AjeulE4XYsdCplCAJAMz8rC1sfvnmm2BdZX9aTvtHAK6DFJ8EgGUAfzjlvtPQjwLQ3qgXAPh6tO5Wv2wNEdGziejDRPThI6MxTJ4G95cdFbBlhWJ5AHYO1coKUJVY/ZuXb2ugtd86guBOKG4XM+m2tRLG791KaU96yNTWSe36iPMVVEiETnhJk1HomLJu2ngB1Z2iL2vez6W1gGdAJhdrIPXL4jHoONJeit6+LvL+7EriZP08NGGjSCnQa019F0qT13Ok16pjZMfBalCEle6bdtMQY6nzYZouq7Sb+koEcZymvldpL0XWF0kcCz1B15lQU03ms74HGhNSa0ID9fEY9FokVrb+qzi5Wm/T+tV4S2zxNEoHBahzHVuJY5Jrjm8MqsGwEUOaNd30lCc0XInh+Y7nNKnnRq08vcfqxpXromju6/dD92mWQJpR5j0DpeWpPicDEdHLiWiBiDIiegcRHSKiH5q0re8wudnn1vXONa0qci0zP5CIbvInPUJEm8KsiOjtkKrIbfoVZv57v82vQMyy1+puE7afeGeY+QYANwDAt5y9v7FN8D8XFdAXQZPO9QC3vcZGJjcSFfI8hkxUw6hV6ylGtChjj9smA2iUsVBUjFo4vX1dKaDpGadmWCfdNMBVAXGPxDGebD5HNayQJP9/e28eb8tWlYd+Y86qWrXW2nuf7t4rSBNBAbv4EK/GXhSDiEYxqDEvCuYl4UGiQRKC8jSKiSR2kSTyEsUutig+I8EeNAqKoqICXhQUAfUGAvdeTrv3XquaOd4fY4xZs9Zee++1z27PuTV+v33OampVzZpVNZpvfGPMDj4zJpHBKD5R1sU019UNQ/SMXeFQ+By0UcScwie/9nWHmrtFeeIbfwev/9RPg6zJnniricHLpzlClURouYfPAVIDR22nTKgghbcCspJQbdaRbdajJ6tyNzzeah6ycRZzMvaZz13Md/ncgcssQnHR0GmXaJOd0cfCWj1+J3QFdP2ulvVcMyWZssFSz34Hs8x3XbIjDd91vcFsHLIfvYeNGVY1QN7lJKwW7DgkTcADABVJDVByr1KPjNFdl/SZ263wMe0wENqAbHxUDNGdy3OfcXkyM79AV468F8CXQPIn+66vclBZ1bDUROShCp6I7sQK69cw82fv9T0RPRPA5wN4EndtQu8F8Ihks4cDeM++I6QulAcAlyu8og52aFu08zn86HC9kEIV4LSgj1tGsVaguiH1IJ/+u7Ja8+v+1if3mDmpcTHFHYe9JGfSbdspBql6T1YCVG84bT+fSloMaasG5msF5tfmOwoljVAQIoNGxjDaGMF5wse+8tU4LvmU3/pNvP5TPw3OO8kRJQygqEyT+co3MlSbdYQBO4+9v/6JLzx8UjNEjmLBaar4nSdQnsVjhNzF/TbbTb9TthqPYlrELsPp/BkbL8JWBu0k0W2veLJqpRI8iTaBnRTjNM+wTHZS1Pssw0X4rc+6k+R9KkbVl9+3wDFCYUDX72txhcd+lNGnVFseKc0nLUcF9LOFtjlHIbcaFIaoDfFUAC9n5g/QHj3wDiOr3jH/GcDPAriLiF4MyYccark5InoKZGWyz2DmreSrVwH4CSL6Lkjy/jEAVsjldMlloP/whKpBO6sQxiP48QRbP/OdmDz9+Tc1bmvLLfvucOCwYGfFm+peL/NEyRPQdg/EYt8n+czHynRTbItssGa76SVo7bzTh9H2nyqovMxkRT3dd60JbRtrM2vwt16zlBBypDLaKARHTwyDkRscXJ867R3GF0rMrs3hc6894AQKsrlr66CeadYZzTbAF8KSM4ZeqzmVvMwwv6YLo6lRy5WRZ7TWtmqRlxL1kCfk2q6nm2+Ch+8ZQl/0C/oWDU7qNPQhrL7Bks/6UU6PcJC7hPlnuZWORhyN5oJCtdVPyXuhGtd1fA8kxIMjbNx6zz/4vJ6jkN6P1mfNxr60W3RiNNLnrl9IulgX1BWFHhUMBuCWq2OBLHfyNgDbAP6pBgjHshLwqk0of5yI/gDAkyAa/GnM/KeHPPZLAYwAvEat5huY+dnM/FYiegWAP4FAZP9sP0aYCSVhcTsTRWFskLhNfrgcQZqAt9cpzBKPkyQQ7XfCttLoRY1I9M4VTjEv2R4QWyTKJxCBedBO8eYIDSwoqLYOWoXP0ZMenRtFT88gMxuvS6qd4XEiRgUA7v7FX8MfPe3JqNWg+FyiF2nprzmKBepvuTESGNFr3qIwYxFi144InU1dpDU7TzHi6eeuhHU02hihvlHBFx4j71BtVrDFtIC+4QmhW144JM0N840sGrc0iW7LGliR3yI8CiyBvBb6gKX/R6MV+gbO5oicg1fWF3kH73cq6vS1K7pnw+cZ2rqRiOWI5B1f9aW9923VIldCSajaeA3Jd8W/KXmiI0R0uUev7ZuscNjOu99CR6HO4mgjr1stYmHmr1MG7jVmboloC7uzwg4l+zWhvJi8fT+Al6ff7bOk5Z7CzB+2x3cvBvDiA+4Q3Lbx4QhVA1dIQ0rnPdx0LIrJHb6dg61EmXYFXlYvAPS9qtQYsHLxzUNLE9OhZXhLRi5p/w0gQjvUUuxam1YtC+zTJoqra+5I3qHerMS4VVIb4woPulYhS5TpSYok6lvkCr+lkF+sF9LlffMyQ8ilPfrO1RgZVGQgL4npYk26DKSECOs84MpMIyUHbmXdeJuf2ODQdx2EbZxmgD26HJktCSxJfS9OQ0oRhkKSLkQKrF3zxW7Si7DXYsLfxOpfaEneZVHSwmFL2lvzSZs7n3dGCAAoHN098OfP+WJYEa8YeZnT+kYVk/a9801Yij1jmjg/vTlKmmsCYhh7cGACiR3l0hknkWMhoi8B8CIAHwHgE5j5jcl3B6r5Y+bLyetNJF2Oj1L2M+F/ADHMBOCRAC7r6/MA/grAo45jUDclybPknIu4sayU14JDQF6MAbd7a4u95E1P/xy0VYhwlC8c2rpTBL7w+KOnPbnD4deKCLWIYlMDkncNJy0Rma7kaIpQxt5RJ+NpJpALgBiNkdeHR5cFIOfgCvtN90BlZYHQBuTTAs12HcdLzmG0UcCaFhbTHG/7x0/Dh3//Kw88Vzcjf/Plv4g/ecbfif2tfNE1IOS2jXPBvjOo5YUSHLrOzz533fIIaAB0HQ4sgQ90SqitWkzvmsgy1nVAeaGMzLJ6U+o6UvhRSBUhUrEtb5V6y7b/HB0Vua0Cco0427aDuIxMYd0SLBdm0s+79JWk3EcOoTVCws78gkuMhv2/L+U4yc+41u2fSF1BLFKxeiMAmF0WBCYEjs1DLTKNY9EasEUIKyuNpZfF0oKu8LR/zi7PuswCdqd834wYK+wE5B7I8vDfm354szV/JyH79Qp7FAAQ0fcAeBUz/6K+/1wAeybmT1yIEJtP6gPUzirUm3IDjy6si3KdbwP5zbFCLDlrRXnGpgoJ1AQA+TSXAj2v0YAanh1JY4U4/DRHvVkvtK2wJLJDs13HnA15B5d3IX730GV67CxGQQGALzK0lRROOmXD2GNlD7p4z5lWXYuCKy+Mj7yYbD/5yB/5Obz5Sz83Mu0AgZxo2uUzrAuBKJeqB4PE3NM4Rz4ddfOjyfs03yLbqZOQe5RKpba+Wtk4R7MtxiX9XTFVONOgLKUKWxI5L8tIQshiTqRrfNhjpxUetRI/LJcGdOvsLHaftvobY9DZ/FgEtEy4Dcim4y4yzjM0s7nmVbpOFW3dwE2yWMfi8kycs6pZaoAOKq7I0GxXmF+bx3kwEoTNua1YuijZOI/QnDRizWIuqG3nyKclQt30I5OYq+k/U62e+1EIg08ECrO0w5JE+03V/J2ErAo6fjwzP9veMPMvEdG/PaYx3ZQQCPm07Iq8gimPApbA92UBrmZw44O3z08TtT53qLXTMIAIoQDA7NpcHpgEfrG8RlvLQzvaGEUqrM89fJl3rCjfdbs1WUYOsDVnUDU9j04Mj3hpHuKhxe/0YbQkbXnpHJrNbbgii5RSgcHoxI1KKtaaxuC+rCzQzCpRkj6DrdeeT0td1G0keQvfeaPZWJZM8Au4ussdqG6i0baO0aOo+Lu5yqcjhLqJ8+ksmb0deq11bD+m4K2rMzktYLT8jBogu2/yhHxh9TBAv2mpdWaQa6hFtgvsPZeUo7k863nlaX1QmkvpIh81rqoK7FyNTSbLUBzuXnB5hlA3aGd1jPoyLfQV+nhH2150rlIID+jqb6Twue0Zkl4u1XsE1GBFK3w5ilDfUcoBApY7iOiNyfuXabnEYeRhkOJyk11r/kyI6AKEEBXb4jPz0dYRYHXDcj8RfQOE78wAvhzAA0c9mEOJGvPFpKRx903M6Gz91L/H5O+9cOXdp9XWlsOwYzgv7VSMMSTwR1Jop8lGDmxrfGG0MZLooLRWKx7OlJFPCrucA/kQq/J7UIZCfl6jsz7mnCNUdfJeHm5SpZOVI4Qg3iyAGNkV0wLVZoXZle2V5+YoxTzz6Ik71ynAHOJZ+67tuyw9bd1wxaAs9vey7aJi1XV5Ql2jcRVCLUbVYMI04Z1GhQFy/2TjvKeoAch1nNW9iDK0IZIrmu06Oghpe3iBSC3K7nukzjuQ6wgABoFa7U53/I5iL/U8We/7xbHKcbOeMmbfV7jZuNDebIevYSHvwbOqi/hLRPix9yy1cs/n0xLVta14Tmk3gHwq+jDW5RirzZYv9kn+RW9/chKVWcR2VHLA5P39zHz3bl+uUvO37Ge7DGu3Y/xjAM+FlHC8CcAnQqKbz9rtNzcrq87y3wfwTRDKMQC8Dl0V/pmQNEwUGClDHRZCY71BefPagfb9p//wC3qslK37t2LobjRZ5wijjREmd4zh8gzbD2xGLNxWtiSNRkxRpkYitp+xSEu9dGEklbrEspNlAOo6Jl6dcz06qC0VAAB1G+CKHL4cRbgjHss7NNc2uwcxricO+NqjWC9PPIEPpG1XVOmocfVlgXpzFsfu0NFlAwSu8Tp/vhRyBZUuQiimUKrrkquUfeTI1SgHreewffSMNMQwO3RKutmuhBiSy/LXxcYU2BBjxRpNVde3BKbR9kIAMLu8iXw66nICMbLOe7maSBduu/cuiarSvJw5IKKku/tqWT7B2FHRoLRdE8pQNzp/RdfA1R++mNDnGeZ1E6PzdlYjIKBQSneacJfuGTnKS+fQzuYyDxqdL0bctqCfXKMci8bTHCibE2OKLtbt3LQw0B7REpL71fztIget+XsugI+HMHA/k4g+HAdbVXJlWZVu/AEd1NkVR3HNla6SOFOopIyfu0kGZDlwQGaI9WWypobGZsnXHOobFYx3b4pvfGkaPWF7SM2jBDSC2K6EVDAtJVQvA5rN7bgKZlq8lk/L6Hm79UnElF2RgdoQFZdFHoAYEPP4gmLl+aTU8+miFlOE9eZMMHgna7f46V6LyB2PWLFcxNwLS7jnGEXGXx0NYqjrqFCycoS27ucEXAJ/uFyUlkU5TSXnaXOYTyR6MwMspAEH9sKg8nmGEALa2RyjC+twC46BKOdanQiPYn0Sx+nyHNy2yKejGBFZRJaN8xiZyhx0CecWYhRbNJE+DAANqh6ddrFDMTlZcjjUdTyXrBxFBZ3Wudj4GyBGf9GY5cKqvFl537d/dRxbPpXr4woPB4E4681ZnA8zAEZx9uUoMj3dgoE0GMw+bxcgLuccvOaWgL4xyY/ovmYA9emuTXzQmr8ZM8+ICEQ0Yua3EdHjjmNgqzah/HUsCbGY+chDqJuVxcRWbF/hrSlljXw61pyDB/IC2698CcZPe95K+7c6hAjTVC3KC2N5OKaizFM825LhQJcPSSEJX45iCw2DJdgFIHkYfFkgK/udAlJP05Soy3fuv4MJXMyhGFvOjIwpVFNeQfc32hij3pxHw3iSEg2vnpPRxu2cAURP1eeZKkOrsO/fzkEjDztn9qGnVIqNaczdBNS9BLcowLyX37H5ApJr1gaMzsv6SewDaOaiIgyJN23jHl1YR6hqBDTw3qGtmuipc6JgXZ6jnc1jxGD/W74iS5woc0SiMdIVIS0qSxVwWuPRh04z5Av3kBmim5X3f+dz4zHT6+PLIt5r6b3bjSVX58H1ILt2Vsk18E6iedcZ1MyLE1VvzToCz8LY43NwRBHLSdWxaAuW7wZwJ4BfIKI3MfPn3ETN371EdB7AKyH1g5exSleTm5BVZzgtUy8BPB1yImdKvHrnTqOGFB9OvS6ez0DT1Rb9evOXfq7WKFBcvx7KDIuKO4lEAETPq8g745JPxhGyShX84k2eF2U0Glk5QqaK0CKU3jHMq5wm61U4GY8vui6uzncrajrvwIk3GtDAKcZvHiIgOYN6c4Z3Pu//xKNf8hMrzdVRiBVChrpBsTFBPhnH62gRA6CQVJnkWGJkk8Vzy1S55EWJWguMo2I1iC3P4NeLHkziywK+alBvyW/yibCOIjHAFzGBbGJQqzkC5JOCxBxoNAfm8ww+z1Bd3+z93pcjyYnliIYphToXIU8A3XmooU1zCDGXsETJmjJPK/JjdFL6SM+3uTz/rH9309ezurYljWCNsac5PUoiZVfkEaoSRyDvVf1bYaM9a36qxmRa9nKLKRkhRma2nyUOyqGFOdY3Hacw88+iS0MsfrdyzR8zf5G+fJEGC+cA/PKRDHJBVoXCFhffeD0RvfYYxnNoIdclJM3QCOzQghRDRgn4pgKyvfHjP/77T430R2OrTO7aQHVtKyZQLToBuofGxDxf+a6D44CdLBZfCFYvbKQyJk4tx8BKGzaPMhsXaKumR7PkNiDfmHS/t+4DS2C/NBFq8Fkzq6KycgDy6doqU47rP/Kijj2lsv6MF63021Te/bXPRFYWKNanqLe2e3PFbQDlWcwPNJuzyBAy5pJRq2POxaAVhRtbi9R8V8/RzKoIxQCaS/GSqM8npbDL2hDnm7yTiGEqOaj0mnhf9CIBU3rmZNhvyTmUF88JU0rzPy7P0KrxiMbBHIDEAOWTElwK4y2E0ItckKOjCavYeFjPoev8kCb4FfoLAfAe2XgU5za9pjcj5J3moMYx92VGMy3ILNYn4DZgfuW6jK8cRdjWrkmj+/NlEa85JdcS6J//IonDHJCjisQZB2KFnQkhok8F8Bhm/iFt6fIwAO866uOsCoWlFfgOwMdhOYPh9IQ6psxuRVCpAhb8fP86Ip97FBtj1JszuDxDPpH6DkvYCm4+7iVB04fRFIcptFS5tUrxNQUYPUlVNClsRd5FxhIAjO+8oJ5gQHVtC6Pza5hfudE9rPrwmsIvFqqqfUIWMOVUaO4GkCT3orFI5eoPfENkpZkSYtft//qPvAjrz3jRgdh3Bn/UW9uasB91ijZPFEXZwWVtW/UMZ1YWaFM4MPFmlxUHxnPfmPQUUa5zI4tDCZMwGio1buZUWJ2QLwu0vuqdUwpR+jyLvzWF6xJj5csiRlmAGCaDvlyexUS/nVemy2/XamQzzcMB3b3eddFulN6bePTewec56iVG5LAGBZB7rPpf98HnGUbn1zv4Ns+Q67ybE2COgCsudE5DCCjWJ921HUtOptmcxXxqmkNLRTqF90kWi4bmKORWaulCRN8EWTXycQB+CAKe/hiATznqY61696QV+A3Ewv2jox7MYSWtxwCAfGrJWEnSUpGLojYFs0t7l/d8y3Owfd9lFOtlpK+O77ogkU8IyKdj5FOBaPJp2fOa2YXO620DcpS9BzytqzClBHSUWIsYem3OzYNVZRFqCettyWXZV47JXRek+M05uMTzrDe3I+wFAMX6NB6T24DgWjiFU6yo1BLZtn8AuPZD39iDjOrN7dgQgkOAV+VmcpC1b+79pn8CV0hewWR8aQPzyzdiriXfmKC+tgVuA8pLGyDvMFcWUy8SDB0dOZ0/9qHnXNj5NVuzHZCkK7IYcZihNiMaSSK+20c8vusMt8u7a9uDMeOS2QWC3icOWTRyFuUapNkm0CWAaESikUjHZueRnLd5+TG3ltS6WA4qnT/7/DAKePPl3wJf5CDv0ejaSFlZRLKIteVPFq7ctUmkGfiOUqyRX9sfYxq1ks2xRmku72DSozIsErHcOoYFwBcB+FgAfwgAzPweIlo/jgOtalg+gpl7XTCJ6HD9549auOsCm3q2sZgqgcTC5jZC3WD0wQ/D9qv+M6r73g9yLobhgMBDo/PrMdpI2UaZUkgBRIUdw/IUy/YOflruiKAsB+OLvGcIWqWqRqWX59E4tOrNk3fI8jJ6s+SdRk7y8FkC1t6zCzGiMskWKpUze+Bah2J9KlDU5ja27ruC8YZ4jPe95HkxSS3zsx3PgVuBUNq6Bjb7xghZvjJJwjkHViZTsTFFs10hm5Yxge7zDEENuDkN2UNGaOs6Xo9oBBZvj6R+p6fYbJjjUcThm+05fJHDawK81esDoG+EqxqhbuL8tspQs7HZ3Fh+q9meA7k4PPMr1+XalWLkm9k8Rice3eJniwbL7sPxpW5V2EVD6byX65MYRV/k4DxHW9dxXDF6zfM4HymU5IrV2VNbP/XvAecx+ZIXYPvn/19QUSKbBo1W1hTinPQS+UascRp52ZhGatgMmrVzdOiivQiDKoQcr70aY5urrCzgS3FMjiy3onKCLV2OSipmZiKy5U8OXim+oqxqWH4bwBMWPvudJZ+dnhBFGmU+HUsCUg2JKRV7LTdkLvUsmVBp0nW+r//V++SGTB+ytt+DKIWJUthgN48o7V3mvI/MmHw6joYvWOI8aQ4Ysf+klXk27my6KS6LJBbhHlMUlhC3B9P7IhqnRh/MbDySeaobNLMKxfoEzazCe77lOVqXsZnAOnmM0ixvENlXvcrv1aiqo/NrAuldkLxO7Ayc1N/kCWMutC280o/zYozGzXvG0hUZfJ73trM5aas6GgfJPXQJd5uHRejEvP0s8e5tvzEyMkJG4l3beELdwBc5MlWgkfqbRCPZeAQkjoTzHsG1PcNlc0DeiaECekn3bCxRrJ13M5vHe4e8Q+b13HyX1F4mrpwATb1609YsB+UFZr/8Mol4xlNUD3xA80DWtLXqRUcuz5Av1tm4zlFKn6V8Oka9uR27LaQ5wjSSSyMxM9A+WcAsn47RJFHxYeVWgsIAvIKIvhfAeSL6J5CVe7/vOA60X3fjh0CSO2Mi+lh0lZ4bACa7/vAUhJIcSzYt0WzOoqJOla3dsI0WXxmM0czm2PzfH0CoxFv2C8YiNQyp8kqxbMufAB08krKJ7MaO+1qAHXJV6gB6ysne2yNuME2POpx30I0dWyjHPh7b5VlvoTOaJMWjbUCm7CfvC4wvncP2A1dlnzqu0XQ9EgTSfS7OT5y3pgL0/dZPfzsmX/KCpddu8+Xf0kuoknNACj9pTY3AhSOEtu0pS6CDPa0A0q5B2NyO25kCsvxJmtcyiDSNOnvwldFaJxItmoNiRigdewplGnsvbY1i96gfjRCauqfc7dpFuKrtxpRe23ifWJ1MkQFV93tf5LHQcREuMnhUoKqFThV6TyIcrBP45OnPx/bP62rlWQEqSrRVDZ9nyKalOClbsxgNt4kTkOa/bF5tLixiB9Bn3KnzZfuorm1FRt9iLsUV4lzNq+u9e+WwwrfYCpLM/J1E9LcBXIPkWb6RmV9zHMfaL2L5HABfCano/K7k8+sA/p/jGNBNC1GsOLcHN12TJYWZ7EYsNiZRSV3/q/fHKCW9sck7ZOuSNLR2MNZ/LG0zYscFhHGTT8eYX74elQvQedyL3lj6f/qwpA+dKRI/GoFDGw2cJOE9UHj46Zp4mVnSzjUEUJbDA6AFaMPlU4TNayjOrSM0NSgrpLo8BIwunkO+MUE7q+DLAlvvfQD15jZKhWDM4Filtkm6+mA7q+Jx91oHh4PUgrTVZZBzsWJdqq+r6HGmPbt682ckhEmJrG7QauFjqJuoeNPi2Z5DkOXa8bqKRljG1CJbWwNCC9bGl3b9YyFfCHBZjnY+j4bADHd63VyWS78w3ZddLwDaIFOWza5vbMYxxnvBBbSzKhrz9Hvz7O14aQRt52bH49CClAXZbm/1or/ePWgKvqlARXmgBfHIOVCWg6YbqN75Vsyv3MDaw+6MUVocpxmAhIzgvAe8RF32TMboK4GduQ0oLEKxLgF1Fu8RabDZGZ7q2iYwBbJJGXNTRwaJHWHl/UmIQl//k5lfo4WRjyOinJnr/X57UNmvu/EPA/hhIno6M//MUR8cAIjo+QC+A8CdzHy/fnagNQbkN5035rIcAXVMJLezKkYobd3A6ToTlSaCreBsR5K1kJqD8tK5Ht5rEsNqvSxpDyv7PbchQjKmLAzLD1XTS3YDnYLoeiF5UXyJJ0dFGRtMUlZorYIt0CXbI7TgthWF7rwYmLwAz2ey/Wgc94Wmhl8/D4QWMMWX5aDQyoOb5RjfdaGrM8jyGO0A6Mao1eU824LLcriilKglaQoKANVv/iQAoPi0LxM83jnk6xMUeo0Ekx9FJW4KJZ2b1Djbaw4B+foEfOUG2oVoL31NzoNd2xmSROnG987HaIvUUFMIcj7x/pBr58cTcFNFI5W+JhuH7QsL4lw0+PnaFBySbgNZLgZf5zw00sokm4yjMaORGI1QN/DjCUI166BI58FNDXK+O7/Q9hwWu37R8ISEKdkcTN9QUcb6sK33PoB8WqI4v4ZQN6g3t8EhxNzP6MI66mtb/TyS73JjNr5FssVi7gmwjgpaOKqfZdNSoDZllZHzS6Hrwwjj1jIskFZcn6aNKH8VwBsB/D0A/+CoD7QfFPblzPxjAD6EiP7F4vfM/F1LfrayENEjAPxtyNou9tnNrTGglebOlEAKA7mmKwYMrsfECgvRRrExiQaHW6lKb2dVpPPaDc1tiLBLCptlubSRcbli/E4e5FzzIKnHnHr6aTFXHL9FGGny34xEVsQ2IQghLgVgv+GmBtrtCGe46booHPP+184j3LgiBiYX6IKrWcTKEVqBNEILOA8/nkSljCyH82V3bKAzYNrHyQwNFFbhppIEb5Yju+vhAIDZq38gjjnMtmK+CUC8LtxKIWSEp5Lmg73GiXV3zX1ZdFGFDz3PXPbRRmNoijRGVM6D60qUquYYbOneXtSlRtjaA5HzoJHMiSloyrtWJOS90CrzHFzX0QjH650Xcg3nMxAQDZjLcrBrNapqoyPhk6jU+RLkJE/mNDo0B8IMXDRsoxI+tPp9uvaJOgZmS5bQ9Rdl/hs/jtET/wHmv/pDUhPmPPz6eVTvfCvqzRnKSxtotMjUKu0tmnDlBIVzqK93q5KnkVoqKUFGxtpd/5QtZ1ChzRuALnoPAdlkDA7t0rqumxG+xSIWAMTMW0T0jwB8NzN/OxH90XEcaD/TbayBZZVyRzGjLwHwAgBp986bW2OASNrh68NCkAp7bipU1zflZvY+cvaLDUlMA+h5vVahPbqwJtx/jS7ycgLnParrm6pgXVfcZjmPPMPkIZcwe+AqAKE5m9dl7TbMA08fFD9dR7t5vf+QZIXi3C4qP1ESnaLydz4MXNfgrWtdsni2CX/poeC2RYAkYVk9TwKAyUb3+3OX5Dd1BcoLBOdBWR63d+vne8oX+lB2HnxHjABU6Y9K0KiM20fDowYJANqrD3T7m8/i/rNJ2XWyDa6Hl8s+OmeBQxuVB7tWrol65cUHPRTt1QciFLr4255CHZX990UJynKE61e684AqfkAihLrqDEM01rkYYtfG+eGm7jGrKDmGvecQOuNWlIlBL7uos4EYieT+jv/bb8dTGddoLGsOOR+35yohdFp0mb636+s8aJS836eAGBDjAgBoKrhzl0Djdbi187j4cR+D62/7sx7BoVifivKfrolDEwLa6moks2QbG2iuXetyfmtr8Ry9RmYRPitKoJr12ZZqRIOdrzlJybxzU4O3j2bRRAajao6WaXbMQkT0SZAIxcpFjq7dcyL7QWG2YtmvMvPr0++I6FBFNUT0BQD+FzO/eaHP18prDBDRswA8CwAeceeF7oEtyu6BVe8mYq9lgfkVqY2YX7kRq4BH59dirqRYn8AXOaprm2hmFULVoNmeI5+OMdEQW9aXqJCvT+DGU2zd+x5ho1l1Pzrqa725LcnSskCW95ks5B3qy5fFm3JGNkgebhVWr1h+48F1jbB5XRRhUYJnW/HcqZxIe3zv4TYuor18n3rQ4u1yLetUkPegooTbuAiuZqBak+1ZATQJy6woJbrRKCqNrkxxcgigpko8xBbcqCKezyRKCGoU5ppQVzhOlGS9g1pqr3tGIct3QDQWLURF6Lo26j1DreOSL7oIkCKEVXfnYvsz6Cu5HqIUE+jIjIFznbflPChDz2j1RK+xXGvXHSMr4MqJzLfClJwcMzpOEerqoDqu9ZrlRfwMzsNN1nsGDKEFpxHborFacv8tE4vkOAS46TrcdAPsMtCo1M4NwmYb3XERozzvInCdN3YCiTWbM2Rra/CXHirDmylEXc2EnaZjjIY5095oi+PVc3alwF/REDsPNFU3TytEYyvJrRexPBfACwH8rPYZezSAXz+OA61qrb4bO6nFyz7ryV5rDECS/09e9rMlny29erpQzssA4OMe92h26xfk4XJevJmiBDmPYuMGtt9/Gc2sQlYWWH/kB6GZzTE6v6b05BKj8+s9ZhNlhVA+iwyjOy6iunwF2WQM0kXCqK6QXey8w/GdF8GhxeyBqzGRiFxosaML69Ew+dEI7Xze98S96yVZAfQxb1OmCSwDAOHqAxIhqJctSfIC4fqVCEuEG1cEkinPy75DiJ5buH5FWsJP1wUbDwFhe1OouMUlhOuXe14rFWVUwG48jQbC5gyTDfGOmyoq1rC9mSjQDnayzsEwaM45UaIxx7NwsRM4Z5ni47aFK3M1aHUvxxOhD+ciNGeKiTLpS0V6fnH8iZGSyV5U6k7H4QBXdlGhwYBZDq4WfmNj1aiPqy5SiIbG5lPrW6jQcTon94Htz16r185NDRqNo1Kl8TR65jSeAtrlGE2lxsjG0vaMECCRFiVQ26IYhOnWzwNtC7d+XiC40SQ+qNvveFtHLXZOomh9NrmadeMNAUUpXaANavTTdQDXu2dkId8VnwHrUu6c/DZLlp7QexCw+9bFeXfrF3Y9t4PIrZZj0QW9Xpe8fyeAf34cx9ovx/JJAD4ZwJ0LOZYNILJfd5Xd1hggor8J4FEALFp5OIA/JKJPwMHXGBDRh53GU2kymeVKd/XwoxHKS+eUBz9GtrEBPPCBiOkLHVGbPWoYTeMpJg9XBZgXkgQ0CKicxgc/htXOg5xHeemcJp3lZjaWVn4eceVKDg8goInwG4B+vsCUjGLe5MVIAKpQqpnkRbI8KkCuZh10Nd8GQttBAoAm45MH1HmwnfPmdVWwiTfvHGi60UFP1Uy8Zp1j2YeL+QIbdw92gXraGmGlXiWNpNM0skLmUBWnjdGishTr7xEUVLipJN803YgecXjgf0dllCboEVrJgxiVNrRIF0S36CwqIVPm+vsYiSZwILKiU/LmWWvU18vjpLkaex9zaGaAunmk5HeLUJ3MX6mJeN1nup3eE71Gq2Em5A0Ucd8SxYQYQcSxJzJ79Q+gfPJCkw01RjxP8jqjEuyUrTWfRZLF6K47UT3wALKHPDI6Qew9WnvORiXc2nnwbDPOERUlsqJE2Nb1gkZjNcSzbox6fTiJXqNRt2tv95r3gB/Lc5EVcAnEeRi5BXMsJyb7RSwFJL+SAUhL/68B+OKbPSgz/zGAu+w9Eb0bwN3MfD8RvQoHW2OgE0tqn7sknuqog8Z8U6GoZvHmLC6c34GT02RDWCnmsWa5wCzOwW1ckpsf6jWNSvgLd4EvdQFZmG3CrZ2Xh6OcalRwGfU73yr4czkFzzZj2xZAYZ9yIh5dNUO4fhnNtWvIL90Bf+fD0LznXR1cA/UmC4Fp3HRdjOioBDTsd0WJAIWHDOKpZkkuQK+BwT1Q5VxOlE3XzaUrSoRqJoya6bpAEFo4F5J9Sm4giPFxLsIsVJSahxE8PCrKpgKaCuOnPS+2fYlYeGIIaOR777uJbrskuCady6c8C9Xv/AzC5ff34C4alcC8M3Zc111eJc0ZpZCVkhYsmon3qXrEURL4TGArhaLUAJtTIEy9LpKJuSc9h+hZm8F0DhLuGjFCnYe6km0WIgzJ1bSd0fFC8BCDLTAh5d21k+jXCcstRiaSj5CoYsW8QVOJ8zEqQZNzAHcRhi8L5B/yERIhbFzq5l3Pz5+7BKhxcuMp2tmmsBMVDuO6gi+nEt1UMyC4GJVbpObOXQJCi/byfTFSSfNoNBp3BIksj47QKjDfqjIYluWyX47ltQBeS0T/jZn/8iQGdBNrDEShcgI33RCFoVAYAFEAQC9haYlS3t6Em65Hr42KUsJ5VYJuuh4fCq8RQYwOIHCAhd7Z+TvBxVggHeeBpoZbvwCabsBfeogYjmuCG5Pvcg+AKAN37pIYwXOXQNMNGdt4Kg9MOQWNSsmXZEUPqjDFblFH9CD1IeJKjuMiFyMRe8h6np8qrJEYBKtpsDxFjGwMoktpuM538I334Pl24vmLUrXC1dkvv6xjky1CRoveaBIVGIQUk+bOo/mDXxAjWyTkgQXs3aKgqKCT/ZrCT48PY95tb8r9Y9+l8KDmUaJXHCGbhHxhXnRy/9k8kjeCRuhFZ71IZSGaoAQCQpD70fJnERpKHCOuZhophUi1judo+aEEYjXPHwBGn/WMHbcMZXmMWp1G70wO1FaAy4DQIvugRwLqBMUINyWGTNbh1s8jbG/K83TuUoyc4bw4cVmB7M6Hob36gEC8m9cE5pyOezCXP3dJYNmqY7pJNJ/Hc4mIhJv1r8UhJDBjfgsl74nooi7aeOyyao5li4i+A0IBjnHkUS30xcwfsvB+5TUGopCwZqhcQ3vfvZ13pg9XaGrAj6MXQ6My5i3cdEPot7NN8Oa1iLW76TpcOUU7f0CUjhotC+lBDiAH4gAeTcWYhADObWncdVBbYfTYxwPZCOHy+8Br57sHBxCMejyVcZYT6Zel+Q+DusgS75ofIefie0CgLDftAkoObcTUaVQKW0aTtxbxRMhKlW+EANXoRthBDYLRXOM2pSVgTfkFUeBtGyEa2z+NSvGq61ohtWQsmpeJpICmTiCgrs9bNCqWYE+ousWjPiqSFwCAJhuixGab4CA03Q6SShhYSSS4WyKbsgIYQ2jAeRFJD1HyPJ57rH0x5W5waLxFfT+XpuSS+H0yx53hCDFvYYadzFFySlJIc0/xPFyEY02hm+NBWRHri5AVYqAt7+R9ZAmmc5xKhEjjcTMxKuQADghb16OhDkmkFsdf6MJoCsXZPUnzGbwiAG1TiUHS59NyQTQay1zbX1GK86X1U2F7U+435yOUZs4Gty3ckvzcYeQWi1h+l4jeBOls/EvMx9c2YFXD8uMAfgrA5wN4NoBnArjvuAZ1syLUzDnc+nnQaAK0dVctvXExQjh207u18/FzGpXgqw/AX7hLvadr8fPsIY/s8h5ZDs7H8mC3WtnvC4BIvGxfCDMmNODcRaiB1Mi56QaonCBslaALd8XakezSQ8SYOA9utjrPPFF+buMi/KWHIGxeAzmPNknSA4gPFGvCHJbQ9zm4notX19QIW9dkwtSTpnIKyvPIMnMbF9F+4H8LCQDm1bsOKkmjpQxxG25qhXeS65F3njvlucAnRoFOabYLEiGjrBBjq8ZFajy66HP8tOeh/v1XqcEqgdyS9rUopqaWaxlarSMpOoJHjBxCl/dIFKpAZuL99woMFwxEhKCSXEX8fCEaIVtNEnlvP3Y9onEzEgf6if/0ulkOKebHkqg1Lr1t81ZYRNgZHp5tieI1QwPE+pbofC1I8+ZXi8Gy+zrv7neDwopHfxTqd/8p3PpdOr9tdGyicfF6X+c5nBNDk33wo6IjkT/ysYDzQjDRiKSdzyQfU826+cylfsaVOTCeClV86zo4g8BtStPmppZ7xmq0jkBuwRzLYwF8NqRH2HcT0U8B+G/M/GdHfaBVDcslZv4BInpuAo+dqYW+SBO+weAjlwG+ADUzsHn4edEVTlnNhbGCnEP+iMcAHMBZCbfmwPMtzWF0RgU+B0gMhhkWAH0YQwYE9gVcdQNs25+7C5RfAZVrcOsXgWYuynw8BbKRwF+b10SxOS8w26bWqJinPN6AH03EM9SEJ9ST4/m2bKc1DQYLgkXhUD4CK2vMoAJA6amjCciYQ23dOycOLVC1vZqAHoNr8VoovNVRkVVJbmt+qdAIJviEQaZEBItIjAVnUJWxu5wHUEc4s3r9K+RYk/XO4DU1qOgr93D1gcjmivtMoaVEmacRiUUH8Z5JoBajCi+DuQjoIhLnehFcNB7eS4SXSty2yyHB9rU4x+jokukYxDnp+rSlMJcdAzONXHWMneEMvf2k0t7za/1zTCm/8Xw7UgJXM7kuwSJRF1ldgDhkNMmAraua69SxFEpDDw2sXACjMfz6eXEAmwrc1Go4bD7HCOU6XDaKkU6MdrNRhOWQjSLJ4CjkFusVxgBeA1mW+DMha7H8UyJ6M4CvY+a9awUPIKvOsIHd7yWiz4OwtB5+VIM4EnEeWLsEV22Ci6lECi4DOIiCVZGaj01RvhZ5cAD7jv0D74Gq7sNe6c1IJL/Jx51nqZCYQQJMDiCSm5gcEBrwaCrGj/SBJwd3bgTOZUlgXr8DWLsI7/4adOEhEgVdvx/t5fuEtgvIviAPrT1YvL2pnliInjmaWrZ13W8AgMo1qdr2uYxt3hWLufXzCJvXEDavd3Olyft4nEQidNPUXRI762P9gCpZ50AaWbWX7xP4r9UowqAaWL5oeYGeKIp+FAdAIR8fz98IB0DCU1+SeN/hufYMhNtpNBdgrfh/uq/eMVxf+cZtOvZWXGxuyVgoR9fdIEY1rjPw6TEXcjQSqSbGRj31COVZPi5LDNTCPvK/9TTskCZxOkx5E8k9lmUSvVSbUrw72+rycaEFubI3H5yPxOkpSnE6NJIkQBy8ppYIRR1E0hyn03uRyjVw1XWXIA5oJxdAI2Vfat5HesRLjRGFJiEsHE4YfEtFLER0CcCXA/gKAO8D8NUAXgXg8QB+GsLUPRJZ1bB8CxGdA/AvIfUrGwC+5qgGcRTCROBiLDeNKe5WbtSQr0suxG7CfIyQFRJR1FsILotGiF0G4iC4ei6ekxkfOU5XCR9Ga6C2AtVzUNtEKAyQG1g8rgzsc6me5sQbbJv4YAFAKNdBzUzG/kGP0ge9AZ27C379DmTNHGE0BTW1EARcBs5HoHoO1JVWXZfRcGI0BtdzYDSNEZZh4Lx2B6it4bavirEL2j7FF8LQufqAsq2KSDKIUBZE2XE167xyoE9XThVUCFGB9RhU5onafBQ+YvxRTGECgB/LOWZY2I/v1TPEyDJt25EY3KXGJVX8CYPLCAQxN2HQ1WJFukYlQBLtpBFK9J47gkIsBjXmWephp/Pqu4hJDMKisfXduVkkEI1H0R1jthlJHjFKTPMkVd/oLYtE/Uc/Ce3bfrMrGgXEQcnL7lm4fj8YENLKuUto3vfXvYgopUa72XXJg00vgKqZUte1+zYHUD5CKPXZDQ04K8VZK9fFeNTbcn/r9gihIxDYpS/WdO4q2SYd+yHlFoTCfgfAjwJ4GjPfm3z+RiL6nqM80Kpr3v+8vrwK4DMBgIi+5igHciRCDvCFKHRY7qMRxapGgpoKnJfxZgUgSloT8YAaj6yUfXEAhQbBF+L9WP7EDJDLAMzBXjn8elNbDoBdJnegRi0mnBUyLg5dREGuNxYupqC21nyN4t3UIBRrsN5QnI8kIrMaAnKAfkf5SL01HY2Ol5o5EBqEch2cj+E2lShCBC6m8OcgEVJUil5yVuTA1TYIXfHZcs+/S/JD4SJLvnIjdS+UFWJwxkXHjFODkDLM4v5DiPmRCGXlBZzWE0UFabUwCXONMs2bLCqVFD7KcyD0E/dmrNLeYsuaSAJ9+GyRABCPkVLGF/JGcZsFskI0AInnnzKfug7BnXG0vEPMe5HrjIzW6iwm5ikrIqFjr+JIKsZJvZPv7l3nAc4iAYNriRTcZD3paNDCeXF+wua1mPNzZQVYBwkzCr7oIFCfIYzPAWo02BeCGJADj6byXFVdsW6sp/HJeSiawFkhHSKOQBhA1RwdEeAE5HG7JeyZ+duO8kCHARv/BYD/eETjOBJhu8nzMWxFyei9kEMYrcO57V7kwApbcTYSQ8FBFK/mSEwZQz0m9rkm7usIf8F7eRBMnAezer0cYp4j7jNCZWZQSLaxzxce2HgncOgKIn0BCo08JC5DGE3h5pugelthjiDH8j6Ok8n1DCJnI3BexoeTQgP2BUK5Dp9dluO10lIELPtL61TS80WifDtWlyrdZC2pjgLuOtp120qSfbYZE+XR2LikG0GbGCBd8yPi8rafSPFViE5hMX/uEpr3/VWSU0sUd+wIvZw6uuO84nkvGJUlXv6SnXW5HaP77iaL0U5qVAyKQpfg7+WlAL33694xl+WS2HJr6HJH2cd93o7hhL/4vX7U7Vx85qjalPtwehFu+6pU4ddzKeQdbwDzTYTrl8WoWMShBifkI7j5ZnxGKDQIxRicT0D1ltyrdt9q1A1olG/PVT6SiCYOjgAvKASZ0dHn+MiEbw0ojIh+DjAfc2djE2b+gqM+5mEMy7LWK6coFBWyRQ8IoQdhufl1cD6ORgGqWKEGheF6NzH7TDxop0agbUBtLZ69zzsDlJWdoTDlQuiURmqkACEAcNBIhrptslH3Oo2eigJUbffCfFLYDj6TcXKQh0aTk1RtCvYdgmDfgDyoGm3F/E9oEPIxXLWJkE+i4gnjc6BqC2Tn1QqUwMUUGBHIXYuFpJSPwNvX+zCN0mmj0oqRRytU8JRqOpLuxinElNZqRAinmsWiQukDpdFE8BEKipCQ86LUQqs5oxrZnQ8DBy2oUwZRpCnHRPsiPBYitTcaywUDtIOCbPtDP6qIkh5jMe9CDuRCx9wK3aqIHe04MRy2PbnOWGajeH9SlsfrRqERJ4OafnPGHBoB5jGX197za/Af/aS4TfuuP0wICZ0hpbYCB4VkqQZnuUQXCbQMcqB8BJ+PEMbnwNkIwVCDSua/VaPg5jcQinF01ML0kjx3oZHng1meA43qGB7UzMWoJA5Z1AEuE8cJaoDnmz2o7DByC7V0+c6TPuBhZvhszajzolDbujMmWQH2ylDJRkC9LTehwlkhG4FqbetdbsBtXQaXBbiYgOY3ZL9ZIdFMtSlGRaMQdhmQuy5qWYiCgC7o6CqSk9CcHJQaFV/HiAsQBWaQmC/kAanzeG4wkkBWRkgvTC5EuC7Fpm0MXIzBHEQJWERntTCFKGpqKmHS5RMgL+UBJQeEXM61qQVOGG/IfCTV1lHhhaQocSznFG5c6VrBjzakZ1Xd5Up8OdWIougV0sXpMgYToBX5yVwl1Nz0foi/1RwJN7WslaJGRRbcUpZRXUuR6mRD6NgxJ7MzQtlVYqTWRRo7hJcYmcQLT+czGqc0wpEXyc2F/u+d73J72SheI2orcDHWa79wjKwAtOaKYuudJZ2Nack4dF/EXb3NjoicAzifiPHQ5xRtI/dXIRArzW+AXYa2XBdKujl2um/bpzh2EpEarMX5uHt2nIsGN065yyR/CdED0egcUpiB5gQMi9YQ/h0AFYC/APAPmfmKfrfv2lXK5D1R2a9X2HUsNyAESW+fLfEZ0NYSkWhCPhRTUFuhogzF9JKE6Qr5ABBD43O54cp1wWWZuxs1NRg+B/JSlDoAdgprJco7FYaTpH7qRWnkY0JA/6EA+kpD98suA4qJwlMVgHGE2NgK1BSuEjhvBs5KBMstcZDjOicKxh7erIifA0nuJyvAbl2UlBojMTgKOVjEBXTGy6A2y2slEZxbvwie3ZAanCwH8hGA6+C5EAEwle7Uki/ywOa1DsuH5ECoWshZLIGeaDSWBDBbNCTdcYVFNAHfSCni0iiTshycdY0XXTntHztdAMtgNo244uJmadSyYOho0Qgm0QnaGj3mXs9IAORz3cZ326eKdnECDE5SQ8JuFCnNEuFKzrBzBJQ+7yUXBkDJEC3CO98I9+i7u/snNDHa5qYGeUT4yZ6baHwUeo2vdb7je1PuRijRe5tdBspdH361nIi3KMiLo+gysFNUIo1Qso4UE6+b5V59R7A5rJxgxPIaAC9k5oaIvg3SofhrV127iohewcxfSkR/jCX6nJk/5qgHvF9Ll/WjPuCxiSaewQE0q8R46FchG6GuA1pXYjyVBzR4WSGR+VxM0LejNTSBkSMAynKBeUmAeHExYY8kD5J1n9sDqGJwWoxAgJ7ny3D97xeNCpKH1OdggzWaOdz2FbAfCRyGqTyAbS3b+SxuB0oe1GwkilZzNEYYiMYLAFzTGSuSKAZEcm5K/QQE43b1Fth3CWjOR2BKPGNm9Vglh+VCi3b9g4SRpn2yaLwOnm+KUZmcB2480PfYLdmckgKAPjSV1qTYNUEthaFtUsSoHa97bWnS3ynEkjYq5LruFYValCetgEYxcUzOacGrUpuzEbiSlRNj4l47KUcxWFTvp56xsEg4Qqehn38Duog53iypEQvC1vJNl2uIUW7b3cehkXwUuUg5NwgNAJp73wqMpsp+TAxzWwtUWisslY+6ZwDoiiYtye4LoJ5pJKwsxdB2EJ1F4Xa+6qiE0Vo0SuyLSMRx1absS+eK00iFWeeKY56SqqYfxR9WTogVxsyvTt6+AV2fxlXXrnqu/v/5xzrQRI5lkZfTEIZDTRnyrARnFXi0BvY5agZIPWvvCEwFZi0DTUDmHDLX3WjcuZgx9xEf3KQeJX14oleG5LfpA0sO8Ave6AJMAJ/AXgvtMeI2dp6+kNxJNpJzzEZiPFphz8B31GkziP19cvdwJePhxFimry1iYZcpUy4HOI8PboC2aSeNznSMVM9BHAQvVziPSIrfqK3E0IxKULkmNGpfALPrWtDqQeMCLr8Yi1Tj9OoaJQA6Y5I0E+XZZtdy3qr3kxoOa/9BDWIhJ89nyRLOCZxlOaF0LRGIcYjbmRK1XmNF2Xn1PgflfZiUjPZtkE5y36QkC5tTcRTc3onnRXgtfi75CAqNRC2a5KY0F6eklF7UWc2kqanu11Xbeh2pi1ac6wyeOV9JHmjH+HwhubzF7zRfKfedUqcp75YPsNwKEGGwhoGWgTKJ8iwKA6BEGHWEkmQ1K9PsqOSUFvr6vyBdUIAV165i5vfq/yfS7xG4rQwLo2oZyMYoigbQByobj8AMFF4NC3r3mvxWH2jHLVyq0NMEuhmY5GaOODIRGD4yu8AhoSKjC/et9cWSqKRnaFJYBIjJdgAIIIABBgGFUm19iMn86FFq/YwRBKKxUK/Xtu1VISfjinkjhcUs6ol4t+siuTSnZQ85ZznYjZNzkYiSNYKB88CFhwEa/cB7YHoeqOfRuw+jqRgiPCDbGMRkDDSjA5vBsCR3Go2h7SIfW6ALiNBMrBKvq5jf6RTmKHrBrMw9LsYRGjRCA+WjvkNhc6mwIwBJGtv9ZuSQ5HqxRY3ptdcoVc696cNoS+6TRaHQgOZaL5Xe9MEiYK+kjhHQkDooSS+5rEB4xxuE8KLwFc0TlCVG4JJE54Xol5qZ/q4AVVvw1RZCuYHIlFwctjph7DzAyXMIdKiB88i5gXN5B93Z/NlrnbNopJN5CuR3PP83KwesY7mDiN6YvH+ZriclQ9tj7Spm/h+6zddDGvP+uP1s2bB2GwARfSKkDvEjIJ3rPYBNZt5Y9SRWldvGsACAs3INDZWDz9EGBgMI8R/AE+Di3SUefKs3nEU3MSQ3Q5MyvBYfaIs0oEqdPNCGpdvuaCexAGX0CjETpWyvW2YQZPxMBKdQV1DYa9Eriw+Zwg49IwP0FZQdK7017XtWuKlpkoiG4rlGVg5zVOo9Jpydu1Mopq0AZrh6DrgWYXpRmGxlAaqSZCs50Pol2cHc6ixS9pmt9jhaWG63610VO1c7F/MHtp4HyEk7m6SpaC+K03MCcwebmeOQRJlxPoAI/6GpBQLiAB4L5OrMwKRRo81zHLsUajIvwGa2qRqm1Pgvk5QMEhPh9TxCT0JHT7bXqNKta45q+7oQLppakvtexkeAwK12/j6hsKcGUfN8khdU2nMziwaYORk3i9EMPhfkQO8tex7ZC3GlpgzekTgHPt8Ba0U4bMGoBBACHJgZdIToFa9uWO5n5rt33c8ua1eZENEzIVDWk5JalIOuXfVSSE7mpwHcDeAZAD5s/6EfXG4bw0LoIpJt1qdlIUxtevcAwzsCiKSMJDD8oivTMwpd65ZFhR8LsoxrD/QTiIlSx5L6JN5FMaTHSM8TABwYpPTnyP4BoofLLhMvG8JcKdBFKvG4iWfXfZjAb2n0Zp6o0bMBIS+EkJyfA0gMOrHWA6nHatToqIyzMiZR3dZl9aA1f+M9wuhShB2DGnh/bo5w+X2xXT9NtYtxVgLVJsL6XULOsLkIrcxHnkXvVWCgDLDiUYPVLIeSFg7quibwPhpJLqbA/Dq4XBdjHhpQvQ03ux6hJs7KDkK0342mQKuwEwfQ9lUdY6O3SKJs0/vOFKRFu8k906vrSJ2Y3v2jOai86EdH3vd/k5dJ+xMC+wyOg3ZxUKqy/aYou3YsHCIEmq7jEsbnQD6TvI5BXXnZz4MsuQeZgcDcc24cERyA1uVwiEF4dy52nhADQgQQXLz/Jarg6EweVX8vZtEbxy1E9BQAXwvgM5h5K/nqVTjg2lXM/A4i8prg/yEi+u3jGPNtY1gsINmtE/Ti9XckN1ygxQhmF0VrD6Hh6sy9h4SJ5GZO9hGr8BXrNTp0D86IhZwLn6lBMQ/ObBIzy3kCyHwOchkCCB4AUw52Hm1gOBCqJmCUOdjyXSGpRCbmOMZ0zKL4eWHsCworNa7ede1nbKEr9aRjrZDCQvG3NrWZ1DWEyQUxKEp8kLY5XUGqEBBYyALnNI9TzUDFGMEotD4Hza53sGFo+zh9YvzZoCvWpLpVhvsCcKG3vTG2LApmnwH5RI6Rl0KmyMvYwYDqeZ915Dw4G0v7EpeJkq234zjs89TwxYpxjVp6olFM77ohcXhSZ4b1XLRNikCxCh+m19TnQr1vMrj2ukSdBmGNnbROcR5oBNoLxZoY1GpL9pWLIbXiXxBJoa4vACf3RJhckPFYVJ300QsgkJPoxIFBjtAEjtcvMAv0yd17B5IoHIgGhJj7qINK21UYy/sjMwa8q745YnkpgBGkeSQAvIGZn30Ta1dtEVEB4E1E9O0A3gssW6Tp8HKqhoWIvhrAV0Em5ReY+QX6+b7c7GWyeJH3un/id8xoAThiFL5j5KQPapoY7BLhGq2kUc5uSVQkIXqaPCQnEEBKFABiWCIKhyXSWsBzW04BVoZTA9MuRF7ygApJwX7vSKAEiSx2mSTLHZGyqdJ6GG2pseMcfeKNpgapXUg6JzAScZA8i8vAxUTyNBwi80qopJZn0Egny0FZGZsLYr4JFOMu0au5ELZxNwmrK3TjiueQRnIuQ9CeVAhtbFDKCSki0tItX5ON0I7Pg9buFKZSU4kxdD7mncLkgjCisgJudlWjAu2ioNET1fMIF0ZJxm3MQPlRAtmm7DhKa3x0hcsQALLalqK7vmZ4nBe4UIt/O6ZdlzeJcxVCZ/ySCIR0PkMxlt+2jdSXTC6A6m20LocPVmlfwrWyGF70oUK3ts+yW7IJ4lB5vbVDEraEoNFIkofsPZa6w+aobQAD7Qkk75l5V7jqgGtXfQXEVfoqAM+DwGhPP/QAl8ipGRZt2/yFAD6GmedEdJd+vhI3ey+5WYekDgxHQJ6wvgD04SOi2EPMEunxJlZDQ1oHk76ON7vL0DCQJb8hXoiKOMSHrrUIRR+OwF0uqQkMIoIni5gA4i7Ut/YNDGAeZB9EBGfbM8dIy7Yj7hSKjGvnPPTyQDru3nksGh1nWHzRhxE5oB2twVXJIl0RmimkSFUhlMXrQXxDIpWGgPFGbLMDoGtR0zYSWRozzWVwdXesXnsdqxex3IGN2RyM2LGgcw44LxCIhCDHjEAeKKagbASmc3H5gaiIXQa3fQXt+IJASPW2VKIbQaCQhqHQ/nBxblkNg+/PP0ICrS52A7B7tpS6K+LQ9dPyHiGfdIwxLRzuXbe0XUvbgF0ea8NipFGug9oGodyQguN8rBRjlmgUhMAMHhWom4Ayy4FRjrpleMqQMYPVGWrZwYXuHvdJqq/nUKk75UiNC8TJYmG0dI7hCVCBGXv6kmdOmPkviehOff3Nx3ms04xYngPgW5WDDWZ+v36+Kjd7qdzMveRI8jNygwO5T9ydhXAbQAzdoXiwTzylECR3Q9pHyJE+XHrjeyIwOHpPPmHABBBaZngSOMsMyuI5pcaFNaLh5Ls0cmtZDIx9FvfHcs7xfAji7WpCNV24qUdkAASeiR6zDUQPmORklnUloFRBQ3HqYiKrZBqElrKmLLppqvgdYHkCJRS0XaFoLwekbUNibgWI7XdSqFImoe0blUX6dRItmMKVSNKiQJ0KRjfumO/S2pNQx4iEbYW00EaoCl67PGxfBdWWW/FiMA1eMyYfC009GnkbI7lYwGvXwxLonDvNCc00Cs+kuapdayVo8KJjleTG7BqGUopnQ7khxcWQcwjWlggSSTSBkel9NmtC5xQxULEgBSZtcp/vFl1EByuN15l7v43HN5T5iFhge43nLAuJh/lNkEhFuT/UAPhuZv43x3HM0zQsjwXwaUT0YgAzAM9n5t/HitzsRbEci3gyBx9MSBTwPACenOK4un/dp7GywJpUVI9V9tFhuE61zGLE0aKLGqIhc4QWDtbUrk0yl8vOZdmDspd3tnjztzpPliiVSAU9OCHWwiTJ426HoW8wzGil7Cg7pik+S2IrxGZimxIkOeudMtdICAAxx+CcGr0Q25QYbEYeQKuQkiW5zQCpcraiO4bAU2n/rmURmFGW43jJaxNPH+fbZtWbRYEYEdI5tddWUyTEAeuMoIl0MyoKV1EDKXjlkVbZc8/AWPJfzn8k1OskJxTZWcZoW2huahFxZzCpvyxEeqMswJ8pHBwKW/OE4i0QmNG0YUejw1admMX7cPE+Tg3BXvd9YCGvrPqcH1vgwieTvD8C+RoAnwLg45n5XQBARI8G8F+J6HnM/JKjPuCxGpa9uNl67AsAPhHAxwN4hZ7sytxsInoWgGcBwMMfIay7m73OPS8/MFithz0k6c1uYXoduAcvQQcfQ3P0FX58cMCoTa/dRIfU/TyxZUSFVJi5gykS2GwRTsjIRQUJIEJnIA+E5EIJ/3QpLtDlLuQ3i0UEFFoxqqRKKxt10UU+7kVACDektqLckAaZQVlxALjIOoWpzQpj7ioT9lpMcrsMnGmurG367UVcFgvy4jSqYm7gwGHJ9XJ6XvYxEVp2HZPXxpQQNKxLhOVx4CGKvO5o1ogdtyXRHSM5i7acFKWaYZBtjB5ttSLJe3gxJEpEiGy+OC8p95hl3OV6Es1kkQnX2yyBZuWzhXs+ieriZ9jdgKzyOJwNfc4HoRufpjwDwN9m5vvtA2Z+JxF9OYBXA7i1DMte3Gwieg6A/66c7N8jogDgDhyAm60FRi8DgMc/4Qk3fYV3u7nFq+9wXJNmwQMzRZ2MK25/sw/OquNeJUJLv08htEUxhekTS2S5HgBghfpkPz4yeeTLTnF369yIoqoV9vF5DmdU4EQcUUcBJZIi1cSzjrmeuCaN67o6AwInJbTVMFrrIiozTL7LX0XygOU2rFg0yQGJYu8eD4MblzkBrbjQ8Q7wGgVYPUasxUiLVGl5CxMy4oLCT5xAePA5ML8uNHPWbtw6z0x51z5I58GiOlnTZCStZqqtLopxDsRdkSsj6Q7sXJ8ersy4eQAKNbwStTFC2GlMljk9yyDdg8hZ0+EM3CqGJU+Nigkz30dER7Oc5oKcJhT2SgCfBeA3iOixkErQ+3ET3OzjkmUK2R4ay4HYd60m/k/6PjvMw5k+/MsYdY1+budM8Xcd/GHuaqRaG8Rir4mQJYa3dXkHfykbSCiiqriZEeDgjb5NBMv79GjPVozXNnHpYwSFx1TRUrXZ3xZIoD0fYRzJC0zimCLMxRGQRqvR3a7znXzBkN/1mEkJJbZHCwYkarEOBIAazSKOVz7LEXwOl5VAUAKC1QfF/fTrXJCSHry2hCEHNoPppIdXSo+OK5/CIkaZFHYZapa5sQzOXizMW0PfHlIYaNtbInu/18pmR7ekZiKnaVh+EMAPEtE9kJN7pkYvB+Vmn4js9dDY+1vtYdoNUmuVbdZyZ1iYECG/kCgUI2hzUosQki4GzJaI5QgVeYXhHDS3lLCBbDxVDBG5Gx9l0cj5IocDd7AOoPUuYrWoTepJYp7EyTLUdv5Klggt96I1O2fv5HwN8lx1Lm1fZmAcEVyac6KkuBFaE2WLVC0U0cZzaGt4XfIhTC7IpkmnY4OqpMBUacwaFfFoDZbE5yx0nYoV9osti6yoU6OjlHzRwPWi+GX3ziLU9WCQWyRi+T+I6NqSzwlAueTzQ8upGRZmrgB8+S7fHYSbPcghZRl0sdQbZdkwpYKakUk9fIBjQn5ZesxgLyMOLB7TbIrlt5YZ8sA7I0TnR7EDg7N1d9TYtH6EOjAQulqI9OiLEKEp0L0gzWXS2y5YtKc1UiSRXcq445SVlcByMXyyNXw4yE8WiRNaTwNoNGaQmS2pkHV1SG1Wwrdz+akZMg7S6VhZbE2x1uUOxSxKMMjYEbGtAnfdzsLMt0Tynpn9/lsdrdw2lfeDHE52w8WBnZDZzRSaLVsS1fa33za7/SbNexFRVGptywAVcAB8Ptnx+/R3e0Uiq+iMveDPNJI1qq13BKIMzgnjsAksHRQ07xFAcAk0xUmOxlqmGNvM1uDptnUdZTrLul5yLJ0UiNSQUdfQsqtX8qgpi91608iNl0QijrrrdYusongscivQjU9DBsNym0tqFFZJ8h9k+72Ot+O3zLsarm6Tm39Il5IRGD3q9n5yszR1k/1ybIGB0No8dBuKYjZqu0aAbIzELoKCwlFoGZ4An42EDMFtRwJQ41S1Ad4BWTaSbgyOovFqXQ5SYxbIA+R7/bMW4a3dCCieBsV6KxVInqQMhuUE5OK6eM1XbmwdC1SwqjFIlcTF9Qk+cH2r953J+TUZb/r94jbpsVbF1veiSe9HoU6LPA+672XbHtaA7jWGVbfbD35cLHRNpQWhaQIyl8HnWoipEGGjzLA6+dG8CQhMcMQYZRIF1XC7tvTZk6iwy3gfbMJ8y9SxnLgMhuWYxYwKIAp7UVkvykESoMuUaHq8ZXLlxlY0HPtta9+nY7bfmnzg+lbPWO0nV27sPP+9iBF70aMPQ+U+C/pgtzHsZvTMMUhp8E3gSLJYtp+6lWglheXmSX+rlFq+KgnlNNiPZ1IYCCe/0NctIYNhOUZZpmgXP1tmaFZROMBOJb+K3Mxv9jIYqxiTZcc3A3MzNTi3u1JLI5pUiS/+v5fRNTEjlM5dz5C3KdQ2yMGEewzJQToZDMsRyEGV636/XaZ0l0FVt7IsnkMayeyXDH+wyEGM7nEfC+iM3IPtOuwmjFuGbnziMhiWFcXR8tzDYYzKMlkGFQH93MjtKLsZy/2gw0GORxYdGbsv0+dgt3v1QSM8GJbd5LY0LItV8nvJfvfFKnDWccvtakxWkYvrk2MjPQyyU5Y9L1dubHVLMNzC0M9xQKlD8n653DaGxZ6H20kJp97hg1mGaOb05dx0vC/x4izIXo5kbCh7AAr6XsLMCLdGS5cTl9vGsHjnbiujMsj+skiZfrDKUVbAp79LGWO7wV5niSF2GnnIIWJZLreNYbldZIhQVpPBoCyXVbzyvViHKeOsK9JcLmnu5bRkmVE9yWeIw6m3MTyTMhiWQW4puZ3yLTdbs7TX9uem492/TGTRMFsuq7dkAvPK+ztp2Q3yOlHHjHkwLLvIYFgGuWXktL3jo5BlCjH9bL8OA7b9bsYlLYDdS5bBxgdRylc3t08lkb9XDuWko33GYFh2k8GwDDLIEUoahRwnFd2MS9ou6Cj3v0xJn7ZhP0tGBYDQjdvBsCyTwbAMcsvIqn3QTkIWowxgdRjqZmUv5XnUijWFHPej7O/WQ+6o5MwWB3NAaI5lnaxbXgbDMsgtLcsWmlqU41R0t4NYvsXyLMDO/m0HOeejrhc5U8ZkQQYobLm4/Tc5HiGixxPRG4joTUT0RiL6hOS7FxLRO4jo7UT0Oac1xkHOllxcn8T+WfaXym4Mpr1+s8r26d8ivHV+bYJz0/GZTXIfRKyh6G7NPQ/6dxi5uD6Jf2dVLMeyyt9hhIj+LRG9RXXlq4nog5PvzqSuPM2I5dsBfDMz/xIRPVXfP5GIPhLAlwH4KMia979KRI89C8sTD3L6cha817MwhqOUs6y8z7TwiUUs38HM/xoAiOifA/hGAM8+y7ry1CIWSA+3DX19DsB79PUXAvhJZp4z87sAvAPAJyz5/SCDDDLIKQojhHalv0MdhTldr36KbpW4M6srTzNi+RoAv0JE3wkxcJ+snz8MwBuS7e7Vz3YIET0LwLMA4JGPfOSxDXSQQQYZZIccrI7lDiJ6Y/L+Zcz8slV/TEQvBvAMAFcBfKZ+vLKuPGk5VsNCRL8K4CFLvvp6AE8C8Dxm/hki+lIAPwDgs9G1/UplKWKrF+ZlAHD33XffJmVzgwwyyK0gzIxQr8wKu5+Z797ty710JTP/D2b+egBfT0QvBPBVAL4JB9CVJy3HaliY+bN3+46IfgTAc/XtTwP4fn19L4BHJJs+HB1MNsgggwxyZuSocix76coF+QkAvwAxLGdWV55mjuU9AD5DX38WgD/X168C8GVENCKiRwF4DIDfO4XxDTLIIIPsLnxirLDHJG+/AMDb9PWZ1ZWnmWP5JwD+ExFlAGbQXAkzv5WIXgHgTwA0AP7ZWWA5DDLIIIP05cRaunwrET0OQADwlwCeDZxtXXlqhoWZfwvAx+3y3YsBvPhkRzTIIIMMsrrI0sTHvx4LMz99j+/OpK4cKu8HGWSQQW5GmIeWLrvIYFgGGWSQQW5GmA9do3K7ymBYBhlkkEFuQhhDd+PdZDAsgwwyyCA3I8NCX7vKYFgGGWSQQW5KBsOymwyGZZBBBhnkJmUwLMuFTmN50eMQIroO4O2nPY4FuQPA/ac9iCVyFsc1jGk1Gca0uuw1rr/BzHceZudE9Mt6jFXkfmZ+ymGOdyvJ7WRY3rhXL57TkLM4JuBsjmsY02oyjGl1OavjejDIabZ0GWSQQQYZ5DaUwbAMMsgggwxypHI7GZaV1zY4QTmLYwLO5riGMa0mw5hWl7M6rttebpscyyCDDDLIIGdDbqeIZZBBBhlkkDMgg2EZZJBBBhnkSOW2MCxE9BQiejsRvYOIvu4Ux/FuIvpjInqTrW9NRBeJ6DVE9Of6/4VjHsMPEtH7ieie5LNdx0BEL9R5ezsRfc4JjulFRPS/dK7eRERPPeExPYKIfp2I/pSI3kpEz9XPT22u9hjTac9VSUS/R0Rv1nF9s35+mnO125hOda4GUWHmW/oPgAfwFwAeDaAA8GYAH3lKY3k3gDsWPvt2AF+nr78OwLcd8xg+HcATANyz3xgAfKTO1wjAo3Qe/QmN6UUAnr9k25Ma00MBPEFfrwP4Mz32qc3VHmM67bkiAGv6OgfwuwA+8ZTnarcxnepcDX/ydztELJ8A4B3M/E5mrgD8JIAvPOUxpfKFAH5YX/8wgKcd58GY+XUAPrDiGL4QwE8y85yZ3wXgHZD5PIkx7SYnNab3MvMf6uvrAP4UwMNwinO1x5h2k5OaK2bmG/o21z/G6c7VbmPaTU5krgYRuR0My8MA/HXy/l7s/TAepzCAVxPRHxDRs/SzD2Lm9wKiOADcdQrj2m0Mpz13X0VEb1GozGCUEx8TEX0IgI+FeL1nYq4WxgSc8lwRkSeiNwF4P4DXMPOpz9UuYwLOyH31YJbbwbDQks9Oi0P9Kcz8BACfC+CfEdGnn9I4VpXTnLv/CuBDATwewHsB/IfTGBMRrQH4GQBfw8zX9tp0yWfHMq4lYzr1uWLmlpkfD+DhAD6BiD56j81PZFy7jOnU52qQ28Ow3AvgEcn7hwN4z2kMhJnfo/+/H8DPQkLt9xHRQwFA/3//KQxttzGc2twx8/tUMQQA34cOljixMRFRDlHgP87M/10/PtW5WjamszBXJsx8BcBvAHgKzsh9lY7pLM3Vg1luB8Py+wAeQ0SPIqICwJcBeNVJD4KIpkS0bq8BPBnAPTqWZ+pmzwTwP056bHuM4VUAvoyIRkT0KACPAfB7JzEgU0gqXwSZqxMbExERgB8A8KfM/F3JV6c2V7uN6QzM1Z1EdF5fjwF8NoC34XTnaumYTnuuBlE5bfbAUfwBeCqEQfMXAL7+lMbwaAjr5M0A3mrjAHAJwK8B+HP9/+Ixj+PlEAighnhp/2ivMQD4ep23twP43BMc048C+GMAb4E89A894TF9KgQKeQuAN+nfU09zrvYY02nP1ccA+CM9/j0AvnG/e/sE5mq3MZ3qXA1/8je0dBlkkEEGGeRI5XaAwgYZZJBBBjlDMhiWQQYZZJBBjlQGwzLIIIMMMsiRymBYBhlkkEEGOVIZDMsggwwyyCBHKoNhGWSQQQYZ5EhlMCwPAiGiG/tvdaj9/yIRnde/f3oTv38iEf38Abe/SkS/uMv3/42Ivvig47gVRefik5P3zyOivyKil57muAZ5cMtgWAY5tDDzU1naapwHcGDDcpPym8z81P03u3khouw4939E8kQA0bAw80sAfOOpjWaQQTAYlgetENHjiegN2gX2Z60LLBH9BhF9my6i9GdE9Gn6+YSIXqHb/xQR/S4R3a3fvZuI7gDwrQA+VBdY+o7FSISIXkpEX6mvn0JEbyOi3wLwd5NtptqV9veJ6I+IaN8lEEjkpUT0J0T0C0g6SBPRxxHRa7Xj9K8kva0+Xs/ld3Ss9+jnX0lEP01EPwfpVL10PCSddb9DP38LEf3f+vlDieh1Ogf32PztMu4n6/H/UI+5pp9/o+73HiJ6mbZ6ARH9cz3HtxDRT5J0QH42gOfp8XY91iCDnKicdun/8Hf8fwBuLPnsLQA+Q1//GwD/UV//BoD/oK+fCuBX9fXzAXyvvv5oAA2Au/X9uwHcAeBD0F/M64kAfj55/1IAXwmghLQwfwyk6+wrbDsA/w7Al+vr85BWPdOFsS/u9+8CeA1k0bcPBnAFwBdD1uj4bQB36nZ/D8AP6ut7AHyyvv5WG7eO715oe5LdxgPgWQC+QT8fAXgjZAGpf4munY8HsL7LNbkDwOvs3AB8Lbq2JGlrlB8F8Hf09XsAjGws+v+LsLCwlZ7DS0/7vhv+Hrx/t0KoP8gRCxGdgyim1+pHPwzgp5NNrNPvH0CMBSB9rP4TADDzPUT0lkMM4cMBvIuZ/1zH82MQRQ1I884vIKLn6/sSwCMhi17tJp8O4OXM3AJ4DxH9T/38cRAj+Bp1+j2A92rzwnVm/m3d7icAfH6yv9cwsy1Mttt4ngzgY5JczjmIofx9AD9I0qX4lcz8pl3G/ImQVQ1fr2MrAPyOfveZRPQCABMAFyG9534O4gz8OBG9EsAr95iPQQY5VRkMyyDLZK7/t+jukWXrWewnDfpwa5m83q1JHQF4OjO//YDHWrY/AvBWZv6k3ofJ2uy7yOZ+41F46quZ+Vd2HFTW4fk8AD9KRN/BzD+yy9hew8x/f+G3JYD/AokG/5qIXoRu3j4PYkS/AMC/JqKP2uc8BhnkVGTIsTwIhZmvAricYPJfAeC1e/wEAH4LwJcCABF9JIC/uWSb65C12k3+EsBHkrQqPwfgSfr52wA8iog+VN+nyvVXAHx1klf42BVO6XWQluhecyifqZ+/HcCdRPRJuq+ciD6KmS8DuE5En6jbfdke+95tPL8C4DkamYCIHqv5mL8B4P3M/H2QFvhP2GW/bwDwKUT0Yfr7CRE9Fp0RuV9zLl+s3zsAj2DmXwfwAggst4adcz7IIKcuQ8Ty4JAJEd2bvP8uyPoZ30NEEwDvBPAP99nHfwHwwwqBWbvyq+kGzPwAEb1eE+G/xMz/ioheodv+uf4OzDwjWbr5F4jofojRshUJ/y2A/wjgLarM340+TLVMfhbAZ0Hapf8Z1Egyc6VQ1X9Ww5bpvt8Kad3/fUS0CckrXd252z3H8/0QmPAP9fP7IGu+PxHAvyKiGsANAM9YtlNmvo+EyPByIhrpx9/AzH9GRN+n5/JuCLQGCIz3Y3oeBOAlzHxFSQb/n5IKvpqZf3OfuRpkkGOXoW3+ICsJEXkAuRqFD4Wsv/FYZq5OYSxPhCSs9zM4e+1jjZlv6Ouvg6zb8dyjGeHpihqsu5n5q057LIM8OGWIWAZZVSYAfl2hHwLwnNMwKioVgI8mol/km69l+TwieiHkGfhLCJPqlhcieh6Egvwzpz2WQR68MkQsgwxyzEJEvwuhJKfyFcz8x6cxnkEGOW4ZDMsggwwyyCBHKgMrbJBBBhlkkCOVwbAMMsgggwxypDIYlkEGGWSQQY5UBsMyyCCDDDLIkcr/D3+DVuTRlMImAAAAAElFTkSuQmCC\n", + "text/plain": [ + "
" + ] + }, + "metadata": { + "needs_background": "light" + }, + "output_type": "display_data" + } + ], + "source": [ + "ds.sst[0].plot()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "But not the end..." + ] + }, + { + "cell_type": "code", + "execution_count": 29, + "metadata": {}, + "outputs": [ + { + "data": { + "text/plain": [ + "" + ] + }, + "execution_count": 29, + "metadata": {}, + "output_type": "execute_result" + }, + { + "data": { + "image/png": "iVBORw0KGgoAAAANSUhEUgAAAaAAAAEXCAYAAADr+ZCUAAAAOXRFWHRTb2Z0d2FyZQBNYXRwbG90bGliIHZlcnNpb24zLjMuMiwgaHR0cHM6Ly9tYXRwbG90bGliLm9yZy8vihELAAAACXBIWXMAAAsTAAALEwEAmpwYAAAznUlEQVR4nO3deZxcRbn/8c83IewEhARkiwncAAKiwLCJemUVEAkKIigYFY0oCMhVCBd/4IYXQVG5qBhULiiLAUSDRjFEwBUhIFtYwx6IkCBLACEk+f7+qBrTDL2c6WV6uud5v17nNX3WevqI/aTq1KmSbUIIIYSBNqzdAYQQQhiaIgGFEEJoi0hAIYQQ2iISUAghhLaIBBRCCKEtIgGFEEJoi0hAQ4CkMZKelzS83bF0EknvlDS33XGE0K0iAXUhSQ9J2r133fYjtle1vaSdcVWTY35C0iol2z4u6doC556TE+zzkhZJeqVk/TctDXwAKfm6pKfycrokVTl+N0l3S3pR0jWS3jCQ8YZQSySgMJgsBxzT35NsH5ET7KrA14Cf9a7b3rvpUbbPJGB/4M3AVsC+wCfLHShpFPBz4P8BawKzgJ8NSJQhFBQJqMtI+gkwBrgy1wCOlzRWkiUtl4+5VtJXJf0lH3OlpLUkXSjpOUk3Shpbcs3NJM2Q9E9J90g6qEXhnwF8TtIaFb7bdyQ9mmO8SdLbm1GopGdKakwv5Hs1tsxx60m6XNJ8SQ9KOrpk+78krVly7NaSFkga0YwYs4nAN23Ptf0Y8E3gIxWOfR8w2/altl8Cvgi8WdJmTYwnhIZEAuoytg8DHgHek2sAp1c49GDgMGB9YGPgr8B5pH8t3wWcApCbxGYAFwFrA4cA35O0RbmLSvpe/kEvt9xWI/xZwLXA5yrsvxF4S47xIuBSSSvWuGZNttcoqUF9B/gj8FjpMZKGAVcCt5Lu2W7AsZLeZftx0v07oOSUDwKX2X6lb3mSPljlHj0jaUyFULfI5fe6NW+reaztF4D7qxwfwoCLBDR0nWf7ftvPAr8B7rd9te3FwKXA1vm4fYGHbJ9ne7Htm4HLgQPLXdT2p/MPerllqwJxnQx8RtLoMtf+qe2nchzfBFYANu3/Vy9P0gdIieOAMoljO2C07S/bXmT7AeBcUiKHlBAPyddR3n5RuXJsX1TlHq1h+5EKIa4KPFuy/iywaoXnQH2P7T1+tQrXDmHALdfuAELbPFHy+V9l1lfNn98A7CDpmZL9ywE/aUVQtu+Q9CtgMqkm9m+S/gv4OLAeYGAkMKoZ5UraGjgb2NP2/DKHvAFYr899GE6qLQFcBvyvpPWA8Tm+P9Jcz5O+c6+RwPMuP6Jw32N7j1/Y5JhCqFskoO7UzCHOHwWus71HkYMlnQMcWmH3w7aLNAGdAtxMesbRe923AyeQmr5m214q6WmgYi+wonJt6wrgKNt/r3DYo8CDtseX22n7GUm/Aw4C3ghcXCExIOlDwA+qhLR5hVrQbFIHhBvy+pvztnJmk54Z9Za5CqmptdLxIQy4aILrTk8AGzXpWr8CNpF0mKQRedlO0hvLHVzaI63MUuj5g+05pB5bR5dsXg1YDMwHlpN0Mq/9F35FuZv3R8psX47UpHih7Wq9xG4AnpN0gqSVJA2XtKWk7UqOuQj4MOlZUNnmNwDbF1a5R6tWaYK7ADhO0vq5pvVfwP9VOPYKYEtJB+TnZCcDt9m+u8p3DGFARQLqTv8DfCE/0K70QL8Q2wuBPUnPNB4H/gF8nfT8pZW+DKxSsn4V6VnVvcDDwEukWklNkpYH1gKuL7N7A+DtpA4Fz5csr+oIkN+heg+pE8SDwALgh8DqJYdNIzW/PWG7tLNAs/yA1BHiduAO4NeU1KQkzc61K3Iz4gHAqcDTwA4se14VwqCgmJAudDtJbwOOtH1Iu2MJISwTCSiEEEJbRBNcCCGEtogEFEIIoS0iAYUQQmiLrnkPaNSoUR47dmy7wwghdICbbrppge3XjLbRH+/aZRU/9c9iA8zfdNvLV9neq5HyulHXJKCxY8cya9asdocRQugAkh5u9BpP/XMJN1xVadi+Vxu+7n1NGbGj23RNAgohhIFkYClL2x1GR4sEFEIIdTDmlcE7x2NHiAQUQgh1ihpQYyIBhRBCHYxZEi/yNyQSUAgh1GlpUweeH3oiAYUQQh0MLIkE1JBIQCGEUKeoATWm7SMhSPpsHkb+DkkXS1pR0pqSZki6L/99XbvjDCGEUgZesQstoby2JiBJ65MmHeuxvSVpiuODSdMxz8yzT87M6yGEMGgYs6TgEsprew2I1Ay4Up6ZcmXSpGcTgPPz/vOB/dsTWgghVGBYUnAJ5bU1Adl+DPgG8AgwD3jW9u+AdWzPy8fMA9ZuX5QhhPBaaSSEYksor91NcK8j1XbGAesBq0g6tB/nT5I0S9Ks+fPntyrMEEIoQywpuITy2t0EtzvwoO35tl8Bfg68FXhC0roA+e+T5U62PcV2j+2e0aMbGtg2hBD6xcBSF1tCee3uhv0IsKOklYF/AbsBs4AXgInAafnvL9sWYQghlGFgUdv/Dd/Z2pqAbP9N0mXAzcBi4O/AFGBVYKqkw0lJ6v3tizKEEMpb6mhea0S7a0DYPgU4pc/ml0m1oRBCGJTSSAiRgBrR9gQUQgidyIgl0QTXkLh7IYRQp6VWoaUWSXtJukfSHEmvefFeyVl5/22Stumzf7ikv0v6VRO/XstFDSiEEOpgxCIPb/g6koYD3wX2AOYCN0qaZvvOksP2BsbnZQfg+/lvr2OAu4CRDQc0gKIGFEIIdUgvog4rtNSwPTDH9gO2FwGXkN6PLDUBuMDJ9cAaJa+qbAC8G/hhU7/gAIgEFEIIderHi6ijel+az8ukksusDzxasj43b6PgMd8GjqcDB12IJrgQQqiDLZa48L/hF9juqbCv3EOivq+vlj1G0r7Ak7ZvkvTOosEMFlEDCiGEOi1FhZYa5gIblqxvQBqUucgxOwP7SXqI1HS3q6SfNvKdBlIkoBBCqEN6D2hYoaWGG4HxksZJWp40Jc20PsdMAz6ce8PtSBq4eZ7tE21vYHtsPu/3tguPp9lu0QQXQgh1MOIVN/4TanuxpKOAq0hzov3Y9mxJR+T95wDTgX2AOcCLwEcbLngQiAQUQgh1WtKkoXhsTyclmdJt55R8NnBkjWtcC1zblIAGSCSgEEKoQ4yE0LhIQCGEUKelxXvBhTIiAYUQQh16OyGE+kUCCiGEOhg17RnQUBUJKIQQ6mDTlF5wQ1nb64+S1pB0maS7Jd0laSdJa0qaIem+/Pd17Y4zhBBerdhLqAVeRB2y2p6AgO8Av7W9GfBm0oiuk4GZtscDM/N6CCEMGgaWeFihJZTX1jsjaSTwDuBHALYX2X6GNPLr+fmw84H92xFfCCFU06SREAYVSW+T9NH8ebSkca0qq913ZiNgPnBenkzph5JWAdaxPQ8g/127nUGGEEJfpthkdEUmpBssJJ0CnACcmDeNAFo2tly7E9BywDbA921vDbxAP5rbJE3qHd58/vz5rYoxhBBew6ROCEWWDvJeYD/SbzG2HwdWa1Vh7U5Ac4G5tv+W1y8jJaQnSiZbWhd4stzJtqfY7rHdM3r06AEJOIQQkmJzAS3prE4Ii/KwPwbILVIt09YEZPsfwKOSNs2bdgPuJI38OjFvmwj8sg3hhRBCRSaNhFBk6SBTJf2ANOPqJ4CrgXNbVdhgqBt+BrgwD0P+AGmU12GkG3E48Ajw/jbGF0IIZXVY7aYqSQJ+BmwGPAdsCpxse0arymx7ArJ9C1BupsDdBjiUEEIozFan1W6qsm1Jv7C9LdCypFOqe+5eCCEMsC58D+h6SdsNVGFtrwGFEEInShPSDW93GM22C/BJSQ+TesKJVDnaqhWFRQIKIYQ6pE4I3fMMKNt7IAuLBBRCCHXqtFEOCvBAFhYJKIQQ6tA7EkKX+TUpCQlYERgH3ANs0YrCIgGFEEKdlnZZDcj2m0rXJW0DfLJV5UUCCiGEOth0/YR0tm9uZa+4qglI0vsKXOMl29ObFE8IIXQEIxYv7a5ecJKOK1kdRhoarWUDbdaqAZ1LGganWpp/BxAJKIQw5HTTSAhZ6cCji0nPhC5vVWG1EtBvbH+s2gGSWjZUdwghDFZd2g37TtuXlm6Q9H7g0grHN6TqEzTbh9a6QJFjQgih+6gbByM9seC2pijcCUHSW4GxpefYvqAFMYUQQkdY2iVNcJL2BvYB1pd0VsmukaSmuJYolIAk/QTYGLgFWJI3G4gEFEIYkmx4pXs6ITwOzCJNRndTyfaFwGdbVWjRGlAPsHmeqCiEEIa8bnoR1fatwK2SLrL9ykCVWzQB3QG8HpjXwlhCCKGjdEsTXImxkv4H2Jw0EgIAtjdqRWG13gO6ktTUthpwp6QbgJdLgtqvFUGFEMJg16W94M4DTgG+RRoZ+6NUfw2nIbVqQN9oVcGlJA0ntT8+ZntfSWuSZuYbCzwEHGT76YGIJYQQiuqwHm5FrGR7piTZfhj4oqQ/kpJS09Xqhn2d7euAfXo/l25rYhzHAHeVrE8GZtoeD8zM6yGEMHg4PQMqsnSQlyQNA+6TdJSk9wJrt6qwoul7jzLbmjJvhKQNgHcDPyzZPAE4P38+H9i/GWWFEEKzGFjsYYWWDnIssDJwNLAtcCgwsVWF1XoG9Cng08DGkm4r2bUa8OcmxfBt4HhePQTEOrbnAdieJ6lsBpY0CZgEMGbMmCaFE0IItXXbM6D8KOQg258Hnic9/2mpWqn5IuA9pPHg3lOybNuMERAk7Qs8afummgeXYXuK7R7bPaNHj240nBBC6JdmNcFJ2kvSPZLmSHrNIwclZ+X9t+VpEpC0oaRrJN0labakY+r9LraXANtKGrCsWrUGZPtZSQuBN+UHUs22M7CfpH1IXf5G5rHlnpC0bq79rAs82YKyQwihbs16DyjXPL5LetQxF7hR0jTbd5YctjcwPi87AN/PfxcD/5WnTVgNuEnSjD7n9sffgV9KuhR4oXej7Z/Xeb2qajZO2l5KekGp6W1ctk+0vYHtscDBwO9zzWoay9odJ5JqYCGEMKgsRYWWGrYH5th+wPYi4BLSc/BSE4ALnFwPrNH7j3TbNwPYXkjqzLV+A19pTeApYFeWtXjt28D1qir6Iuq6wOz8HlBpVmzVe0CnAVMlHQ48Ary/ReWEEEJ93LRnQOsDj5aszyXVbmodsz4lgwNIGgtsDfyt3kBst/y5T6miCehLLY0CsH0tcG3+/BSwW6vLDCGEehlYvLRwD7dRkmaVrE+xPSV/LpfF+g57VvUYSauS5u051vZzRYPqS9ImpOa9dWxvKWkrYD/bX633mtUUSkC2r5O0DtA7NesNtuO5TAhhyOrnM6AFtnsq7JsLbFiyvgFpcNBCx0gaQUo+FzbhWc25wOeBHwDYvk3SRUBLElCh9C3pIOAGUlPYQcDfJB3YioBCCKFT2Cq01HAjMF7SOEnLk56HT+tzzDTgw7k33I7As7mTloAfAXfZPrMJX2ll2zf02dbe6RiAk4Dtems9kkYDVwOXtSqwEEIY7JoxGKntxZKOAq4ChgM/tj1b0hF5/znAdNLoM3OAF1n2js7OwGHA7ZJuydv+2/b0OsNZIGljcvNermi0bBDqogloWJ8mt6coPopCCCF0HTevEwI5YUzvs+2cks8Gjixz3p9o7mChRwJTgM0kPQY8CHyoidd/laIJ6LeSrgIuzusfoM/NCiGEoUUsKd4JoSPYfgDYXdIqpIrHwlaWV7QTwuclHUCq7onUg+OKVgYWQgiDXYHnOx1F0lqkka/fBljSn4Av557JTVe0BoTty0k9LUIIYcjrtrHgskuAPwAH5PUPkabG2b0VhRXtBfc+SfdJelbSc5IWSqq7r3kIIXQ8p+dARZYOsqbtr9h+MC9fBdZoVWFFGzBPJ72MtLrtkbZXsz2yVUGFEEInaNJQPIPJNZIOljQsLwcBv25VYUWb4J6wfVftw0IIYWgw3fcMCPgkcBzw07w+DHhB0nGkznhNrXgUTUCzJP0M+AXwcu/GVo2QGkIIg59YsrS7EpDt1Wof1TxFE9BI0stPe5ZsMxAJKIQwZHVhDYg8/ttYSvJDqyobRbthVx0hVdKJtv+nOSGFEMLglzoYdFcCkvRjYCtgNrA0b25ZZaNwN+wa3g9EAgohDCld2A17R9ubD1RhzXqNt+v+VwghhFq6sBv2XyUNWAJqVg2orlssaUPgAuD1pOreFNvfkbQm6eWnscBDwEG2n25OqCGE0DgjlnbZUDzA+aQk9A9ShzORer9t1YrCmpWA6q0BlZ3PHPgIMNP2aZImA5OBE5oTagghNEdnVW4K+TF5dG2WPQNqmWYloEvrOcn2PPJQ37YXSuqdz3wC8M582PmkmVIjAYUQBo8u7IQAPGK771xELVN0KJ7TJY2UNELSTEkLJB3au9/21xoNpM985uvk5NSbpNaucM4kSbMkzZo/f36jIYQQQv+44NI57pZ0kaRD8hBs75P0vlYVVrQBc888z/i+pKlhNyFN29oU9c5nbnuK7R7bPaNHj25WOCGEUEiTZkQdTFYiPfvZE3hPXvZtVWFFm+BG5L/7ABfb/meaCbZxFeYzf0LSunnK2XWBJytfIYQQ2qPDerjVVOudz2YrWgO6UtLdQA8wM0/J/VKjhVeZz3waMDF/ngj8stGyQgihmWzw0mGFlk4haZP8mOWOvL6VpC+0qrxCd8b2ZGAnoMf2K6RheSY0ofze+cx3lXRLXvYBTgP2kHQfsEdeDyGEQaUL3wM6FzgReAXA9m3Awa0qrFATnKSVSXOFjwEmAesBmwK/aqTwGvOZ79bItUMIoeU6K7kUsbLtG/o8YlncqsKKPgM6D7gJeGten0vqet1QAgohhM7VcR0MilggaWNyapV0IPlVmXIkFemy/U/bHym3o2gC2tj2ByQdAmD7X2pWL4QQQuhU3VcDOhKYAmwm6THgQdK03JW8Efh4lf0CvltpZ9EEtEjSSizLihtTMi9QCCEMOd35Iqpt7y5pFWBYHiBgXJXjT7J9XbULSvpSpX1Fu2ecAvwW2FDShcBM4PiC54YQQneyii2d43IA2y/YXpi3XVbpYNtT+27LU3mPrHZMr6LzAc2QdDOwI6lKdYztBUXODSGErtUlTXCSNgO2AFbvM/LBSGDFAudfBBwBLCH1F1hd0pm2z6h2XtGheATsDWxr+1fAypK2L3JuCCF0re4ZimdT0ogHa7BsBIT3ANsAnyhw/uZ5FJv9gemkHtOH1Tqp6DOg75FGRt0V+DKwkFRV267g+SGE0F1MpzWvVWT7l8AvJe1k+691XGJEHtVmf+Bs269Iqpl6iz4D2sH2keTRD/LcPMvXEWQIIXSNbnsRtc7kA/AD0txtqwB/kPQGoOa4nkVrQK9IGs6yXnCjGYC5IkIIYVBb2h01oEbZPgs4q2TTw5J2qXVe0QR0FnAFsLakU4EDgZaNDxRCCJ2gdiPT0CDp5Aq7vlztvJoJSNIw0stIx5OGxxGwv+27+htkCCF0jc7pYFCYpHWArwHr2d5b0ubATrZ/VOPUF0o+r0jq0FAzR9RMQLaXSvqm7Z2Au2sdH0IIQ0PHveNTxP+Rhl47Ka/fC/yMNGtBRba/Wbou6RukWQ2qKtoJ4XeSDojhd0IIoUT3dMPuNSq/OLoUwPZi0rs9/bUysFGtg4o+AzqO1LthsaSXSM1wtj2y+mkhhNDFOiu5FPGCpLVY1uFsR+DZWidJup1ld2M4MJoaz3+g+EgIqxU5LoQQhgzTjb3gjiM1nW0s6c+kRHJggfNKp+1eDDyRa09VFR0JYZsyy8aSitag+k3SXpLukTRH0uRWlRNCCPWSiy01r1Pj907JWXn/bZK2KXpuf9i+GfhP0tQ7nwS2yJPSVYq7txVsYcnyL2CkpDVrldefkRC2AW7P628CbgXWknSE7d8VvE4h+Z2j75JmQ50L3Chpmu07m1lOCCE0pAlNcAV/7/YGxudlB+D7wA7N/q2UdCRwoe3Zef11kg6x/b0Kp1xEqv3cRLobpVVCU+M5UNFOCA8BW9ve1va2wFuAO4DdgdMLXqM/tgfm2H7A9iLgEpozBXgIIQw2RX7vJgAXOLkeWEPSugXP7Y9P2H6mdyWPelNxLDjb++a/42xvlP/2LjU7IRRNQJv1ZsRc2J2khPRAwfP7a33g0ZL1uXnbq0iaJGmWpFnz589vUSghhFBeP5rgRvX+VuVlUsllivzeVTqm0G9lPwwr7e2ca1g1h12TtHOeQwhJh0o6U9KYWucVbYK7R9L3SdkV4APAvZJWAF4peI3+KPdk7zWVXdtTSLP30dPT0339UUIIg1vx94AW2O6psK/I712lYwr9VvbDVcBUSefk6xxBmguulu8Db5b0ZtKgBT8CfkJ6nlRR0QT0EeDTwLGkL/wn4HOk5FNzvJ86zAU2LFnfAHi8BeWEEEJ9TLNGxCzye1fpmOULnNsfJ5A6H3yK9Fv/O+CHBc5bbNuSJgDfsf0jSRNrnVS0G/a/JH0P+JXte/rsfr7INfrpRmB8ngr2MeBg4IMtKCeEEOrWpLHgivzeTQOOknQJqRPCs7bnSZpf4NzCbC8l1Wa+389TF0o6ETgUeEduuhtR66Si3bD3A24hV8UkvUVSzWEW6pX7jx9Fqg7eBUwtfQYVQgiDQhNGQqj0eyfpCElH5MOmAw8Ac4BzSS1STf+tlDRe0mWS7pT0QO9S4NQPAC8Dh9v+B+k5VNXZUKF4E9wppN4W1wLYvkXS2ILn1sX2dNJNDyGEwalJT57L/d7ZPqfks4Eji57bgPNIv/ffIj1e+SjlnzP1jeEfwJkl648AF9Q6r2gvuMW2aw7HEEIIQ0XRHnAdNmXDSrZnArL9sO0vkmbCLkvSQknPlVkWSmrahHR3SPogMFzSeOBo4C8Fzw0hhO7UfUPxvJSn4LlP0lGk50prVzq40WHaitaAPgNsQWrju5g01eqxjRQcQgidrgtrQMeSRrI+GtiW1KmgZm82AElvk/TR/HlU7hhRVdFecC+S5oc4qdaxIYQwZHRWcqlI0k9sHwa81faNpN7NH+3H+acAPcCmpOdIywM/BXaudl7VBCTpSqrcYtv7FQ0whBC6SufVbqrZVtIbgI9JuoA+HQ9s/7PG+e8FtgZuzsc/Lqlm81ytGtA38t/3Aa8nZTSAQ0jjw4UQwtDVPQnoHNJrNhuRBhbt16CiwKL8ImrvPEKrFCm0agKyfV2+2Fdsv6Nk15WS/lCkgBBC6FpdkoBsnwWcJen7tj9VxyWmSvoBaZDUTwAfI72vVFXRXnCjJW3UO/hofrg0uo4gQwiha3RRExy599vb6znX9jck7UHqoLYpcLLtGbXOK5qAPgtcW/JG7FhgUuXDQwhhCOiiBGR7qaRbJY3JL5LWJOk/gHVs/zknnBl5+zskbWz7/mrnF+0F99v8/s9medPdtl8ucm4IIXSl7uqE0GtdYLakG4AXejdW6XD2beC/y2x/Me97T7XCavWC2yZP0UpOOLdWOyaEEIaU7ktAX+rn8WPLTdlte1aR4dpq1YDOk/ROqo8F9CNS97sQQhhauiwB9XY864cVq+xbqdbJtRLQ6ry2S15fMRVpCGHIEd3XBCdpIcvS6vKkKRVesD2ywik3SvqE7Vf1eJN0OCl3VFWrG/bYmhGHEMJQZFBzJqQbNPqO7SZpf9JMCJUcC1wh6UMsSzg9pOT13lrlFe0FF0IIoa8uqwH1ZfsXkiZX2f8E8FZJuwBb5s2/tv37ItdvWwKSdAaph8Qi4H7go7afyftOBA4HlgBH276qXXGGEEJFXZaAJL2vZHUYqTZT8VtKutn2NravAa6pdky5fe2sAc0ATrS9WNLXgROBEyRtTppWdgtgPeBqSZvYXtLGWEMI4TW67RkQr+42vZg05NqEKse/UdJresGVEKkvQVmFEpAkAR8CNrL9ZUljgNfbvqHI+eXY/l3J6vXAgfnzBOCS3O37QUlzSG2Qf623rBBCaIkuS0C2C4+AnW1W+xAqVh6Kzgf0PWAn0iCkAAuB7xY8t4iPAb/Jn9cHHi3ZNzdvew1JkyTNkjRr/vzojBdCGEC5E0KRpVNIOl3SSEkjJM2UtEDSoZWOz7Om1lrmVjq/aALawfaRwEu50KdJvRxqfZmrJd1RZplQcsxJpKrehb2byn3Pcte3PcV2j+2e0aNjaLoQwgBzwaVz7Gn7OWBf0j/+NwE+36rCij4DekXScPKtlDQaqJnXbe9ebb+kiaQvupvt3v+Z5gIblhy2AfB4wThDCGHAdOEzoBH57z7Axbb/mZ7AtEbRGtBZwBXA2pJOBf4EfK2RgiXtBZwA7JdnXO01DThY0gp51O3xQN3PmkIIoWW6rwZ0paS7Sb3fZubKxkutKqzoYKQXSroJ2I3URLa/7bsaLPtsYAVgRs6w19s+wvZsSVOBO0lNc0dGD7gQwqDTecmlJtuTc6/k52wvkfQi1XvBNaTWYKRrlqw+CVxcuq/ANK0V2f6PKvtOBU6t99ohhNBqovoYZZ0qP+Pv/fwCJaNiN1utGtBNpBwvYAzwdP68BvAIMK5VgYUQwmDXST3cBqOqz4Bsj7O9EXAV8B7bo2yvReo48POBCDCEEAat7nsGNKCK9oLbzvYRvSu2fyPpKy2KKYQQOkMXJhdJryN1/vr3VAu2/9CKsoomoAWSvgD8lHTLDwWeakVAIYTQEbpwRlRJHweOIb3+cguwI2kUml1bUV7RbtiHAKNJXbF/AazNslERQghhaOq+JrhjgO2Ah23vQppstGXDzBTthv3PHFgIIYSsCzshvGT7JUlIWsH23ZI2bVVhRQcjvYYyedx2S6plIYTQCbqtCQ6YK2kNUkvXDElP08KRaIo+A/pcyecVgQNIL4mGEMLQ1HnNazXZ7p3F9Iu54rE68NtWlVe0Ca7v3N5/lnRdC+IJIYTO0WUJCEDS24Dxts/LQ/GsDzzYirKKNsGVjogwDNgWeH0rAgohhE4guq8JTtIppHHgNgXOIw1O+lNg51aUV7QJrnREhMWkbHh4KwIKIYSOMQAJKFcAfgaMJc1QelDpcDklx+0FfAcYDvzQ9ml5+xmkmU4XAfcDH7X9TIXi3kvq+XYzgO3HJa3WxK/zKkW7Yb/R9kZ5ZITxtvcEbmxVUCGEMOgZtNSFlgZNBmbaHg/MzOuvkqfL+S6wN7A5cIikzfPuGcCWtrcC7gVOrFLWojw1Tu/UO6s0Gnw1RRPQX8psiymyQwhDmlxsadAE4Pz8+Xxg/zLHbA/Msf2A7UXAJfk8bP/Odm+nsetJL5lWMlXSD4A1JH0CuBo4t+FvUEGt0bBfT3oAtZKkrVk2+OtIYOVWBRVCCB1hYJ4BrWN7HoDteZLWLnPM+sCjJetzgR3KHPcxUnNeWba/IWkP4DnSc6CTbc+oO/Iaaj0DehfwEVLGPLNk+0Lgv1sUUwghdIR+1G5GSZpVsj7F9pR/X0e6mvIdu04qGkqZba+KTtJJpGf4F1a8SGpy+73tGfkF1E0ljbD9SsE4+qVqArJ9PnC+pANsX96KACR9DjgDGG17Qd52IqmTwxLgaNtXtaLsEEJoSPEEtMB2T8XL2LtX2ifpCUnr5trPuqS52fqaC2xYsr4BJS+QSppImsVgt/yMp5I/AG/PA5JeDcwCPgB8qMo5davVBHeo7Z8CYyUd13e/7TPLnFaYpA2BPUhzC/Vu2xw4GNgCWA+4WtImMStqCGFQGbjBSKcBE4HT8t9fljnmRmC8pHHAY6Tf0A/Cv3vHnQD8p+0Xa5Ql2y9KOhz4X9unS/p7k77Ha9TqhNDbA2JVYLU+y6pNKP9bwPG8+t8RE4BLbL9s+0FgDukBWwghDBoijQVXZGnQacAeku4j/YO9t3v1epKmA+ROBkeR5m67C5hqe3Y+/2zSb/YMSbdIOqfa15K0E6nG8+u8rejrOv1WqwnuB/nj1bb/XLpPUkMvJknaD3jM9q3Sq5ov1yf11Og1N28rd41JwCSAMWPGNBJOCCH0X9XWrGYV4aeA3cpsfxzYp2R9OjC9zHH/0Y/ijiF1077C9mxJGwHX9Dvogopmtv8Ftimw7VVqPFj7b2DPcqeV2Vb2f+X8EG8KQE9PT5e9kxxCGOy6bSSEPPHcH0rWHwCOblV5tZ4B7QS8FRjd5xnQSNLbtlVVerAm6U3AOKC39rMBcLOk7anxMC2EEAaFLhyMdKDVqgEtT3rWsxypDbHXc8CB9RZq+3bSpHYASHoI6LG9QNI04CJJZ5I6IYwHbqi3rBBCaJUunA9oQNV6BnQdcJ2k/7P98EAElNsdpwJ3kvqsHxk94EIIg1G3JSBJa+YJSAdE0WdAL+YB7bYgzQcENG9COttj+6yfCpzajGuHEEJLmAHphDDA/ibpFtJI2L+p8c5Qw4qOBXchcDfpuc2XSCOyxmCkIYQhbYDGghtIm5A6dh0GzJH0NUmbtKqwogloLds/Al6xfZ3tjwE7tiqoEELoCC64dAgnM2wfAnyc9OLrDZKuy53SmqpoE1zvOEDzJL2b1Cut2oiqIYTQ1bp0Qrq1gENJNaAngM+QRmJ4C3ApqRWsaYomoK9KWh34L9L7PyOBY5sZSAghdBS7G58B/RX4CbC/7bkl22fVGEGhLoUSkO1f5Y/PArsASDq22cGEEEIn6bZecMCmlToe2P56swtrZIyf44BvNymOEELoON3SBCfpSpbNgvqa/bb3a0W5jSSgckPmhBDC0GCg8em2B4tvtKPQRhJQ19z5EEKoS5f8CuZBBwZcrbHgFlL+FgtYqSURhRBCh+iiJriptg+SdDtlfvNtb9WKcmsNxbNatf0hhDCkdU8vuGPy330HstCWTTQUQghdzd3TC872vPx3QMb87FV0JIQQQggl0ouoLrR0Ckk7SrpR0vOSFklaIum5VpUXNaAQQqhXl9SASpwNHEwa9aAH+DDQnxlV+yUSUAgh1KmTajdF2Z4jaXieBuc8SX9pVVmRgEIIoR4dNtBoQS9KWh64RdLpwDxglVYV1tZnQJI+I+keSbPzl+3dfqKkOXnfu9oZYwghlGe0tNjSQQ4j5YWjgBeADYEDWlVY22pAknYBJgBb2X5Z0tp5++akNsgtSFNyXy1pk5gVNYQw6HRZE5zthyWNzp+/1Ory2lkD+hRwmu2XAWw/mbdPAC6x/bLtB4E5wPZtijGEEMrL3bCLLIOdki9KWkCafPReSfMlndzKctuZgDYB3i7pb3myo+3y9vWBR0uOm5u3hRDC4NI7JUOtZfA7FtgZ2M72WrZfB+wA7Czps60qtKVNcJKuBl5fZtdJuezXkWZW3Q6YKmkjyg9yWvZ/QUmTgEkAY8aMaUbIIYRQXEfklkI+DOxhe0HvBtsPSDoU+B3wrVYU2tIEZHv3SvskfQr4eZ574gZJS4FRpBrPhiWHbkCagbXc9aeQ5i+np6ene/5TCCF0hC7qhj2iNPn0sj1f0ohWFdrOJrhfALsCSNoEWB5YQJr+9WBJK0gaB4wHbmhXkCGEUJaBJS62DH6L6tzXkHa+B/Rj4MeS7iB9wYm5NjRb0lTgTmAxcGT0gAshDDais4bZqeHNFYbcEbBiqwptWwKyvQg4tMK+U4FTBzaiEELopy5JQLaHt6PcGAkhhBDq1SUJqF0iAYUQQj1MNw5GOqAiAYUQQp266BlQW0QCCiGEuhiWRhWoEZGAQgihHiaeATUoElAIIdQrKkANiSm5QwihTgMxJbekNSXNkHRf/vu6CsftlaewmSNpcpn9n5NkSaMaCqiJIgGFEEK9BmYw0snATNvjgZl5/VUkDQe+C+wNbA4ckqe26d2/IbAH8EijwTRTJKAQQqiHDUuWFlsaMwE4P38+H9i/zDHbA3NsP5Bf8r8kn9frW8DxDLLhUyMBhRBCvYrXgEZJmlWyTOpHKevYnpeK8zxg7TLHVJzGRtJ+wGO2b63rO7ZQdEIIIYR6FW9eW2C7p9LOGlPXFFF2GhtJK+dr7FnwOgMqElAIIdTDwNLmtGjVmLrmCUnr2p4naV3gyTKHVZrGZmNgHHCrpN7tN0va3vY/mhJ8A6IJLoQQ6mLw0mJLY6YBE/PnicAvyxxzIzBe0jhJywMHA9Ns3257bdtjbY8lJaptBkPygUhAIYRQv4HpBXcasIek+0g92U4DkLSepOkpDC8GjgKuAu4Cptqe3WjBrRZNcCGEUA/TjB5utYuxnwJ2K7P9cWCfkvXpwPQa1xrb7PgaEQkohBDqFUPxNKRtTXCS3iLpekm35G6J25fsOzG/zXuPpHe1K8YQQqisYPNbJKmK2lkDOh34ku3fSNonr78zv717MLAFsB5wtaRNYlruEMKgYmI07Aa1sxOCgZH58+qkLoOQ3t69xPbLth8E5pDe8g0hhMElakANaWcN6FjgKknfICXCt+bt6wPXlxz37zd6+8pvE08CGDNmTMsCDSGEsiK5NKSlCajG2727AZ+1fbmkg4AfAbtT4Y3ecte3PQWYAtDT0xP/JYQQBo6Nl8STgUa0NAHVeLv3AuCYvHop8MP8udIbvSGEMLg0aSSEoaqdz4AeB/4zf94VuC9/ngYcLGkFSeOA8cANbYgvhBCqi2dADWnnM6BPAN+RtBzwEvlZju3ZkqYCdwKLgSOjB1wIYdCxoxdcg9qWgGz/Cdi2wr5TgVMHNqIQQuinqN00JEZCCCGEukQnhEZFAgohhHo0cTqGoSoSUAgh1KvxqRaGtEhAIYRQBwOOGlBDIgGFEEI97KgBNSgSUAgh1ClqQI2Ru6QboaSFwD3tjqOPUcCCdgdRxmCMK2IqJmIqrlpcb7A9upGLS/ptLqOIBbb3aqS8btRNCWiW7Z52x1FqMMYEgzOuiKmYiKm4wRpXWKadQ/GEEEIYwiIBhRBCaItuSkBT2h1AGYMxJhiccUVMxURMxQ3WuELWNc+AQgghdJZuqgGFEELoIJGAQgghtEVXJCBJe0m6R9IcSZPbGMdDkm6XdIukWXnbmpJmSLov/31di2P4saQnJd1Rsq1iDJJOzPftHknvGsCYvijpsXyvbpG0zwDHtKGkayTdJWm2pGPy9rbdqyoxtfterSjpBkm35ri+lLe3815Viqmt9yr0k+2OXoDhwP3ARsDywK3A5m2K5SFgVJ9tpwOT8+fJwNdbHMM7gG2AO2rFAGye79cKwLh8H4cPUExfBD5X5tiBimldYJv8eTXg3lx22+5VlZjafa8ErJo/jwD+BuzY5ntVKaa23qtY+rd0Qw1oe2CO7QdsLwIuASa0OaZSE4Dz8+fzgf1bWZjtPwD/LBjDBOAS2y/bfhCYQ7qfAxFTJQMV0zzbN+fPC4G7gPVp472qElMlA3WvbPv5vDoiL6a996pSTJUMyL0K/dMNCWh94NGS9blU/z9tKxn4naSbJE3K29axPQ/SDwywdhviqhRDu+/dUZJuy010vc03Ax6TpLHA1qR/RQ+Ke9UnJmjzvZI0XNItwJPADNttv1cVYoJB8t9VqK0bEpDKbGtX3/KdbW8D7A0cKekdbYqjqHbeu+8DGwNvAeYB32xHTJJWBS4HjrX9XLVDy2xrSVxlYmr7vbK9xPZbgA2A7SVtWeXwAYmrQkxtv1ehuG5IQHOBDUvWNwAeb0cgth/Pf58EriBV8Z+QtC5A/vtkG0KrFEPb7p3tJ/IPyFLgXJY1hwxYTJJGkH7oL7T987y5rfeqXEyD4V71sv0McC2wF4Pkv6vSmAbTvQq1dUMCuhEYL2mcpOWBg4FpAx2EpFUkrdb7GdgTuCPHMjEfNhH45UDHViWGacDBklaQNA4YD9wwEAH1/nBl7yXdqwGLSZKAHwF32T6zZFfb7lWlmAbBvRotaY38eSVgd+Bu2nuvysbU7nsV+qndvSCasQD7kHoM3Q+c1KYYNiL1srkVmN0bB7AWMBO4L/9ds8VxXExqeniF9K++w6vFAJyU79s9wN4DGNNPgNuB20g/DusOcExvIzXB3Abckpd92nmvqsTU7nu1FfD3XP4dwMm1/tsegHtVKaa23qtY+rfEUDwhhBDaohua4EIIIXSgSEAhhBDaIhJQCCGEtogEFEIIoS0iAYUQQmiLSEAhhBDaIhLQECDp+dpHNXT96ZLWyMun6zj/nZJ+1c/jn5U0vcL+/5N0YH/j6ET5Xry1ZP2zkh6RdHY74wqhiEhAoWG293EaDmUNoN8JqE5/tL1P7cPqJ2m5Vl6/Sd4J/DsB2f4WcHLbogmhHyIBDVGS3iLp+jxq8BW9owZLulbS1/NkX/dKenvevrKkqfn4n0n6m6SevO8hSaOA04CN80RgZ/St2Ug6W9JH8ue9JN0t6U/A+0qOWSWPYnyjpL9Lqjm1hpKzJd0p6deUjDguaVtJ1+URyq8qGbtsu/xd/ppjvSNv/4ikSyVdSRrZvGw8SiMxn5G33ybpk3n7upL+kO/BHb33r0Lce+byb85lrpq3n5yve4ekKXmIHiQdnb/jbZIuURox+wjgs7m8imWFMBhFAhq6LgBOsL0VaeiSU0r2LWd7e+DYku2fBp7Ox38F2LbMNScD99t+i+3PVypY0oqkgSLfA7wdeH3J7pOA39veDtgFOENpbL1q3gtsCrwJ+AS5RqA0sOf/Agfa3hb4MXBqPuc84AjbOwFL+lxvJ2Ci7V2rxHM48Gzevh3wiTzG2AeBq5xGaX4zaTidcvdgFPAFYHenEdRnAcfl3Wfb3s72lsBKwL55+2Rg6/y/wRG2HwLOAb6V7/kfa9ynEAaVTmhiCE0maXVgDdvX5U3nA5eWHNI7MvRNwNj8+W3AdwBs3yHptgZC2Ax40PZ9OZ6fAr3zJ+0J7Cfpc3l9RWAMaXK2St4BXGx7CfC4pN/n7ZsCWwIzciViODAvD2K5mu2/5OMuYtmPPKS5ZXon0KsUz57AViXPmlYnDXB5I/DjnPx+YfuWCjHvSJql8885tuWBv+Z9u0g6HlgZWJM0tuCVpPHNLpT0C+AXVe5HCB0hElAo5+X8dwnL/hspN59KLYt5dS17xZLPlQYhFHCA7Xv6WVa56wmYnWs5yzYum6SskhdqxZObxT5j+6rXFJrmgXo38BNJZ9i+oEJsM2wf0ufcFYHvAT22H5X0RZbdt3eTku1+wP+TtEWN7xHCoBZNcEOQ7WeBp0ueGRwGXFflFIA/AQcBSNqc1NzV10JgtZL1h4HNlYbAXx3YLW+/GxgnaeO8XvojfBXwmZLnHlsX+Ep/IA21Pzw/49klb78HGC1pp3ytEZK2sP00sFDSjvm4g6tcu1I8VwGfyjUdJG2Snxe9AXjS9rmkqRW2qXDd64GdJf1HPn9lSZuwLNksyM+EDsz7hwEb2r4GOJ7U4WNVXnvPQ+gYUQMaGlaWNLdk/UzS/C3nSFoZeAD4aI1rfA84Pze99Q6D/2zpAbafkvTn/ED/N7Y/L2lqPva+fB62X1KasvzXkhaQklvvDJtfAb4N3JZ/9B/i1c1j5VwB7Ep6lnUvOZnaXpSbyM7KCXC5fO3ZpGc450p6gTSZ2bOvvWzVeH5Iap68OW+fD+xP6pX2eUmvAM8DHy53UdvzlTpkXCxphbz5C7bvlXRu/i4PkZr0IDUf/jR/D5Ge+zyTO0tcljtHfCaeA4VOEtMxhEIkDQdG5OSxMWn+l01sL2pDLO8EPme7VmKqdo1VbT+fP08mzRtzTHMibK+c2HpsH9XuWEKoJmpAoaiVgWtyk5OAT7Uj+WSLgC0lTW/gXaB3SzqR9P+Bh4GPNCu4dpL0WVLX7MvbHUsItUQNKIQWk/Q3YIU+mw+zfXs74glhsIgEFEIIoS2iF1wIIYS2iAQUQgihLSIBhRBCaItIQCGEENri/wOECWKdlFUSUAAAAABJRU5ErkJggg==\n", + "text/plain": [ + "
" + ] + }, + "metadata": { + "needs_background": "light" + }, + "output_type": "display_data" + } + ], + "source": [ + "ds.sst[-1].plot()" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Postscript: Execute the full recipe\n", + "\n", + "We are now confident that our recipe works as we expect.\n", + "At this point we could either:\n", + "- Execute it all ourselves (see {doc}`execution`)\n", + "- Create a new recipe feedstock on Pangeo Forge\n", + "\n", + "If we wanted to execute it ourselves, one way would be to simply run the following code\n", + "```python\n", + "for input_name in recipe.iter_inputs():\n", + " recipe.cache_input(input_name)\n", + "recipe.prepare_target()\n", + "for chunk in recipe.iter_chunks():\n", + " recipe.store_chunk(chunk)\n", + "recipe.finalize_target()\n", + "```\n", + "\n", + "We aren't going to do this in this notebook because it would take too long.\n", + "\n", + "But hopefully now you have a better understanding of how Pangeo Forge recipes work." + ] } ], "metadata": { diff --git a/docs/recipes.md b/docs/recipes.md index a45789fc..4328f13b 100644 --- a/docs/recipes.md +++ b/docs/recipes.md @@ -12,7 +12,7 @@ you are reading matches your installed version of pangeo_forge. Recipes need a place to store data. The location where the final dataset produced by the recipe is stored is called the -``Target``. Pangeo forge has a special class for this: {class}`pangeo_forge.storage.Target` +``Target``. Pangeo forge has a special class for this: {class}`pangeo_forge.storage.FSSpecTarget` Creating a Target requires two arguments: - The ``fs`` argument is an [fsspec](https://filesystem-spec.readthedocs.io/en/latest/) @@ -20,20 +20,17 @@ Creating a Target requires two arguments: [built in](https://filesystem-spec.readthedocs.io/en/latest/api.html#built-in-implementations) and [third party](https://filesystem-spec.readthedocs.io/en/latest/api.html#other-known-implementations) implementations. -- The `path` argument specifies the specific path where the data should be stored. +- The `root_path` argument specifies the specific path where the data should be stored. For example, creating a storage target for AWS S3 might look like this: ```{code-block} python import s3fs fs = s3fs.S3FileSystem(key="MY_AWS_KEY", secret="MY_AWS_SECRET") target_path = "pangeo-forge-bucket/my-dataset-v1.zarr" -target = Target(fs=fs, path=target_path) +target = FSSpecTarget(fs=fs, root_path=target_path) ``` -Temporary data can be stored in an {class}`pangeo_forge.storage.InputCache` object. -``InputCache`` is similar to ``Target``, but instead of specifying a ``path``, -you specify ``prefix``. - +Temporary data is recommended to use a {class}`pangeo_forge.storage.CacheFSSpecTarget` object. ## The Base Recipe Class @@ -45,9 +42,6 @@ recipe = Recipe(option1='foo', option2=) All recipes follow the same basic steps. - - - ## Specific Recipe Classes ```{eval-rst} diff --git a/pangeo_forge/recipe.py b/pangeo_forge/recipe.py index ebe82dda..55cb88cf 100644 --- a/pangeo_forge/recipe.py +++ b/pangeo_forge/recipe.py @@ -138,7 +138,9 @@ class NetCDFtoZarrSequentialRecipe(BaseRecipe): :param consolidate_zarr: Whether to consolidate the resulting Zarr dataset. :param xarray_open_kwargs: Extra options for opening the inputs with Xarray. :param xarray_concat_kwargs: Extra options to pass to Xarray when concatenating - the inputs to form a chunk. + the inputs to form a chunk. + :param delete_input_encoding: Whether to remove Xarray encoding from variables + in the input dataset """ input_urls: Iterable[str] = field(repr=False) @@ -151,6 +153,7 @@ class NetCDFtoZarrSequentialRecipe(BaseRecipe): consolidate_zarr: bool = True xarray_open_kwargs: dict = field(default_factory=dict) xarray_concat_kwargs: dict = field(default_factory=dict) + delete_input_encoding: bool = True def __post_init__(self): self._chunks_inputs = { @@ -245,6 +248,11 @@ def open_input(self, fname: str): # explicitly load into memory ds = ds.load() ds = fix_scalar_attr_encoding(ds) + + if self.delete_input_encoding: + for var in ds.variables: + ds[var].encoding = {} + logger.debug(f"{ds}") return ds @@ -255,8 +263,6 @@ def open_chunk(self, chunk_key): # CONCAT DELETES ENCODING!!! # OR NO IT DOESN'T! Not in the latest version of xarray? ds = xr.concat(dsets, self.sequence_dim, **self.xarray_concat_kwargs) - for var in ds.variables: - ds[var].encoding = {} logger.debug(f"{ds}") # TODO: maybe do some chunking here? From 63e2297366584702853368fbe8918a2e3c7890ee Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 23:46:39 -0500 Subject: [PATCH 31/34] last commit of the night --- docs/_static/custom.css | 27 +++++++ docs/api.md | 24 ++++++ docs/execution.md | 81 +++++++++++++++++++ docs/index.md | 5 +- docs/recipes.md | 41 +++++++--- docs/tutorials/index.md | 9 +++ .../netcdf_zarr_sequential.ipynb} | 17 +++- pangeo_forge/executors.py | 3 + tests/test_executors.py | 3 +- 9 files changed, 194 insertions(+), 16 deletions(-) create mode 100644 docs/_static/custom.css create mode 100644 docs/tutorials/index.md rename docs/{recipe_tutorial.ipynb => tutorials/netcdf_zarr_sequential.ipynb} (99%) create mode 100644 pangeo_forge/executors.py diff --git a/docs/_static/custom.css b/docs/_static/custom.css new file mode 100644 index 00000000..9b2ac1b6 --- /dev/null +++ b/docs/_static/custom.css @@ -0,0 +1,27 @@ +/* Put your custom CSS here */ +@import url('http://fonts.cdnfonts.com/css/panton-black-caps'); + +h1 { + font-family: "Panton Black Caps", sans-serif; + color: #003B71 !important; +} + +h2 { + font-family: "Panton Light Caps", sans-serif; + color: #003B71 !important; +} + +a { + color: #5eb130 !important; +} + + + +/* Fixing up some pygments and code-styling CSS for accessibility */ +code { font-size: 100%; color: #e50051; } +pre { font-family: monospace; } + +/* .highlight { font-size: 125%; } */ +.highlight .c1 { color: #e50051; } +.highlight .si { color: #e50051; } +.highlight .nn { color: #e50051; } diff --git a/docs/api.md b/docs/api.md index 667283b2..1ba199ae 100644 --- a/docs/api.md +++ b/docs/api.md @@ -8,9 +8,16 @@ :members: ``` +```{eval-rst} +.. autoclass:: pangeo_forge.storage.FlatFSSpecTarget + :members: + :show-inheritance: +``` + ```{eval-rst} .. autoclass:: pangeo_forge.storage.CacheFSSpecTarget :members: + :show-inheritance: ``` ## Recipes @@ -24,3 +31,20 @@ .. autoclass:: pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe :show-inheritance: ``` + +## Excutors + +```{eval-rst} +.. autoclass:: pangeo_forge.executors.PythonPipelineExecutor + :members: +``` + +```{eval-rst} +.. autoclass:: pangeo_forge.executors.DaskPipelineExecutor + :members: +``` + +```{eval-rst} +.. autoclass:: pangeo_forge.executors.PrefectPipelineExecutor + :members: +``` diff --git a/docs/execution.md b/docs/execution.md index 7bb5b258..0ec71286 100644 --- a/docs/execution.md +++ b/docs/execution.md @@ -1 +1,82 @@ # Recipe Execution + +There are many different types of Pangeo Forge recipes. +However, **all recipes are executed the same way**! +This is a key part of the Pangeo Forge design. + +Once you have created a recipe object (see {doc}`recipes`) you have two +options for executing it. In the subsequent code, we will assume that a +recipe has already been initialized in the variable `recipe`. + +## Manual Execution + +A recipe can be executed manually, step by step, in serial, from a notebook +or interactive interpreter. The ability to manually step through a recipe +is very important for developing and debugging complex recipes. +There are four stages of recipe execution. + +### Stage 1: Cache Inputs + +Recipes may define files that have to be cached locally before the subsequent +steps can proceed. The common use case here is for files that have to be +extracted from a slow FTP server. Here is how to cache the inputs. + +```{code-block} python +for input_name in recipe.iter_inputs(): + recipe.cache_input(input_name) +``` + +If the recipe doesn't do input caching, nothing will happen here. + +### Stage 2: Prepare Target + +Once the inputs have been cached, we can get the target ready. +Preparing the target for writing is done as follows: + +```{code-block} python +recipe.prepare_target() +``` + +For example, for Zarr targets, this sets up the Zarr group with the necessary +arrays and metadata. + +### Stage 3: Store Chunks + +This is the step where the bulk of the work happens. + +```{code-block} python +for chunk in recipe.iter_chunks(): + recipe.store_chunk(chunk) +``` + +### Stage 4: Finalize Target + +If there is any cleanup or consolidation to be done, it happens here. + +```{code-block} python +recipe.finalize_target() +``` + +For example, consolidating Zarr metadta happens in the finalize step. + +## Execution by Executors + +Very large recipes cannot feasibly be executed this way. +To support distributed parallel execution, Pangeo Forge borrows the +[Executors framework from Rechunker](https://rechunker.readthedocs.io/en/latest/executors.html). + +There are currently three executors implemented. +- {class}`pangeo_forge.executors.PythonPipelineExecutor`: a reference executor + using simple python +- {class}`pangeo_forge.executors.DaskPipelineExecutor`: distributed executor using Dask +- {class}`pangeo_forge.executors.PrefectPipelineExecutor`: distributed executor using Prefect + +To use an executor, the recipe must first be transformed into a `Pipeline` object. +The full process looks like this: + +```{code-block} python +pipeline = recipe.to_pipelines() +executor = PrefectPipelineExecutor() +plan = executor.pipelines_to_plan(pipeline) +executor.execute_plan(plan) # actually runs the recipe +``` diff --git a/docs/index.md b/docs/index.md index 319199e4..7e475018 100644 --- a/docs/index.md +++ b/docs/index.md @@ -14,7 +14,8 @@ The most important concept in Pangeo Forge is a ``recipe``. A recipe defines how to transform data in one format / location into another format / location. The primary way people contribute to Pangeo Forge is by writing / maintaining recipes. Recipes developed by the community are stored in GitHub repositories. -For information about how to write a recipe, see {doc}`recipes`. +For information about how recipes work see {doc}`recipes`. +The {doc}`tutorials/index` provide deep dives into how to develop and debug Pangeo Forge recipes. ## Recipe Execution @@ -35,7 +36,7 @@ For more information, see {doc}`bakeries`. :caption: Contents recipes -recipe_tutorial +tutorials/index execution bakeries contribute diff --git a/docs/recipes.md b/docs/recipes.md index 4328f13b..527df8b8 100644 --- a/docs/recipes.md +++ b/docs/recipes.md @@ -32,20 +32,43 @@ target = FSSpecTarget(fs=fs, root_path=target_path) Temporary data is recommended to use a {class}`pangeo_forge.storage.CacheFSSpecTarget` object. -## The Base Recipe Class +## The Recipe Object + +You define a recipe by instantiating a class that inherits from {class}`pangeo_forge.recipe.BaseRecipe`. +The `pangeo_forge` package includes several pre-defined Recipe classes which +cover common scenarios. You can also define your own Recipe class. + +For a the common scenario of assembling many NetCDF files into a single Zarr +group, we use {class}`pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe`. +Initializing a recipe looks something like this. -A recipe is initialized from a recipe class. ```{code-block} python -recipe = Recipe(option1='foo', option2=) +from pangeo_forge.recipes import NetCDFtoZarrSequentialRecipe +input_urls = [...] # build a list of inputs +recipe = NetCDFtoZarrSequentialRecipe( + input_urls=input_urls, + sequence_dim="time" +) ``` -All recipes follow the same basic steps. +There are many other options we can pass, all covered in the [API documentation](api). +For a deeper dive on how to pick these options and what they mean, check out the +tutorial: {doc}`tutorials/netcdf_zarr_sequential`. +Your recipe will also need storage. +If you have already defined a `Target` object (as in the the [Storage section](#storage)), +then you can either assign it when you initialize the recipe or later, e.g. -## Specific Recipe Classes +```{code-block} python +recipe.target = FSSpecTarget(fs=fs, root_path=target_path) +``` -```{eval-rst} -.. autoclass:: pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe - :show-inheritance: - :noindex: +This particular class of recipe also requires a cache, a place to store temporary +files. We can create one as follows. + +```{code-block} python +recipe.input_cache = CacheFSSpecTarget(fs=fs, root_path=cache_path) ``` + +Once your recipe is defined and has its targets assigned, you're ready to +move on to {doc}`execution`. diff --git a/docs/tutorials/index.md b/docs/tutorials/index.md new file mode 100644 index 00000000..ef34442c --- /dev/null +++ b/docs/tutorials/index.md @@ -0,0 +1,9 @@ +# Recipe Tutorials + +These tutorials are deep dives into how to develop and debug Pangeo Forge recipes. + +```{toctree} +:maxdepth: 1 + +netcdf_zarr_sequential +``` diff --git a/docs/recipe_tutorial.ipynb b/docs/tutorials/netcdf_zarr_sequential.ipynb similarity index 99% rename from docs/recipe_tutorial.ipynb rename to docs/tutorials/netcdf_zarr_sequential.ipynb index 3e7b2cfe..4d501d87 100644 --- a/docs/recipe_tutorial.ipynb +++ b/docs/tutorials/netcdf_zarr_sequential.ipynb @@ -4,9 +4,11 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "# Recipe Tutorial\n", + "# NetCDF Zarr Sequential Recipe\n", "\n", - "This tutorial describes how to create a recipe from scratch.\n" + "This tutorial describes how to create a recipe from scratch.\n", + "The source data is a sequence of NetCDF files accessed via HTTP.\n", + "The target is a Zarr store." ] }, { @@ -567,7 +569,7 @@ "For our first recipe, we will want to use a pre-defined Recipe class from Pangeo\n", "Forge.\n", "\n", - "By examining the {doc}`recipes` documentation page, we see that our scenario is\n", + "By examining the {doc}`../recipes` documentation page, we see that our scenario is\n", "a good case for the {class}`pangeo_forge.recipe.NetCDFtoZarrSequentialRecipe`\n", "class. Let's examine its documentation string in our notebook.\n" ] @@ -3602,7 +3604,7 @@ "\n", "We are now confident that our recipe works as we expect.\n", "At this point we could either:\n", - "- Execute it all ourselves (see {doc}`execution`)\n", + "- Execute it all ourselves (see {doc}`../execution`)\n", "- Create a new recipe feedstock on Pangeo Forge\n", "\n", "If we wanted to execute it ourselves, one way would be to simply run the following code\n", @@ -3619,6 +3621,13 @@ "\n", "But hopefully now you have a better understanding of how Pangeo Forge recipes work." ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [] } ], "metadata": { diff --git a/pangeo_forge/executors.py b/pangeo_forge/executors.py new file mode 100644 index 00000000..bca10b1b --- /dev/null +++ b/pangeo_forge/executors.py @@ -0,0 +1,3 @@ +from rechunker.executors import DaskPipelineExecutor # noqa: F401 +from rechunker.executors import PrefectPipelineExecutor # noqa: F401 +from rechunker.executors import PythonPipelineExecutor # noqa: F401 diff --git a/tests/test_executors.py b/tests/test_executors.py index b21ad56a..6294809a 100644 --- a/tests/test_executors.py +++ b/tests/test_executors.py @@ -1,6 +1,7 @@ import pytest import xarray as xr -from rechunker.executors import ( + +from pangeo_forge.executors import ( DaskPipelineExecutor, PrefectPipelineExecutor, PythonPipelineExecutor, From e0c97b009555d9d8fcdeeef5264acee78b993256 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 23:48:27 -0500 Subject: [PATCH 32/34] update doc requirements --- docs/requirements.txt | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/requirements.txt b/docs/requirements.txt index 8a7fbe2f..88595554 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -1,3 +1,5 @@ sphinx -sphinx-pangeo-theme +sphinx_book_theme myst-parser +myst-nb +sphinx-copybutton From 382663d4f6b963fd02553f00c94956f87aa35be8 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Thu, 21 Jan 2021 23:57:40 -0500 Subject: [PATCH 33/34] use rechunker from github --- requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/requirements.txt b/requirements.txt index 83520897..b8770e9b 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,7 +2,7 @@ setuptools click dask distributed -rechunker +-e git://github.com/rabernat/rechunker.git@refactor-executors#egg=rechunker xarray >= 0.16.2 zarr >= 2.6.0 fsspec[http] From 57304e86ff31e5e4c56cc8dd5f27f79a8e9b47f4 Mon Sep 17 00:00:00 2001 From: Ryan Abernathey Date: Fri, 22 Jan 2021 07:34:20 -0500 Subject: [PATCH 34/34] fix requirements --- docs/requirements.txt | 1 + requirements.txt | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/requirements.txt b/docs/requirements.txt index 88595554..51875dbb 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -1,4 +1,5 @@ sphinx +pydata-sphinx-theme>=0.4.2 sphinx_book_theme myst-parser myst-nb diff --git a/requirements.txt b/requirements.txt index b8770e9b..83520897 100644 --- a/requirements.txt +++ b/requirements.txt @@ -2,7 +2,7 @@ setuptools click dask distributed --e git://github.com/rabernat/rechunker.git@refactor-executors#egg=rechunker +rechunker xarray >= 0.16.2 zarr >= 2.6.0 fsspec[http]