diff --git a/analysis/webservice/algorithms/DataInBoundsSearch.py b/analysis/webservice/algorithms/DataInBoundsSearch.py index dcac93b4..6e90952d 100644 --- a/analysis/webservice/algorithms/DataInBoundsSearch.py +++ b/analysis/webservice/algorithms/DataInBoundsSearch.py @@ -29,6 +29,7 @@ EPOCH = timezone('UTC').localize(datetime(1970, 1, 1)) ISO_8601 = '%Y-%m-%dT%H:%M:%S%z' +logger = logging.getLogger(__name__) @nexus_handler @@ -157,6 +158,8 @@ def calc(self, computeOptions, **args): tiles = self._get_tile_service().get_tiles_by_metadata(metadata_filter, ds, start_time, end_time) need_to_fetch = False + logger.info(f'Matched {len(tiles)} tiles') + data = [] log.info(f'Matched {len(tiles):,} tiles.') diff --git a/analysis/webservice/nexus_tornado/request/handlers/NexusRequestHandler.py b/analysis/webservice/nexus_tornado/request/handlers/NexusRequestHandler.py index b95716a7..234998d8 100644 --- a/analysis/webservice/nexus_tornado/request/handlers/NexusRequestHandler.py +++ b/analysis/webservice/nexus_tornado/request/handlers/NexusRequestHandler.py @@ -22,6 +22,9 @@ from webservice.webmodel import NexusRequestObjectTornadoFree, NexusRequestObject, NexusProcessingException from webservice.algorithms_spark.NexusCalcSparkTornadoHandler import NexusCalcSparkTornadoHandler +from nexustiles.exception import AlgorithmUnsupportedForDatasetException + +from py4j.protocol import Py4JJavaError class NexusRequestHandler(tornado.web.RequestHandler): def initialize(self, thread_pool, clazz=None, **kargs): @@ -72,6 +75,56 @@ def get(self): except NexusProcessingException as e: self.async_onerror_callback(e.reason, e.code) + # except pyspark + + except AlgorithmUnsupportedForDatasetException as e: + self.logger.exception(e) + self.async_onerror_callback( + reason='Algorithm unsupported for dataset (backend has yet to implement functionality)', + code=400 + ) + + except Py4JJavaError as e: + self.logger.exception(e) + + if 'AlgorithmUnsupportedForDatasetException' in str(e): + self.async_onerror_callback( + reason='Algorithm unsupported for dataset (backend has yet to implement functionality)', + code=400 + ) + else: + self.async_onerror_callback(str(e), 500) + + except Exception as e: + print(type(e)) + self.async_onerror_callback(str(e), 500) + + @tornado.gen.coroutine + def post(self): + self.logger.info("Received %s" % self._request_summary()) + + request = NexusRequestObject(self) + + # create NexusCalcHandler which will process the request + instance = self.__clazz(**self._clazz_init_args) + + try: + # process the request asynchronously on a different thread, + # the current tornado handler is still available to get other user requests + results = yield tornado.ioloop.IOLoop.current().run_in_executor(self.executor, instance.calc, request) + + if results: + try: + self.set_status(results.status_code) + except AttributeError: + pass + + renderer = NexusRendererFactory.get_renderer(request) + renderer.render(self, results) + + except NexusProcessingException as e: + self.async_onerror_callback(e.reason, e.code) + except Exception as e: self.async_onerror_callback(str(e), 500) diff --git a/data-access/nexustiles/backends/__init__.py b/data-access/nexustiles/backends/__init__.py index 8afd240a..1bee2ee8 100644 --- a/data-access/nexustiles/backends/__init__.py +++ b/data-access/nexustiles/backends/__init__.py @@ -13,3 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. +from nexustiles.backends.cog.backend import CoGBackend +from nexustiles.backends.zarr.backend import ZarrBackend +from nexustiles.backends.nexusproto.backend import NexusprotoTileService \ No newline at end of file diff --git a/data-access/nexustiles/backends/backend.py.template b/data-access/nexustiles/backends/backend.py.template new file mode 100644 index 00000000..84cffa28 --- /dev/null +++ b/data-access/nexustiles/backends/backend.py.template @@ -0,0 +1,284 @@ +############################################################################ +### ### +### THIS IS A TEMPLATE FOR STARTING A NEW NTS BACKEND IMPLEMENTATION ### +### ### +############################################################################ + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging +import sys +from datetime import datetime +from urllib.parse import urlparse + +import numpy as np +import numpy.ma as ma +import s3fs +import xarray as xr +from nexustiles.AbstractTileService import AbstractTileService +from nexustiles.exception import NexusTileServiceException +from nexustiles.model.nexusmodel import Tile, BBox, TileVariable +from pytz import timezone +from shapely.geometry import MultiPolygon, box +from yarl import URL + +EPOCH = timezone('UTC').localize(datetime(1970, 1, 1)) +ISO_8601 = '%Y-%m-%dT%H:%M:%S%z' + +logging.basicConfig( + level=logging.INFO, + format='%(asctime)s - %(name)s - %(levelname)s - %(message)s', + datefmt="%Y-%m-%dT%H:%M:%S", stream=sys.stdout) +logger = logging.getLogger(__name__) + + +class ZarrBackend(AbstractTileService): + def __init__(self, dataset_name, path, config=None): + AbstractTileService.__init__(self, dataset_name) + self.__config = config if config is not None else {} + + logger.info(f'Opening zarr backend at {path} for dataset {self._name}') + + url = urlparse(path) + + self.__url = path + + self.__store_type = url.scheme + self.__host = url.netloc + self.__path = url.path + + if 'variable' in config: + data_vars = config['variable'] + elif 'variables' in config: + data_vars = config['variables'] + else: + raise KeyError('Data variables not provided in config') + + if isinstance(data_vars, str): + self.__variables = [data_vars] + elif isinstance(data_vars, list): + self.__variables = data_vars + else: + raise TypeError(f'Improper type for variables config: {type(data_vars)}') + + self.__longitude = config['coords']['longitude'] + self.__latitude = config['coords']['latitude'] + self.__time = config['coords']['time'] + + self.__depth = config['coords'].get('depth') + + if self.__store_type in ['', 'file']: + store = self.__path + elif self.__store_type == 's3': + try: + aws_cfg = self.__config['aws'] + + if aws_cfg['public']: + # region = aws_cfg.get('region', 'us-west-2') + # store = f'https://{self.__host}.s3.{region}.amazonaws.com{self.__path}' + s3 = s3fs.S3FileSystem(True) + store = s3fs.S3Map(root=path, s3=s3, check=False) + else: + s3 = s3fs.S3FileSystem(False, key=aws_cfg['accessKeyID'], secret=aws_cfg['secretAccessKey']) + store = s3fs.S3Map(root=path, s3=s3, check=False) + except Exception as e: + logger.error(f'Failed to open zarr dataset at {self.__path}, ignoring it. Cause: {e}') + raise NexusTileServiceException(f'Cannot open S3 dataset ({e})') + else: + raise ValueError(self.__store_type) + + try: + self.__ds: xr.Dataset = xr.open_zarr(store, consolidated=True) + except Exception as e: + logger.error(f'Failed to open zarr dataset at {self.__path}, ignoring it. Cause: {e}') + raise NexusTileServiceException(f'Cannot open dataset ({e})') + + def get_dataseries_list(self, simple=False): + raise NotImplementedError() + + def find_tile_by_id(self, tile_id, **kwargs): + return [tile_id] + + def find_tiles_by_id(self, tile_ids, ds=None, **kwargs): + return tile_ids + + def find_days_in_range_asc(self, min_lat, max_lat, min_lon, max_lon, dataset, start_time, end_time, + metrics_callback=None, **kwargs): + raise NotImplementedError() + + def find_tile_by_polygon_and_most_recent_day_of_year(self, bounding_polygon, ds, day_of_year, **kwargs): + """ + Given a bounding polygon, dataset, and day of year, find tiles in that dataset with the same bounding + polygon and the closest day of year. + + For example: + given a polygon minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; and day of year=32 + search for first tile in MY_DS with identical bbox and day_of_year <= 32 (sorted by day_of_year desc) + + Valid matches: + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 32 + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 30 + + Invalid matches: + minx=1, miny=0, maxx=2, maxy=1; dataset=MY_DS; day of year = 32 + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_OTHER_DS; day of year = 32 + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 30 if minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 32 also exists + + :param bounding_polygon: The exact bounding polygon of tiles to search for + :param ds: The dataset name being searched + :param day_of_year: Tile day of year to search for, tile nearest to this day (without going over) will be returned + :return: List of one tile from ds with bounding_polygon on or before day_of_year or raise NexusTileServiceException if no tile found + """ + + raise NotImplementedError() + + def find_all_tiles_in_box_at_time(self, min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs): + return self.find_tiles_in_box(min_lat, max_lat, min_lon, max_lon, dataset, time, time, **kwargs) + + def find_all_tiles_in_polygon_at_time(self, bounding_polygon, dataset, time, **kwargs): + return self.find_tiles_in_polygon(bounding_polygon, dataset, time, time, **kwargs) + + def find_tiles_in_box(self, min_lat, max_lat, min_lon, max_lon, ds=None, start_time=0, end_time=-1, **kwargs): + raise NotImplementedError() + + def find_tiles_in_polygon(self, bounding_polygon, ds=None, start_time=None, end_time=None, **kwargs): + # Find tiles that fall within the polygon in the Solr index + raise NotImplementedError() + + def find_tiles_by_metadata(self, metadata, ds=None, start_time=0, end_time=-1, **kwargs): + """ + Return list of tiles whose metadata matches the specified metadata, start_time, end_time. + :param metadata: List of metadata values to search for tiles e.g ["river_id_i:1", "granule_s:granule_name"] + :param ds: The dataset name to search + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :return: A list of tiles + """ + raise NotImplementedError() + + def find_tiles_by_exact_bounds(self, bounds, ds, start_time, end_time, **kwargs): + """ + The method will return tiles with the exact given bounds within the time range. It differs from + find_tiles_in_polygon in that only tiles with exactly the given bounds will be returned as opposed to + doing a polygon intersection with the given bounds. + + :param bounds: (minx, miny, maxx, maxy) bounds to search for + :param ds: Dataset name to search + :param start_time: Start time to search (seconds since epoch) + :param end_time: End time to search (seconds since epoch) + :param kwargs: fetch_data: True/False = whether or not to retrieve tile data + :return: + """ + raise NotImplementedError() + + def find_all_boundary_tiles_at_time(self, min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs): + # Due to the precise nature of gridded Zarr's subsetting, it doesn't make sense to have a boundary region like + # this + raise NotImplementedError() + + def get_min_max_time_by_granule(self, ds, granule_name): + raise NotImplementedError() + + def get_dataset_overall_stats(self, ds): + raise NotImplementedError() + + def get_stats_within_box_at_time(self, min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs): + raise NotImplementedError() + + def get_bounding_box(self, tile_ids): + """ + Retrieve a bounding box that encompasses all of the tiles represented by the given tile ids. + :param tile_ids: List of tile ids + :return: shapely.geometry.Polygon that represents the smallest bounding box that encompasses all of the tiles + """ + + raise NotImplementedError() + + # def __get_ds_min_max_date(self): + # min_date = self.__ds[self.__time].min().to_numpy() + # max_date = self.__ds[self.__time].max().to_numpy() + # + # if np.issubdtype(min_date.dtype, np.datetime64): + # min_date = ((min_date - np.datetime64(EPOCH)) / 1e9).astype(int).item() + # + # if np.issubdtype(max_date.dtype, np.datetime64): + # max_date = ((max_date - np.datetime64(EPOCH)) / 1e9).astype(int).item() + # + # return min_date, max_date + + def get_min_time(self, tile_ids, ds=None): + """ + Get the minimum tile date from the list of tile ids + :param tile_ids: List of tile ids + :param ds: Filter by a specific dataset. Defaults to None (queries all datasets) + :return: long time in seconds since epoch + """ + raise NotImplementedError() + + def get_max_time(self, tile_ids, ds=None): + """ + Get the maximum tile date from the list of tile ids + :param tile_ids: List of tile ids + :param ds: Filter by a specific dataset. Defaults to None (queries all datasets) + :return: long time in seconds since epoch + """ + raise NotImplementedError() + + def get_distinct_bounding_boxes_in_polygon(self, bounding_polygon, ds, start_time, end_time): + """ + Get a list of distinct tile bounding boxes from all tiles within the given polygon and time range. + :param bounding_polygon: The bounding polygon of tiles to search for + :param ds: The dataset name to search + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :return: A list of distinct bounding boxes (as shapely polygons) for tiles in the search polygon + """ + raise NotImplementedError() + + def get_tile_count(self, ds, bounding_polygon=None, start_time=0, end_time=-1, metadata=None, **kwargs): + """ + Return number of tiles that match search criteria. + :param ds: The dataset name to search + :param bounding_polygon: The polygon to search for tiles + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :param metadata: List of metadata values to search for tiles e.g ["river_id_i:1", "granule_s:granule_name"] + :return: number of tiles that match search criteria + """ + raise NotImplementedError() + + def fetch_data_for_tiles(self, *tiles): + for tile in tiles: + self.__fetch_data_for_tile(tile) + + return tiles + + def __fetch_data_for_tile(self, tile: Tile): + raise NotImplementedError() + + + def _metadata_store_docs_to_tiles(self, *store_docs): + return [ZarrBackend.__nts_url_to_tile(d) for d in store_docs] + + @staticmethod + def __nts_url_to_tile(nts_url): + raise NotImplementedError() + + @staticmethod + def __to_url(dataset, **kwargs): + raise NotImplementedError() + + diff --git a/data-access/nexustiles/backends/cog/SolrProxy.py b/data-access/nexustiles/backends/cog/SolrProxy.py new file mode 100644 index 00000000..92a96cda --- /dev/null +++ b/data-access/nexustiles/backends/cog/SolrProxy.py @@ -0,0 +1,180 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging +from nexustiles.backends.nexusproto.dao.SolrProxy import SolrProxy as SolrProxyBase +from datetime import datetime +from shapely.geometry import Polygon +from typing import Union, Optional, Dict + + +SOLR_FORMAT = '%Y-%m-%dT%H:%M:%SZ' + + +class SolrProxy(SolrProxyBase): + def __init__(self, config): + SolrProxyBase.__init__(self, config) + self.logger = logging.getLogger(__name__) + + def find_tiffs_in_bounds( + self, + dataset, + start, + end, + bounds: Optional[Union[Dict[str, float], str, Polygon]] = None, + **kwargs + ): + search = f'dataset_s:{dataset}' + + search_start_s = datetime.utcfromtimestamp(start).strftime(SOLR_FORMAT) + search_end_s = datetime.utcfromtimestamp(end).strftime(SOLR_FORMAT) + + time_clause = "(" \ + "min_time_dt:[%s TO %s] " \ + "OR max_time_dt:[%s TO %s] " \ + "OR (min_time_dt:[* TO %s] AND max_time_dt:[%s TO *])" \ + ")" % ( + search_start_s, search_end_s, + search_start_s, search_end_s, + search_start_s, search_end_s + ) + + params = { + 'fq': [time_clause], + 'sort': ['min_time_dt asc'] + } + + if bounds is not None: + if type(bounds) in [dict, str]: + if isinstance(bounds, dict): + max_lat = bounds['max_lat'] + max_lon = bounds['max_lon'] + min_lat = bounds['min_lat'] + min_lon = bounds['min_lon'] + else: + min_lon, min_lat, max_lon, max_lat = tuple([float(p) for p in bounds.split(',')]) + + params['fq'].append("geo:[%s,%s TO %s,%s]" % (min_lat, min_lon, max_lat, max_lon)) + elif isinstance(bounds, Polygon): + params['fq'].append('{!field f=geo}Intersects(%s)' % bounds.wkt) + + self._merge_kwargs(params, **kwargs) + + return self.do_query_all( + *(search, None, None, False, None), + **params + ) + + def date_range_for_dataset(self, dataset, **kwargs): + search = f'dataset_s:{dataset}' + + kwargs['rows'] = 1 + kwargs['sort'] = ['max_time_dt desc'] + # kwargs['fl'] = ['min_time_dt', 'max_time_dt'] + + params = {} + + self._merge_kwargs(params, **kwargs) + + results, start, found = self.do_query(*(search, None, None, True, None), **params) + + max_time = self.convert_iso_to_datetime(results[0]['max_time_dt']) + + params['sort'] = ['min_time_dt asc'] + + results, start, found = self.do_query(*(search, None, None, True, None), **params) + + min_time = self.convert_iso_to_datetime(results[0]['min_time_dt']) + + return min_time, max_time + + def find_min_date_from_tiffs(self, paths, ds, **kwargs): + search = f'dataset_s:{ds}' + + kwargs['rows'] = 1 + kwargs['fl'] = 'min_time_dt' + kwargs['sort'] = ['min_time_dt asc'] + additionalparams = { + 'fq': [ + "{!terms f=path_s}%s" % ','.join(paths) if len(paths) > 0 else '' + ] + } + + self._merge_kwargs(additionalparams, **kwargs) + + results, start, found = self.do_query(*(search, None, None, True, None), **additionalparams) + + return self.convert_iso_to_datetime(results[0]['min_time_dt']) + + def find_max_date_from_tiffs(self, paths, ds, **kwargs): + search = f'dataset_s:{ds}' + + kwargs['rows'] = 1 + kwargs['fl'] = 'max_time_dt' + kwargs['sort'] = ['max_time_dt desc'] + additionalparams = { + 'fq': [ + "{!terms f=path_s}%s" % ','.join(paths) if len(paths) > 0 else '' + ] + } + + self._merge_kwargs(additionalparams, **kwargs) + + results, start, found = self.do_query(*(search, None, None, True, None), **additionalparams) + + return self.convert_iso_to_datetime(results[0]['max_time_dt']) + + def find_days_in_range_asc(self, min_lat, max_lat, min_lon, max_lon, ds, start_time, end_time, **kwargs): + + search = 'dataset_s:%s' % ds + + search_start_s = datetime.utcfromtimestamp(start_time).strftime(SOLR_FORMAT) + search_end_s = datetime.utcfromtimestamp(end_time).strftime(SOLR_FORMAT) + + self.logger.warning('CoG backend does not yet support geo subsetting for TIFF selection') + + time_clause = "(" \ + "min_time_dt:[%s TO %s] " \ + "OR max_time_dt:[%s TO %s] " \ + "OR (min_time_dt:[* TO %s] AND max_time_dt:[%s TO *])" \ + ")" % ( + search_start_s, search_end_s, + search_start_s, search_end_s, + search_start_s, search_end_s + ) + + additionalparams = { + 'fq': [ + "geo:[%s,%s TO %s,%s]" % (min_lat, min_lon, max_lat, max_lon), + "{!frange l=0 u=0}ms(min_time_dt,max_time_dt)", + time_clause + ], + 'rows': 0, + 'facet': 'true', + 'facet.field': 'min_time_dt', + 'facet.mincount': '1', + 'facet.limit': '-1' + } + + self._merge_kwargs(additionalparams, **kwargs) + + response = self.do_query_raw(*(search, None, None, False, None), **additionalparams) + + daysinrangeasc = sorted( + [(datetime.strptime(a_date, SOLR_FORMAT) - datetime.utcfromtimestamp(0)).total_seconds() for a_date + in response.facets['facet_fields']['min_time_dt'][::2]]) + + return daysinrangeasc + diff --git a/data-access/nexustiles/backends/cog/__init__.py b/data-access/nexustiles/backends/cog/__init__.py new file mode 100644 index 00000000..67331447 --- /dev/null +++ b/data-access/nexustiles/backends/cog/__init__.py @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +from nexustiles.backends.cog.SolrProxy import SolrProxy diff --git a/data-access/nexustiles/backends/cog/backend.py b/data-access/nexustiles/backends/cog/backend.py new file mode 100644 index 00000000..016cd203 --- /dev/null +++ b/data-access/nexustiles/backends/cog/backend.py @@ -0,0 +1,565 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import logging +import sys +from datetime import datetime +from urllib.parse import urlparse + +import numpy as np +import numpy.ma as ma +import rioxarray +import rasterio as rio +from rasterio.session import AWSSession +import boto3 +import xarray as xr +from nexustiles.AbstractTileService import AbstractTileService +from nexustiles.exception import NexusTileServiceException +from nexustiles.model.nexusmodel import Tile, BBox, TileVariable +from pytz import timezone +from shapely.geometry import MultiPolygon, box +from yarl import URL +from rioxarray.exceptions import MissingCRS +from nexustiles.backends.cog import SolrProxy + +EPOCH = timezone('UTC').localize(datetime(1970, 1, 1)) +ISO_8601 = '%Y-%m-%dT%H:%M:%S%z' + +logging.basicConfig( + level=logging.INFO, + format='%(asctime)s - %(name)s - %(levelname)s - %(message)s', + datefmt="%Y-%m-%dT%H:%M:%S", stream=sys.stdout) +logger = logging.getLogger(__name__) + + +class CoGBackend(AbstractTileService): + def __init__(self, dataset_name, bands, solr_config, config=None): + AbstractTileService.__init__(self, dataset_name) + self.__config = config if config is not None else {} + + logger.info(f'Opening CoG backend at for dataset {self._name}') + + self.__bands = bands + + self.__longitude = 'longitude' + self.__latitude = 'latitude' + self.__time = 'time' + + # self.__depth = config['coords'].get('depth') + + self.__solr: SolrProxy = SolrProxy(solr_config) + + def get_dataseries_list(self, simple=False): + ds = dict( + shortName=self._name, + title=self._name, + type='Cloud Optimized GeoTIFF' + ) + + if not simple: + min_date, max_date = self.__solr.date_range_for_dataset(self._name) + + ds['start'] = (min_date - EPOCH).total_seconds() + ds['end'] = (max_date - EPOCH).total_seconds() + ds['iso_start'] = min_date.strftime(ISO_8601) + ds['iso_end'] = max_date.strftime(ISO_8601) + + return [ds] + + def find_tile_by_id(self, tile_id, **kwargs): + return [tile_id] + + def find_tiles_by_id(self, tile_ids, ds=None, **kwargs): + return tile_ids + + def find_days_in_range_asc(self, min_lat, max_lat, min_lon, max_lon, dataset, start_time, end_time, + metrics_callback=None, **kwargs): + return self.__solr.find_days_in_range_asc( + min_lat, + max_lat, + min_lon, + max_lon, + dataset, + start_time, + end_time, + **kwargs + ) + + def find_tile_by_polygon_and_most_recent_day_of_year(self, bounding_polygon, ds, day_of_year, **kwargs): + """ + Given a bounding polygon, dataset, and day of year, find tiles in that dataset with the same bounding + polygon and the closest day of year. + + For example: + given a polygon minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; and day of year=32 + search for first tile in MY_DS with identical bbox and day_of_year <= 32 (sorted by day_of_year desc) + + Valid matches: + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 32 + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 30 + + Invalid matches: + minx=1, miny=0, maxx=2, maxy=1; dataset=MY_DS; day of year = 32 + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_OTHER_DS; day of year = 32 + minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 30 if minx=0, miny=0, maxx=1, maxy=1; dataset=MY_DS; day of year = 32 also exists + + :param bounding_polygon: The exact bounding polygon of tiles to search for + :param ds: The dataset name being searched + :param day_of_year: Tile day of year to search for, tile nearest to this day (without going over) will be returned + :return: List of one tile from ds with bounding_polygon on or before day_of_year or raise NexusTileServiceException if no tile found + """ + + raise NotImplementedError() + + def find_all_tiles_in_box_at_time(self, min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs): + return self.find_tiles_in_box(min_lat, max_lat, min_lon, max_lon, dataset, time, time, **kwargs) + + def find_all_tiles_in_polygon_at_time(self, bounding_polygon, dataset, time, **kwargs): + return self.find_tiles_in_polygon(bounding_polygon, dataset, time, time, **kwargs) + + def find_tiles_in_box(self, min_lat, max_lat, min_lon, max_lon, ds=None, start_time=0, end_time=-1, **kwargs): + tiffs = self.__solr.find_tiffs_in_bounds( + ds, + start_time, + end_time, + { + 'min_lat': min_lat, + 'max_lat': max_lat, + 'min_lon': min_lon, + 'max_lon': max_lon + } + ) + + params = { + 'min_lat': min_lat, + 'max_lat': max_lat, + 'min_lon': min_lon, + 'max_lon': max_lon + } + + if 'depth' in kwargs: + params['depth'] = kwargs['depth'] + elif 'min_depth' in kwargs or 'max_depth' in kwargs: + params['min_depth'] = kwargs.get('min_depth') + params['max_depth'] = kwargs.get('max_depth') + + return[CoGBackend.__to_url( + self._name, + tiff['path_s'], + min_time=tiff.get('min_time_dt'), + max_time=tiff.get('max_time_dt'), + **params) for tiff in tiffs] + + def find_tiles_in_polygon(self, bounding_polygon, ds=None, start_time=None, end_time=None, **kwargs): + # Find tiles that fall within the polygon in the Solr index + tiffs = self.__solr.find_tiffs_in_bounds( + ds, + start_time, + end_time, + bounding_polygon + ) + + bounds = bounding_polygon.bounds + + params = { + 'min_lat': bounds[1], + 'max_lat': bounds[3], + 'min_lon': bounds[0], + 'max_lon': bounds[2] + } + + if 'depth' in kwargs: + params['depth'] = kwargs['depth'] + elif 'min_depth' in kwargs or 'max_depth' in kwargs: + params['min_depth'] = kwargs.get('min_depth') + params['max_depth'] = kwargs.get('max_depth') + + return [CoGBackend.__to_url( + self._name, + tiff['path_s'], + min_time=tiff.get('min_time_dt'), + max_time=tiff.get('max_time_dt'), + **params) for tiff in tiffs] + + def find_tiles_by_metadata(self, metadata, ds=None, start_time=0, end_time=-1, **kwargs): + """ + Return list of tiles whose metadata matches the specified metadata, start_time, end_time. + :param metadata: List of metadata values to search for tiles e.g ["river_id_i:1", "granule_s:granule_name"] + :param ds: The dataset name to search + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :return: A list of tiles + """ + raise NotImplementedError() + + def find_tiles_by_exact_bounds(self, bounds, ds, start_time, end_time, **kwargs): + """ + The method will return tiles with the exact given bounds within the time range. It differs from + find_tiles_in_polygon in that only tiles with exactly the given bounds will be returned as opposed to + doing a polygon intersection with the given bounds. + + :param bounds: (minx, miny, maxx, maxy) bounds to search for + :param ds: Dataset name to search + :param start_time: Start time to search (seconds since epoch) + :param end_time: End time to search (seconds since epoch) + :param kwargs: fetch_data: True/False = whether or not to retrieve tile data + :return: + """ + min_lon = bounds[0] + min_lat = bounds[1] + max_lon = bounds[2] + max_lat = bounds[3] + + return self.find_tiles_in_box(min_lat, max_lat, min_lon, max_lon, ds, start_time, end_time, **kwargs) + + def find_all_boundary_tiles_at_time(self, min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs): + # Due to the precise nature of gridded Zarr's subsetting, it doesn't make sense to have a boundary region like + # this + raise NotImplementedError() + + def find_tiles_along_line(self, start_point, end_point, ds=None, start_time=0, end_time=-1, **kwargs): + return self.__solr.find_tiles_along_line(start_point, end_point, ds, start_time, + end_time, **kwargs) + + def get_min_max_time_by_granule(self, ds, granule_name): + raise NotImplementedError() + + def get_dataset_overall_stats(self, ds): + raise NotImplementedError() + + def get_stats_within_box_at_time(self, min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs): + raise NotImplementedError() + + def get_bounding_box(self, tile_ids): + """ + Retrieve a bounding box that encompasses all of the tiles represented by the given tile ids. + :param tile_ids: List of tile ids + :return: shapely.geometry.Polygon that represents the smallest bounding box that encompasses all of the tiles + """ + + raise NotImplementedError() + + # def __get_ds_min_max_date(self): + # min_date = self.__ds[self.__time].min().to_numpy() + # max_date = self.__ds[self.__time].max().to_numpy() + # + # if np.issubdtype(min_date.dtype, np.datetime64): + # min_date = ((min_date - np.datetime64(EPOCH)) / 1e9).astype(int).item() + # + # if np.issubdtype(max_date.dtype, np.datetime64): + # max_date = ((max_date - np.datetime64(EPOCH)) / 1e9).astype(int).item() + # + # return min_date, max_date + + def get_min_time(self, tile_ids, ds=None): + """ + Get the minimum tile date from the list of tile ids + :param tile_ids: List of tile ids + :param ds: Filter by a specific dataset. Defaults to None (queries all datasets) + :return: long time in seconds since epoch + """ + paths = [URL(t).query['path'] for t in tile_ids] + + min_time = self.__solr.find_min_date_from_tiles(paths, self._name) + return int((min_time - EPOCH).total_seconds()) + + def get_max_time(self, tile_ids, ds=None): + """ + Get the maximum tile date from the list of tile ids + :param tile_ids: List of tile ids + :param ds: Filter by a specific dataset. Defaults to None (queries all datasets) + :return: long time in seconds since epoch + """ + paths = [URL(t).query['path'] for t in tile_ids] + + max_time = self.__solr.find_max_date_from_tiles(paths, self._name) + return int((max_time - EPOCH).total_seconds()) + + def get_distinct_bounding_boxes_in_polygon(self, bounding_polygon, ds, start_time, end_time): + """ + Get a list of distinct tile bounding boxes from all tiles within the given polygon and time range. + :param bounding_polygon: The bounding polygon of tiles to search for + :param ds: The dataset name to search + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :return: A list of distinct bounding boxes (as shapely polygons) for tiles in the search polygon + """ + raise NotImplementedError() + + def get_tile_count(self, ds, bounding_polygon=None, start_time=0, end_time=-1, metadata=None, **kwargs): + """ + Return number of tiles that match search criteria. + :param ds: The dataset name to search + :param bounding_polygon: The polygon to search for tiles + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :param metadata: List of metadata values to search for tiles e.g ["river_id_i:1", "granule_s:granule_name"] + :return: number of tiles that match search criteria + """ + return len(self.__solr.find_tiffs_in_bounds( + ds, + start_time, + end_time, + bounds=bounding_polygon + )) + + def fetch_data_for_tiles(self, *tiles): + for tile in tiles: + self.__fetch_data_for_tile(tile) + + return tiles + + @staticmethod + def __open_granule_at_url(url_s, time: np.datetime64, bands, config, **kwargs): + url = urlparse(url_s) + + logger.debug(f'Opening cog at {url_s}') + + if url.scheme in ['file', '']: + tiff = rioxarray.open_rasterio(url.path, mask_and_scale=False).to_dataset('band') + elif url.scheme == 's3': + try: + aws_cfg = config['aws'] + + key_id = aws_cfg['accessKeyID'] + secret = aws_cfg['secretAccessKey'] + except KeyError: + raise NexusTileServiceException(f'AWS config not provided for dataset {url.path}') + + session = boto3.Session( + aws_access_key_id=key_id, + aws_secret_access_key=secret, + region_name=aws_cfg.get('region', 'us-west-2') + ) + + with rio.Env( + AWSSession(session), + GDAL_DISABLE_READDIR_ON_OPEN='EMPTY_DIR' + ): + tiff = rioxarray.open_rasterio(url_s, mask_and_scale=False) + + ##### + # NOTE: This will likely be inefficient so leaving it disabled for now. I don't know how it will + # handle accessing data when the rio.Env context exits so maybe we want to pull it into memory before + # it does??? + # + # tiff = tiff.load() + + tiff = tiff.to_dataset('band') + else: + raise NotImplementedError(f'Support not yet added for tiffs with {url.scheme} URLs') + + # Broadcast the dataset attributes to the vars for decoding + for var in tiff.data_vars: + tiff[var].attrs.update(tiff.attrs) + + # Save the dtypes of the vars cause they will be lost to float32 on decode + var_dtypes = {var: tiff[var].dtype for var in tiff.data_vars} + + # Decode + tiff = xr.decode_cf(tiff) + + # Cast variables back to original dtypes + for var in tiff.data_vars: + tiff[var] = tiff[var].astype(var_dtypes[var]) + + try: + tiff = tiff.rio.reproject(dst_crs='EPSG:4326') + except MissingCRS: + pass + + rename = dict(x='longitude', y='latitude') + + drop = set(tiff.data_vars) + + for band in bands: + band_num = bands[band] + + rename[band_num] = band + drop.discard(band_num) + + drop.discard('spatial_ref') + + tiff = tiff.rename(rename).drop_vars(drop, errors='ignore') + + tiff = tiff.expand_dims({"time": 1}).assign_coords({"time": [time]}) + + return tiff + + def __fetch_data_for_tile(self, tile: Tile): + bbox: BBox = tile.bbox + + min_lat = None + min_lon = None + max_lat = None + max_lon = None + + if tile.min_time: + min_time = tile.min_time + else: + min_time = None + + if tile.max_time: + max_time = tile.max_time + else: + max_time = None + + # if min_time: + # min_time = datetime.utcfromtimestamp(min_time) + # + # if max_time: + # max_time = datetime.utcfromtimestamp(max_time) + + if bbox: + min_lat = bbox.min_lat + min_lon = bbox.min_lon + max_lat = bbox.max_lat + max_lon = bbox.max_lon + + granule = tile.granule + + ds: xr.Dataset = CoGBackend.__open_granule_at_url(granule, np.datetime64(min_time.isoformat(), 'ns'), self.__bands, self.__config) + variables = list(ds.data_vars) + + lats = ds[self.__latitude].to_numpy() + delta = lats[1] - lats[0] + + if delta < 0: + logger.warning(f'Latitude coordinate for {self._name} is in descending order. Flipping it to ascending') + ds = ds.isel({self.__latitude: slice(None, None, -1)}) + + sel_g = { + self.__latitude: slice(min_lat, max_lat), + self.__longitude: slice(min_lon, max_lon), + } + + sel_t = {} + + if min_time is None and max_time is None: + sel_t = None + method = None + elif min_time == max_time: + sel_t[self.__time] = [min_time] # List, otherwise self.__time dim will be dropped + method = 'nearest' + else: + sel_t[self.__time] = slice(min_time, max_time) + method = None + + tile.variables = [ + TileVariable(v, v) for v in variables + ] + + matched = ds.sel(sel_g) + + if sel_t is not None: + matched = matched.sel(sel_t, method=method) + + tile.latitudes = ma.masked_invalid(matched[self.__latitude].to_numpy()) + tile.longitudes = ma.masked_invalid(matched[self.__longitude].to_numpy()) + + times = matched[self.__time].to_numpy() + + if np.issubdtype(times.dtype, np.datetime64): + times = ((times - np.datetime64(EPOCH)) / 1e9).astype(int) + + tile.times = ma.masked_invalid(times) + + var_data = [matched[var].to_numpy() for var in variables] + + if len(variables) > 1: + tile.data = ma.masked_invalid(var_data) + tile.is_multi = True + else: + tile.data = ma.masked_invalid(var_data[0]) + tile.is_multi = False + + def _metadata_store_docs_to_tiles(self, *store_docs): + return [CoGBackend.__nts_url_to_tile(d) for d in store_docs] + + @staticmethod + def __nts_url_to_tile(nts_url): + tile = Tile() + + url = URL(nts_url) + + tile.tile_id = nts_url + + try: + min_lat = float(url.query['min_lat']) + min_lon = float(url.query['min_lon']) + max_lat = float(url.query['max_lat']) + max_lon = float(url.query['max_lon']) + + tile.bbox = BBox(min_lat, max_lat, min_lon, max_lon) + except KeyError: + pass + + tile.dataset = url.path + tile.dataset_id = url.path + tile.granule = url.query['path'] + + try: + # tile.min_time = int(url.query['min_time']) + tile.min_time = datetime.utcfromtimestamp(int(url.query['min_time'])) + except ValueError: + tile.min_time = datetime.strptime(url.query['min_time'], '%Y-%m-%dT%H:%M:%SZ') + except KeyError: + pass + + try: + # tile.max_time = int(url.query['max_time']) + tile.max_time = datetime.utcfromtimestamp(int(url.query['max_time'])) + except ValueError: + tile.max_time = datetime.strptime(url.query['max_time'], '%Y-%m-%dT%H:%M:%SZ') + except KeyError: + pass + + tile.meta_data = {} + + return tile + + @staticmethod + def __to_url(dataset, tiff, **kwargs): + if 'dataset' in kwargs: + del kwargs['dataset'] + + if 'ds' in kwargs: + del kwargs['ds'] + + if 'path' in kwargs: + del kwargs['path'] + + params = {} + + # If any params are numpy dtypes, extract them to base python types + for kw in kwargs: + v = kwargs[kw] + + if v is None: + continue + + if isinstance(v, np.generic): + v = v.item() + + params[kw] = v + + params['path'] = tiff + + return str(URL.build( + scheme='cog', + host='', + path=dataset, + query=params + )) diff --git a/data-access/nexustiles/backends/nexusproto/dao/SolrProxy.py b/data-access/nexustiles/backends/nexusproto/dao/SolrProxy.py index 18f536a7..0d1d37e7 100644 --- a/data-access/nexustiles/backends/nexusproto/dao/SolrProxy.py +++ b/data-access/nexustiles/backends/nexusproto/dao/SolrProxy.py @@ -38,7 +38,7 @@ def __init__(self, config): self.solrCore = config.get("solr", "core") solr_kargs = {} if config.has_option("solr", "time_out"): - solr_kargs["timeout"] = config.get("solr", "time_out") + solr_kargs["timeout"] = float(config.get("solr", "time_out")) self.logger = logging.getLogger('nexus') with SOLR_CON_LOCK: @@ -842,7 +842,8 @@ def _merge_kwargs(additionalparams, **kwargs): s = None try: - additionalparams['sort'].extend(s) + if s is not None: + additionalparams['sort'].extend(s) except KeyError: if s is not None: additionalparams['sort'] = s diff --git a/data-access/nexustiles/backends/zarr/backend.py b/data-access/nexustiles/backends/zarr/backend.py index cf45cc01..8fb7fcf0 100644 --- a/data-access/nexustiles/backends/zarr/backend.py +++ b/data-access/nexustiles/backends/zarr/backend.py @@ -536,7 +536,7 @@ def __to_url(dataset, **kwargs): params[kw] = v return str(URL.build( - scheme='nts', + scheme='zarr', host='', path=dataset, query=params diff --git a/data-access/nexustiles/exception.py b/data-access/nexustiles/exception.py index d6ed2c64..1f2fd839 100644 --- a/data-access/nexustiles/exception.py +++ b/data-access/nexustiles/exception.py @@ -16,3 +16,15 @@ class NexusTileServiceException(Exception): def __init__(self, reason): Exception.__init__(self, reason) + + +class AlgorithmUnsupportedForDatasetException(Exception): + def __init__(self, reason): + Exception.__init__(self, reason) + + + +class AlgorithmUnsupportedForDatasetException(Exception): + def __init__(self, reason): + Exception.__init__(self, reason) + diff --git a/data-access/nexustiles/nexustiles.py b/data-access/nexustiles/nexustiles.py index cbe42ca1..d5169b05 100644 --- a/data-access/nexustiles/nexustiles.py +++ b/data-access/nexustiles/nexustiles.py @@ -14,6 +14,7 @@ # limitations under the License. import configparser +import io import json import logging import sys @@ -34,11 +35,13 @@ from yarl import URL from .AbstractTileService import AbstractTileService -from .backends.nexusproto.backend import NexusprotoTileService -from .backends.zarr.backend import ZarrBackend +# from .backends.nexusproto.backend import NexusprotoTileService +# from .backends.zarr.backend import ZarrBackend from .model.nexusmodel import Tile, BBox, TileStats, TileVariable -from .exception import NexusTileServiceException +from nexustiles.backends import * + +from .exception import NexusTileServiceException, AlgorithmUnsupportedForDatasetException from requests.structures import CaseInsensitiveDict @@ -101,7 +104,7 @@ def wrapper(*args, **kwargs): try: return func(*args, **kwargs) except NotImplementedError: - raise NexusTileServiceException('Action unsupported by backend') + raise AlgorithmUnsupportedForDatasetException('Action unsupported by backend') return wrapper @@ -173,6 +176,10 @@ def is_update_thread_alive() -> bool: def _get_backend(dataset_s) -> AbstractTileService: if dataset_s is not None: dataset_s = dataset_s + else: + logger.warning('_get_backend called with dataset_s=None') + + logger.debug(f'Getting backend for {dataset_s}') with NexusTileService.DS_LOCK: if dataset_s not in NexusTileService.backends: @@ -257,6 +264,32 @@ def _update_datasets(): } except NexusTileServiceException: added_datasets -= 1 + elif store_type.lower() in ['cog', 'cloud_optimized_geotiff']: + update_logger.info(f'Detected new CoG dataset {d_id}, opening new CoG backend') + + ds_config = json.loads(dataset['config'][0]) + + solr_config_str = io.StringIO() + + NexusTileService.ds_config.write(solr_config_str) + + solr_config_str.seek(0) + solr_config = configparser.ConfigParser() + solr_config.read_file(solr_config_str) + + solr_config.set('solr', 'core', 'nexusgranules') + + try: + NexusTileService.backends[d_id] = { + 'backend': CoGBackend( + dataset_name=dataset['dataset_s'], + solr_config=solr_config, + **ds_config + ), + 'up': True + } + except NexusTileServiceException: + added_datasets -= 1 else: update_logger.warning(f'Unsupported backend {store_type} for dataset {d_id}') added_datasets -= 1 @@ -407,7 +440,7 @@ def heartbeat(self) -> Dict[str, bool]: def find_tile_by_id(self, tile_id, **kwargs): tile = URL(tile_id) - if tile.scheme == 'nts': + if tile.scheme != '': return NexusTileService._get_backend(tile.path).find_tile_by_id(tile_id) else: return NexusTileService._get_backend('__nexusproto__').find_tile_by_id(tile_id) @@ -613,6 +646,7 @@ def get_stats_within_box_at_time(self, min_lat, max_lat, min_lon, max_lon, datas min_lat, max_lat, min_lon, max_lon, dataset, time, **kwargs ) + @catch_not_implemented def get_bounding_box(self, tile_ids, ds=None): """ Retrieve a bounding box that encompasses all of the tiles represented by the given tile ids. @@ -621,6 +655,7 @@ def get_bounding_box(self, tile_ids, ds=None): """ return NexusTileService._get_backend(ds).get_bounding_box(tile_ids) + @catch_not_implemented def get_min_time(self, tile_ids, ds=None): """ Get the minimum tile date from the list of tile ids @@ -628,8 +663,9 @@ def get_min_time(self, tile_ids, ds=None): :param ds: Filter by a specific dataset. Defaults to None (queries all datasets) :return: long time in seconds since epoch """ - return NexusTileService._get_backend(ds).get_min_time(tile_ids, ds) + return int(NexusTileService._get_backend(ds).get_min_time(tile_ids, ds)) + @catch_not_implemented def get_max_time(self, tile_ids, ds=None): """ Get the maximum tile date from the list of tile ids @@ -639,6 +675,7 @@ def get_max_time(self, tile_ids, ds=None): """ return int(NexusTileService._get_backend(ds).get_max_time(tile_ids)) + @catch_not_implemented def get_distinct_bounding_boxes_in_polygon(self, bounding_polygon, ds, start_time, end_time): """ Get a list of distinct tile bounding boxes from all tiles within the given polygon and time range. @@ -651,6 +688,7 @@ def get_distinct_bounding_boxes_in_polygon(self, bounding_polygon, ds, start_tim bounds = self._metadatastore.find_distinct_bounding_boxes_in_polygon(bounding_polygon, ds, start_time, end_time) return [box(*b) for b in bounds] + @catch_not_implemented def get_tile_count(self, ds, bounding_polygon=None, start_time=0, end_time=-1, metadata=None, **kwargs): """ Return number of tiles that match search criteria. @@ -824,6 +862,51 @@ def mask_tiles_to_elevation(self, min_e, max_e, tiles): return tiles + def mask_tiles_to_elevation(self, min_e, max_e, tiles): + """ + Masks data in tiles to specified time range. + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :param tiles: List of tiles + :return: A list tiles with data masked to specified time range + """ + for tile in tiles: + tile.elevation = ma.masked_outside(tile.elevation, min_e, max_e) + + # Or together the masks of the individual arrays to create the new mask + data_mask = ma.getmaskarray(tile.times)[:, np.newaxis, np.newaxis] \ + | ma.getmaskarray(tile.elevation)[np.newaxis, :, :] \ + + # If this is multi-var, need to mask each variable separately. + if tile.is_multi: + # Combine space/time mask with existing mask on data + data_mask = reduce(np.logical_or, [tile.data[0].mask, data_mask]) + + num_vars = len(tile.data) + multi_data_mask = np.repeat(data_mask[np.newaxis, ...], num_vars, axis=0) + multi_data_mask = np.broadcast_to(multi_data_mask, tile.data.shape) + + tile.data = ma.masked_where(multi_data_mask, tile.data) + else: + data_mask = np.broadcast_to(data_mask, tile.data.shape) + tile.data = ma.masked_where(data_mask, tile.data) + + tiles[:] = [tile for tile in tiles if not tile.data.mask.all()] + + return tiles + + def get_tile_count(self, ds, bounding_polygon=None, start_time=0, end_time=-1, metadata=None, **kwargs): + """ + Return number of tiles that match search criteria. + :param ds: The dataset name to search + :param bounding_polygon: The polygon to search for tiles + :param start_time: The start time to search for tiles + :param end_time: The end time to search for tiles + :param metadata: List of metadata values to search for tiles e.g ["river_id_i:1", "granule_s:granule_name"] + :return: number of tiles that match search criteria + """ + return self._metadatastore.get_tile_count(ds, bounding_polygon, start_time, end_time, metadata, **kwargs) + def fetch_data_for_tiles(self, *tiles): dataset = tiles[0].dataset diff --git a/poetry.lock b/poetry.lock index de1ef5d5..731304cd 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,5 +1,20 @@ # This file is automatically @generated by Poetry 1.8.3 and should not be changed by hand. +[[package]] +name = "affine" +version = "2.4.0" +description = "Matrices describing affine transformation of the plane" +optional = false +python-versions = ">=3.7" +files = [ + {file = "affine-2.4.0-py3-none-any.whl", hash = "sha256:8a3df80e2b2378aef598a83c1392efd47967afec4242021a0b06b4c7cbc61a92"}, + {file = "affine-2.4.0.tar.gz", hash = "sha256:a24d818d6a836c131976d22f8c27b8d3ca32d0af64c1d8d29deb7bafa4da1eea"}, +] + +[package.extras] +dev = ["coveralls", "flake8", "pydocstyle"] +test = ["pytest (>=4.6)", "pytest-cov"] + [[package]] name = "aiobotocore" version = "2.3.4" @@ -414,6 +429,40 @@ files = [ [package.dependencies] colorama = {version = "*", markers = "platform_system == \"Windows\""} +[[package]] +name = "click-plugins" +version = "1.1.1" +description = "An extension module for click to enable registering CLI commands via setuptools entry-points." +optional = false +python-versions = "*" +files = [ + {file = "click-plugins-1.1.1.tar.gz", hash = "sha256:46ab999744a9d831159c3411bb0c79346d94a444df9a3a3742e9ed63645f264b"}, + {file = "click_plugins-1.1.1-py2.py3-none-any.whl", hash = "sha256:5d262006d3222f5057fd81e1623d4443e41dcda5dc815c06b442aa3c02889fc8"}, +] + +[package.dependencies] +click = ">=4.0" + +[package.extras] +dev = ["coveralls", "pytest (>=3.6)", "pytest-cov", "wheel"] + +[[package]] +name = "cligj" +version = "0.7.2" +description = "Click params for commmand line interfaces to GeoJSON" +optional = false +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, <4" +files = [ + {file = "cligj-0.7.2-py3-none-any.whl", hash = "sha256:c1ca117dbce1fe20a5809dc96f01e1c2840f6dcc939b3ddbb1111bf330ba82df"}, + {file = "cligj-0.7.2.tar.gz", hash = "sha256:a4bc13d623356b373c2c27c53dbd9c68cae5d526270bfa71f6c6fa69669c6b27"}, +] + +[package.dependencies] +click = ">=4.0" + +[package.extras] +test = ["pytest-cov"] + [[package]] name = "cloudpickle" version = "3.0.0" @@ -701,13 +750,13 @@ test = ["pandas[test]", "pre-commit", "pytest", "pytest-rerunfailures", "pytest- [[package]] name = "dill" -version = "0.3.8" +version = "0.3.9" description = "serialize all of Python" optional = false python-versions = ">=3.8" files = [ - {file = "dill-0.3.8-py3-none-any.whl", hash = "sha256:c36ca9ffb54365bdd2f8eb3eff7d2a21237f8452b57ace88b1ac615b7e815bd7"}, - {file = "dill-0.3.8.tar.gz", hash = "sha256:3ebe3c479ad625c4553aca177444d89b486b1d84982eeacded644afc0cf797ca"}, + {file = "dill-0.3.9-py3-none-any.whl", hash = "sha256:468dff3b89520b474c0397703366b7b95eebe6303f108adf9b19da1f702be87a"}, + {file = "dill-0.3.9.tar.gz", hash = "sha256:81aa267dddf68cbfe8029c42ca9ec6a4ab3b22371d1c450abc54422577b4512c"}, ] [package.extras] @@ -777,59 +826,59 @@ files = [ [[package]] name = "fonttools" -version = "4.54.0" +version = "4.54.1" description = "Tools to manipulate font files" optional = false python-versions = ">=3.8" files = [ - {file = "fonttools-4.54.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:b2957597455a21fc55849cf5094507028b241035e9bf2d98daa006c152553640"}, - {file = "fonttools-4.54.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:18a043a029994c28638bd40cf0d7dbe8edfbacb6b60f6a5ccdfcc4db98eaa4e4"}, - {file = "fonttools-4.54.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cb1dd36e8612b31f30ae8fa264fdddf1a0c22bab0990c5f97542b86cbf0b77ec"}, - {file = "fonttools-4.54.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2703efc48b6e88b58249fb6316373e15e5b2e5835a58114954b290faebbd89da"}, - {file = "fonttools-4.54.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:21a209d7ff42ab567e449ba8f86af7bc5e93e2463bd07cbfae7284057d1552ac"}, - {file = "fonttools-4.54.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:812d04179b6a99bff3241153c928e1b3db98c76113375ce6b561e93dc749da3f"}, - {file = "fonttools-4.54.0-cp310-cp310-win32.whl", hash = "sha256:0d15664cbdc059ca1a32ff2a5cb5428ffd47f2e739430d9d11b0b6e2a97f2b8b"}, - {file = "fonttools-4.54.0-cp310-cp310-win_amd64.whl", hash = "sha256:abc5acdfdb01e2af1de55153f3720376edf4df8bcad84bdc54c08abda2089fd4"}, - {file = "fonttools-4.54.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:96e7a37190a20063dc6f301665180148ec7671f9b6ef089dba2280a8434adacc"}, - {file = "fonttools-4.54.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:a42e0500944de3abf8723a439c7c94678d14b702808a593d7bfcece4a3ff4479"}, - {file = "fonttools-4.54.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:24160f6df15e01d0edfb64729373950c2869871a611924d50c2e676162dcc42d"}, - {file = "fonttools-4.54.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b3c556e69f66de64b2604d6875d5d1913484f89336d782a4bb89b772648436a3"}, - {file = "fonttools-4.54.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:2ee6664fe61a932f52c499d2e8d72e6c7c6207449e2ec12928ebf80f2580ea31"}, - {file = "fonttools-4.54.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:79bb6834403cbb0f851df7173e8e9adbcfe3bb2e09a472de4c2e8a2667257b47"}, - {file = "fonttools-4.54.0-cp311-cp311-win32.whl", hash = "sha256:6679b471655f4f6bcdacb2b05bb059fc8d10983870e1a039d101da50562b90ec"}, - {file = "fonttools-4.54.0-cp311-cp311-win_amd64.whl", hash = "sha256:17d328d8d7414d7a70186a0d5c6fe9eea04b8b019ae070964b0555acfa763bba"}, - {file = "fonttools-4.54.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:34758e8481a5054e7e203c5e15c41dc3ec67716407bd1f00ebf014fe94f934e3"}, - {file = "fonttools-4.54.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:49124ff0efd6ded3e320912409527c9f3dae34acf34dcca141961a0c2dee484e"}, - {file = "fonttools-4.54.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:105b4dbf35bd8aad2c79b8b12ca911a00d7e445a251383a523497e0fb06c4242"}, - {file = "fonttools-4.54.0-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3b6b613894d8e90093326ab6014c202a7a503e34dfb4a632b2ec78078f406c43"}, - {file = "fonttools-4.54.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6587da0a397c9ae36b8c7e3febfca8c4563d287f7339d805cd4a9a356a39f6bf"}, - {file = "fonttools-4.54.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:801bdd3496ec6df3920ae5cf43567208246c944288d2a508985491c9126f4dd9"}, - {file = "fonttools-4.54.0-cp312-cp312-win32.whl", hash = "sha256:e299ecc34635621b792bf42dcc3be50810dd74c888474e09b47596853a08db56"}, - {file = "fonttools-4.54.0-cp312-cp312-win_amd64.whl", hash = "sha256:f7b2e35b912235290b5e8df0cab17e3365be887c88588fdd9589e7635e665460"}, - {file = "fonttools-4.54.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:948fafa5035cf22ed35040c07b7a4ebe9c9d3436401d4d4a4fea19a24bee8fd5"}, - {file = "fonttools-4.54.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:ef61d49d1f724dd8f1bf99328cfbc5e64900f451be0eacfcd15a1e00493779be"}, - {file = "fonttools-4.54.0-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3d037c0b7d93408584065f5d30cd3a1c533a195d96669de116df3b594f6753b6"}, - {file = "fonttools-4.54.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:dbb7646fd6f6fdf754015cbb50db10cd53770432e56bd6b2e6411fb54a1b83b2"}, - {file = "fonttools-4.54.0-cp313-cp313-win32.whl", hash = "sha256:66143c6607d85647ef5097c7d3005118288ef6d7607487c10b04549f830eee01"}, - {file = "fonttools-4.54.0-cp313-cp313-win_amd64.whl", hash = "sha256:f66a6e29a201a4e0ff8a1f33dc90781f018e0dd8caa29d33311110952bdf8285"}, - {file = "fonttools-4.54.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:eb871afe7bd480d233c0c29a694cbc553743e8af9c8daa9c70284862b35c5e80"}, - {file = "fonttools-4.54.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:4f864d49effec5877c1ea559e2cb01bf6162f066c9013b78e1b31c13c120bee4"}, - {file = "fonttools-4.54.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e56abc2aad22298bd62f1314940b22f613eb4e9a50c5d9450d50c4c42e4617bf"}, - {file = "fonttools-4.54.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:633bd642239412115a4d203728980bf57993f1bcd22299c71f0c2ea669b75604"}, - {file = "fonttools-4.54.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:1170ed2208ace22ebe3bd119cec42fec9d393a133c204d6c7a28f28820c1eced"}, - {file = "fonttools-4.54.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:59ed3b6fcdfc29e4ffb75d300710bef50379caa639cd8e1b83415f7f1462d6ec"}, - {file = "fonttools-4.54.0-cp38-cp38-win32.whl", hash = "sha256:c6db5c17464f50ccd1b6d362e54d5e5930e551382c79f36f5f73b2bfd20fc340"}, - {file = "fonttools-4.54.0-cp38-cp38-win_amd64.whl", hash = "sha256:c4392e878e8e8d14ab7963a5accf25802eb6a9499c40e698c9bf571816026daf"}, - {file = "fonttools-4.54.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:7a05cb4ebb638147a11b15eb2fffbe71bbf2df7ec6d6655430a07d97164dddb0"}, - {file = "fonttools-4.54.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7b80c2e5ce6e69291fe73f7a71f26ae767e53e8c2e4b08826d7c9524ef0ebaad"}, - {file = "fonttools-4.54.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:627c0e59883fb97be4ec46cb0561f521214f3d8a10ad7f2a4030d3cd38a0a0ab"}, - {file = "fonttools-4.54.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cc4e10d9c7e9ec55431f49f7425bc5c0472f0b25ff56ad57a66d7e503d36e83e"}, - {file = "fonttools-4.54.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:370a2018eeaeba47742103ac4e3877acfa7819ea64725aa7646f16e1cbab6223"}, - {file = "fonttools-4.54.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:4dc1e6ebff17e2f012d5084058fd89fd66c7fd02ac9960380fab236114a977fb"}, - {file = "fonttools-4.54.0-cp39-cp39-win32.whl", hash = "sha256:fff3ff4a7e864b98502a15b04f3b9eedd26f8ff3f60be325cd715b9af8e54d05"}, - {file = "fonttools-4.54.0-cp39-cp39-win_amd64.whl", hash = "sha256:e7e1c173b21d00f336ab0d4edf2ea64e7a8530863bae789d97cd52a4363fbd6f"}, - {file = "fonttools-4.54.0-py3-none-any.whl", hash = "sha256:351058cd623af4c45490c744e2bbc5671fc78dce95866e92122c9ba6c28ea8b6"}, - {file = "fonttools-4.54.0.tar.gz", hash = "sha256:9f3482ff1189668fa9f8eafe8ff8541fb154b6f0170f8477889c028eb893c6ee"}, + {file = "fonttools-4.54.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:7ed7ee041ff7b34cc62f07545e55e1468808691dddfd315d51dd82a6b37ddef2"}, + {file = "fonttools-4.54.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:41bb0b250c8132b2fcac148e2e9198e62ff06f3cc472065dff839327945c5882"}, + {file = "fonttools-4.54.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7965af9b67dd546e52afcf2e38641b5be956d68c425bef2158e95af11d229f10"}, + {file = "fonttools-4.54.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:278913a168f90d53378c20c23b80f4e599dca62fbffae4cc620c8eed476b723e"}, + {file = "fonttools-4.54.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:0e88e3018ac809b9662615072dcd6b84dca4c2d991c6d66e1970a112503bba7e"}, + {file = "fonttools-4.54.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:4aa4817f0031206e637d1e685251ac61be64d1adef111060df84fdcbc6ab6c44"}, + {file = "fonttools-4.54.1-cp310-cp310-win32.whl", hash = "sha256:7e3b7d44e18c085fd8c16dcc6f1ad6c61b71ff463636fcb13df7b1b818bd0c02"}, + {file = "fonttools-4.54.1-cp310-cp310-win_amd64.whl", hash = "sha256:dd9cc95b8d6e27d01e1e1f1fae8559ef3c02c76317da650a19047f249acd519d"}, + {file = "fonttools-4.54.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:5419771b64248484299fa77689d4f3aeed643ea6630b2ea750eeab219588ba20"}, + {file = "fonttools-4.54.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:301540e89cf4ce89d462eb23a89464fef50915255ece765d10eee8b2bf9d75b2"}, + {file = "fonttools-4.54.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:76ae5091547e74e7efecc3cbf8e75200bc92daaeb88e5433c5e3e95ea8ce5aa7"}, + {file = "fonttools-4.54.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:82834962b3d7c5ca98cb56001c33cf20eb110ecf442725dc5fdf36d16ed1ab07"}, + {file = "fonttools-4.54.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:d26732ae002cc3d2ecab04897bb02ae3f11f06dd7575d1df46acd2f7c012a8d8"}, + {file = "fonttools-4.54.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:58974b4987b2a71ee08ade1e7f47f410c367cdfc5a94fabd599c88165f56213a"}, + {file = "fonttools-4.54.1-cp311-cp311-win32.whl", hash = "sha256:ab774fa225238986218a463f3fe151e04d8c25d7de09df7f0f5fce27b1243dbc"}, + {file = "fonttools-4.54.1-cp311-cp311-win_amd64.whl", hash = "sha256:07e005dc454eee1cc60105d6a29593459a06321c21897f769a281ff2d08939f6"}, + {file = "fonttools-4.54.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:54471032f7cb5fca694b5f1a0aaeba4af6e10ae989df408e0216f7fd6cdc405d"}, + {file = "fonttools-4.54.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:8fa92cb248e573daab8d032919623cc309c005086d743afb014c836636166f08"}, + {file = "fonttools-4.54.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0a911591200114969befa7f2cb74ac148bce5a91df5645443371aba6d222e263"}, + {file = "fonttools-4.54.1-cp312-cp312-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:93d458c8a6a354dc8b48fc78d66d2a8a90b941f7fec30e94c7ad9982b1fa6bab"}, + {file = "fonttools-4.54.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:5eb2474a7c5be8a5331146758debb2669bf5635c021aee00fd7c353558fc659d"}, + {file = "fonttools-4.54.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c9c563351ddc230725c4bdf7d9e1e92cbe6ae8553942bd1fb2b2ff0884e8b714"}, + {file = "fonttools-4.54.1-cp312-cp312-win32.whl", hash = "sha256:fdb062893fd6d47b527d39346e0c5578b7957dcea6d6a3b6794569370013d9ac"}, + {file = "fonttools-4.54.1-cp312-cp312-win_amd64.whl", hash = "sha256:e4564cf40cebcb53f3dc825e85910bf54835e8a8b6880d59e5159f0f325e637e"}, + {file = "fonttools-4.54.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:6e37561751b017cf5c40fce0d90fd9e8274716de327ec4ffb0df957160be3bff"}, + {file = "fonttools-4.54.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:357cacb988a18aace66e5e55fe1247f2ee706e01debc4b1a20d77400354cddeb"}, + {file = "fonttools-4.54.1-cp313-cp313-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f8e953cc0bddc2beaf3a3c3b5dd9ab7554677da72dfaf46951e193c9653e515a"}, + {file = "fonttools-4.54.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:58d29b9a294573d8319f16f2f79e42428ba9b6480442fa1836e4eb89c4d9d61c"}, + {file = "fonttools-4.54.1-cp313-cp313-win32.whl", hash = "sha256:9ef1b167e22709b46bf8168368b7b5d3efeaaa746c6d39661c1b4405b6352e58"}, + {file = "fonttools-4.54.1-cp313-cp313-win_amd64.whl", hash = "sha256:262705b1663f18c04250bd1242b0515d3bbae177bee7752be67c979b7d47f43d"}, + {file = "fonttools-4.54.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:ed2f80ca07025551636c555dec2b755dd005e2ea8fbeb99fc5cdff319b70b23b"}, + {file = "fonttools-4.54.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:9dc080e5a1c3b2656caff2ac2633d009b3a9ff7b5e93d0452f40cd76d3da3b3c"}, + {file = "fonttools-4.54.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:1d152d1be65652fc65e695e5619e0aa0982295a95a9b29b52b85775243c06556"}, + {file = "fonttools-4.54.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8583e563df41fdecef31b793b4dd3af8a9caa03397be648945ad32717a92885b"}, + {file = "fonttools-4.54.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:0d1d353ef198c422515a3e974a1e8d5b304cd54a4c2eebcae708e37cd9eeffb1"}, + {file = "fonttools-4.54.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:fda582236fee135d4daeca056c8c88ec5f6f6d88a004a79b84a02547c8f57386"}, + {file = "fonttools-4.54.1-cp38-cp38-win32.whl", hash = "sha256:e7d82b9e56716ed32574ee106cabca80992e6bbdcf25a88d97d21f73a0aae664"}, + {file = "fonttools-4.54.1-cp38-cp38-win_amd64.whl", hash = "sha256:ada215fd079e23e060157aab12eba0d66704316547f334eee9ff26f8c0d7b8ab"}, + {file = "fonttools-4.54.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:f5b8a096e649768c2f4233f947cf9737f8dbf8728b90e2771e2497c6e3d21d13"}, + {file = "fonttools-4.54.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:4e10d2e0a12e18f4e2dd031e1bf7c3d7017be5c8dbe524d07706179f355c5dac"}, + {file = "fonttools-4.54.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:31c32d7d4b0958600eac75eaf524b7b7cb68d3a8c196635252b7a2c30d80e986"}, + {file = "fonttools-4.54.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c39287f5c8f4a0c5a55daf9eaf9ccd223ea59eed3f6d467133cc727d7b943a55"}, + {file = "fonttools-4.54.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:a7a310c6e0471602fe3bf8efaf193d396ea561486aeaa7adc1f132e02d30c4b9"}, + {file = "fonttools-4.54.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:d3b659d1029946f4ff9b6183984578041b520ce0f8fb7078bb37ec7445806b33"}, + {file = "fonttools-4.54.1-cp39-cp39-win32.whl", hash = "sha256:e96bc94c8cda58f577277d4a71f51c8e2129b8b36fd05adece6320dd3d57de8a"}, + {file = "fonttools-4.54.1-cp39-cp39-win_amd64.whl", hash = "sha256:e8a4b261c1ef91e7188a30571be6ad98d1c6d9fa2427244c545e2fa0a2494dd7"}, + {file = "fonttools-4.54.1-py3-none-any.whl", hash = "sha256:37cddd62d83dc4f72f7c3f3c2bcf2697e89a30efb152079896544a93907733bd"}, + {file = "fonttools-4.54.1.tar.gz", hash = "sha256:957f669d4922f92c171ba01bef7f29410668db09f6c02111e22b2bce446f3285"}, ] [package.extras] @@ -1852,6 +1901,113 @@ files = [ dev = ["pre-commit", "tox"] testing = ["pytest", "pytest-benchmark"] +[[package]] +name = "propcache" +version = "0.2.0" +description = "Accelerated property cache" +optional = false +python-versions = ">=3.8" +files = [ + {file = "propcache-0.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:c5869b8fd70b81835a6f187c5fdbe67917a04d7e52b6e7cc4e5fe39d55c39d58"}, + {file = "propcache-0.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:952e0d9d07609d9c5be361f33b0d6d650cd2bae393aabb11d9b719364521984b"}, + {file = "propcache-0.2.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:33ac8f098df0585c0b53009f039dfd913b38c1d2edafed0cedcc0c32a05aa110"}, + {file = "propcache-0.2.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:97e48e8875e6c13909c800fa344cd54cc4b2b0db1d5f911f840458a500fde2c2"}, + {file = "propcache-0.2.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:388f3217649d6d59292b722d940d4d2e1e6a7003259eb835724092a1cca0203a"}, + {file = "propcache-0.2.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f571aea50ba5623c308aa146eb650eebf7dbe0fd8c5d946e28343cb3b5aad577"}, + {file = "propcache-0.2.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3dfafb44f7bb35c0c06eda6b2ab4bfd58f02729e7c4045e179f9a861b07c9850"}, + {file = "propcache-0.2.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a3ebe9a75be7ab0b7da2464a77bb27febcb4fab46a34f9288f39d74833db7f61"}, + {file = "propcache-0.2.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:d2f0d0f976985f85dfb5f3d685697ef769faa6b71993b46b295cdbbd6be8cc37"}, + {file = "propcache-0.2.0-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:a3dc1a4b165283bd865e8f8cb5f0c64c05001e0718ed06250d8cac9bec115b48"}, + {file = "propcache-0.2.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:9e0f07b42d2a50c7dd2d8675d50f7343d998c64008f1da5fef888396b7f84630"}, + {file = "propcache-0.2.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:e63e3e1e0271f374ed489ff5ee73d4b6e7c60710e1f76af5f0e1a6117cd26394"}, + {file = "propcache-0.2.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:56bb5c98f058a41bb58eead194b4db8c05b088c93d94d5161728515bd52b052b"}, + {file = "propcache-0.2.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:7665f04d0c7f26ff8bb534e1c65068409bf4687aa2534faf7104d7182debb336"}, + {file = "propcache-0.2.0-cp310-cp310-win32.whl", hash = "sha256:7cf18abf9764746b9c8704774d8b06714bcb0a63641518a3a89c7f85cc02c2ad"}, + {file = "propcache-0.2.0-cp310-cp310-win_amd64.whl", hash = "sha256:cfac69017ef97db2438efb854edf24f5a29fd09a536ff3a992b75990720cdc99"}, + {file = "propcache-0.2.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:63f13bf09cc3336eb04a837490b8f332e0db41da66995c9fd1ba04552e516354"}, + {file = "propcache-0.2.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:608cce1da6f2672a56b24a015b42db4ac612ee709f3d29f27a00c943d9e851de"}, + {file = "propcache-0.2.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:466c219deee4536fbc83c08d09115249db301550625c7fef1c5563a584c9bc87"}, + {file = "propcache-0.2.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc2db02409338bf36590aa985a461b2c96fce91f8e7e0f14c50c5fcc4f229016"}, + {file = "propcache-0.2.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a6ed8db0a556343d566a5c124ee483ae113acc9a557a807d439bcecc44e7dfbb"}, + {file = "propcache-0.2.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:91997d9cb4a325b60d4e3f20967f8eb08dfcb32b22554d5ef78e6fd1dda743a2"}, + {file = "propcache-0.2.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4c7dde9e533c0a49d802b4f3f218fa9ad0a1ce21f2c2eb80d5216565202acab4"}, + {file = "propcache-0.2.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ffcad6c564fe6b9b8916c1aefbb37a362deebf9394bd2974e9d84232e3e08504"}, + {file = "propcache-0.2.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:97a58a28bcf63284e8b4d7b460cbee1edaab24634e82059c7b8c09e65284f178"}, + {file = "propcache-0.2.0-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:945db8ee295d3af9dbdbb698cce9bbc5c59b5c3fe328bbc4387f59a8a35f998d"}, + {file = "propcache-0.2.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:39e104da444a34830751715f45ef9fc537475ba21b7f1f5b0f4d71a3b60d7fe2"}, + {file = "propcache-0.2.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:c5ecca8f9bab618340c8e848d340baf68bcd8ad90a8ecd7a4524a81c1764b3db"}, + {file = "propcache-0.2.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:c436130cc779806bdf5d5fae0d848713105472b8566b75ff70048c47d3961c5b"}, + {file = "propcache-0.2.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:191db28dc6dcd29d1a3e063c3be0b40688ed76434622c53a284e5427565bbd9b"}, + {file = "propcache-0.2.0-cp311-cp311-win32.whl", hash = "sha256:5f2564ec89058ee7c7989a7b719115bdfe2a2fb8e7a4543b8d1c0cc4cf6478c1"}, + {file = "propcache-0.2.0-cp311-cp311-win_amd64.whl", hash = "sha256:6e2e54267980349b723cff366d1e29b138b9a60fa376664a157a342689553f71"}, + {file = "propcache-0.2.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:2ee7606193fb267be4b2e3b32714f2d58cad27217638db98a60f9efb5efeccc2"}, + {file = "propcache-0.2.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:91ee8fc02ca52e24bcb77b234f22afc03288e1dafbb1f88fe24db308910c4ac7"}, + {file = "propcache-0.2.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:2e900bad2a8456d00a113cad8c13343f3b1f327534e3589acc2219729237a2e8"}, + {file = "propcache-0.2.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f52a68c21363c45297aca15561812d542f8fc683c85201df0bebe209e349f793"}, + {file = "propcache-0.2.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1e41d67757ff4fbc8ef2af99b338bfb955010444b92929e9e55a6d4dcc3c4f09"}, + {file = "propcache-0.2.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a64e32f8bd94c105cc27f42d3b658902b5bcc947ece3c8fe7bc1b05982f60e89"}, + {file = "propcache-0.2.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:55346705687dbd7ef0d77883ab4f6fabc48232f587925bdaf95219bae072491e"}, + {file = "propcache-0.2.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:00181262b17e517df2cd85656fcd6b4e70946fe62cd625b9d74ac9977b64d8d9"}, + {file = "propcache-0.2.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:6994984550eaf25dd7fc7bd1b700ff45c894149341725bb4edc67f0ffa94efa4"}, + {file = "propcache-0.2.0-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:56295eb1e5f3aecd516d91b00cfd8bf3a13991de5a479df9e27dd569ea23959c"}, + {file = "propcache-0.2.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:439e76255daa0f8151d3cb325f6dd4a3e93043e6403e6491813bcaaaa8733887"}, + {file = "propcache-0.2.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:f6475a1b2ecb310c98c28d271a30df74f9dd436ee46d09236a6b750a7599ce57"}, + {file = "propcache-0.2.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:3444cdba6628accf384e349014084b1cacd866fbb88433cd9d279d90a54e0b23"}, + {file = "propcache-0.2.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:4a9d9b4d0a9b38d1c391bb4ad24aa65f306c6f01b512e10a8a34a2dc5675d348"}, + {file = "propcache-0.2.0-cp312-cp312-win32.whl", hash = "sha256:69d3a98eebae99a420d4b28756c8ce6ea5a29291baf2dc9ff9414b42676f61d5"}, + {file = "propcache-0.2.0-cp312-cp312-win_amd64.whl", hash = "sha256:ad9c9b99b05f163109466638bd30ada1722abb01bbb85c739c50b6dc11f92dc3"}, + {file = "propcache-0.2.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:ecddc221a077a8132cf7c747d5352a15ed763b674c0448d811f408bf803d9ad7"}, + {file = "propcache-0.2.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:0e53cb83fdd61cbd67202735e6a6687a7b491c8742dfc39c9e01e80354956763"}, + {file = "propcache-0.2.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:92fe151145a990c22cbccf9ae15cae8ae9eddabfc949a219c9f667877e40853d"}, + {file = "propcache-0.2.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d6a21ef516d36909931a2967621eecb256018aeb11fc48656e3257e73e2e247a"}, + {file = "propcache-0.2.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3f88a4095e913f98988f5b338c1d4d5d07dbb0b6bad19892fd447484e483ba6b"}, + {file = "propcache-0.2.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5a5b3bb545ead161be780ee85a2b54fdf7092815995661947812dde94a40f6fb"}, + {file = "propcache-0.2.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:67aeb72e0f482709991aa91345a831d0b707d16b0257e8ef88a2ad246a7280bf"}, + {file = "propcache-0.2.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3c997f8c44ec9b9b0bcbf2d422cc00a1d9b9c681f56efa6ca149a941e5560da2"}, + {file = "propcache-0.2.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:2a66df3d4992bc1d725b9aa803e8c5a66c010c65c741ad901e260ece77f58d2f"}, + {file = "propcache-0.2.0-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:3ebbcf2a07621f29638799828b8d8668c421bfb94c6cb04269130d8de4fb7136"}, + {file = "propcache-0.2.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:1235c01ddaa80da8235741e80815ce381c5267f96cc49b1477fdcf8c047ef325"}, + {file = "propcache-0.2.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:3947483a381259c06921612550867b37d22e1df6d6d7e8361264b6d037595f44"}, + {file = "propcache-0.2.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:d5bed7f9805cc29c780f3aee05de3262ee7ce1f47083cfe9f77471e9d6777e83"}, + {file = "propcache-0.2.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:e4a91d44379f45f5e540971d41e4626dacd7f01004826a18cb048e7da7e96544"}, + {file = "propcache-0.2.0-cp313-cp313-win32.whl", hash = "sha256:f902804113e032e2cdf8c71015651c97af6418363bea8d78dc0911d56c335032"}, + {file = "propcache-0.2.0-cp313-cp313-win_amd64.whl", hash = "sha256:8f188cfcc64fb1266f4684206c9de0e80f54622c3f22a910cbd200478aeae61e"}, + {file = "propcache-0.2.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:53d1bd3f979ed529f0805dd35ddaca330f80a9a6d90bc0121d2ff398f8ed8861"}, + {file = "propcache-0.2.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:83928404adf8fb3d26793665633ea79b7361efa0287dfbd372a7e74311d51ee6"}, + {file = "propcache-0.2.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:77a86c261679ea5f3896ec060be9dc8e365788248cc1e049632a1be682442063"}, + {file = "propcache-0.2.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:218db2a3c297a3768c11a34812e63b3ac1c3234c3a086def9c0fee50d35add1f"}, + {file = "propcache-0.2.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:7735e82e3498c27bcb2d17cb65d62c14f1100b71723b68362872bca7d0913d90"}, + {file = "propcache-0.2.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:20a617c776f520c3875cf4511e0d1db847a076d720714ae35ffe0df3e440be68"}, + {file = "propcache-0.2.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:67b69535c870670c9f9b14a75d28baa32221d06f6b6fa6f77a0a13c5a7b0a5b9"}, + {file = "propcache-0.2.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4569158070180c3855e9c0791c56be3ceeb192defa2cdf6a3f39e54319e56b89"}, + {file = "propcache-0.2.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:db47514ffdbd91ccdc7e6f8407aac4ee94cc871b15b577c1c324236b013ddd04"}, + {file = "propcache-0.2.0-cp38-cp38-musllinux_1_2_armv7l.whl", hash = "sha256:2a60ad3e2553a74168d275a0ef35e8c0a965448ffbc3b300ab3a5bb9956c2162"}, + {file = "propcache-0.2.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:662dd62358bdeaca0aee5761de8727cfd6861432e3bb828dc2a693aa0471a563"}, + {file = "propcache-0.2.0-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:25a1f88b471b3bc911d18b935ecb7115dff3a192b6fef46f0bfaf71ff4f12418"}, + {file = "propcache-0.2.0-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:f60f0ac7005b9f5a6091009b09a419ace1610e163fa5deaba5ce3484341840e7"}, + {file = "propcache-0.2.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:74acd6e291f885678631b7ebc85d2d4aec458dd849b8c841b57ef04047833bed"}, + {file = "propcache-0.2.0-cp38-cp38-win32.whl", hash = "sha256:d9b6ddac6408194e934002a69bcaadbc88c10b5f38fb9307779d1c629181815d"}, + {file = "propcache-0.2.0-cp38-cp38-win_amd64.whl", hash = "sha256:676135dcf3262c9c5081cc8f19ad55c8a64e3f7282a21266d05544450bffc3a5"}, + {file = "propcache-0.2.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:25c8d773a62ce0451b020c7b29a35cfbc05de8b291163a7a0f3b7904f27253e6"}, + {file = "propcache-0.2.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:375a12d7556d462dc64d70475a9ee5982465fbb3d2b364f16b86ba9135793638"}, + {file = "propcache-0.2.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:1ec43d76b9677637a89d6ab86e1fef70d739217fefa208c65352ecf0282be957"}, + {file = "propcache-0.2.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f45eec587dafd4b2d41ac189c2156461ebd0c1082d2fe7013571598abb8505d1"}, + {file = "propcache-0.2.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:bc092ba439d91df90aea38168e11f75c655880c12782facf5cf9c00f3d42b562"}, + {file = "propcache-0.2.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:fa1076244f54bb76e65e22cb6910365779d5c3d71d1f18b275f1dfc7b0d71b4d"}, + {file = "propcache-0.2.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:682a7c79a2fbf40f5dbb1eb6bfe2cd865376deeac65acf9beb607505dced9e12"}, + {file = "propcache-0.2.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8e40876731f99b6f3c897b66b803c9e1c07a989b366c6b5b475fafd1f7ba3fb8"}, + {file = "propcache-0.2.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:363ea8cd3c5cb6679f1c2f5f1f9669587361c062e4899fce56758efa928728f8"}, + {file = "propcache-0.2.0-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:140fbf08ab3588b3468932974a9331aff43c0ab8a2ec2c608b6d7d1756dbb6cb"}, + {file = "propcache-0.2.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:e70fac33e8b4ac63dfc4c956fd7d85a0b1139adcfc0d964ce288b7c527537fea"}, + {file = "propcache-0.2.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:b33d7a286c0dc1a15f5fc864cc48ae92a846df287ceac2dd499926c3801054a6"}, + {file = "propcache-0.2.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:f6d5749fdd33d90e34c2efb174c7e236829147a2713334d708746e94c4bde40d"}, + {file = "propcache-0.2.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:22aa8f2272d81d9317ff5756bb108021a056805ce63dd3630e27d042c8092798"}, + {file = "propcache-0.2.0-cp39-cp39-win32.whl", hash = "sha256:73e4b40ea0eda421b115248d7e79b59214411109a5bc47d0d48e4c73e3b8fcf9"}, + {file = "propcache-0.2.0-cp39-cp39-win_amd64.whl", hash = "sha256:9517d5e9e0731957468c29dbfd0f976736a0e55afaea843726e887f36fe017df"}, + {file = "propcache-0.2.0-py3-none-any.whl", hash = "sha256:2ccc28197af5313706511fab3a8b66dcd6da067a1331372c82ea1cb74285e036"}, + {file = "propcache-0.2.0.tar.gz", hash = "sha256:df81779732feb9d01e5d513fad0122efb3d53bbc75f61b2a4f29a020bc985e70"}, +] + [[package]] name = "psutil" version = "6.0.0" @@ -2174,6 +2330,43 @@ files = [ {file = "PyYAML-6.0.tar.gz", hash = "sha256:68fb519c14306fec9720a2a5b45bc9f0c8d1b9c72adf45c37baedfcd949c35a2"}, ] +[[package]] +name = "rasterio" +version = "1.2.8" +description = "Fast and direct raster I/O for use with Numpy and SciPy" +optional = false +python-versions = ">=3.6" +files = [ + {file = "rasterio-1.2.8-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:c1ca9bb81518d369cbeae57aff3608538f61dcf8e12cf4bfe92dd931bc263ade"}, + {file = "rasterio-1.2.8-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:8b13e5f7c57512885bfede605b021bd025684b507dac7b5ac8aee5430930114c"}, + {file = "rasterio-1.2.8-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:84cea32a0675f1b22924664169f0da4be406e790072501d505aa771da9662bca"}, + {file = "rasterio-1.2.8-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:722f76a09091d22fd26200ff5a7820eeb263f1cd22151c77e25a3cc473d5af72"}, + {file = "rasterio-1.2.8-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:74e80044767f41c84121fef373926f0d3e7411b944982429328d11442a649c0a"}, + {file = "rasterio-1.2.8-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:c30caf0823811553ec45865697afc5e4a1fc4c6f30f8dada9067061cd8507af6"}, + {file = "rasterio-1.2.8-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b2d41b39e41a7011d1dba0cb8983336de69ad68fb17857ccbd11cde0fcfdb2e3"}, + {file = "rasterio-1.2.8-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:f248a64f57a0db5b12eb50d9ac1ccd45a6085d979c3e73b406f9e6b5a8355080"}, + {file = "rasterio-1.2.8.tar.gz", hash = "sha256:8196b7a71fea3c1573dd48d9ab0e78955e1d26e81848cce318c4930bd96782fe"}, +] + +[package.dependencies] +affine = "*" +attrs = "*" +certifi = "*" +click = ">=4.0" +click-plugins = "*" +cligj = ">=0.5" +numpy = "*" +setuptools = "*" +snuggs = ">=1.4.1" + +[package.extras] +all = ["boto3 (>=1.2.4)", "ghp-import", "hypothesis", "ipython (>=2.0)", "matplotlib", "numpydoc", "packaging", "pytest (>=2.8.2)", "pytest-cov (>=2.2.0)", "shapely", "sphinx", "sphinx-rtd-theme"] +docs = ["ghp-import", "numpydoc", "sphinx", "sphinx-rtd-theme"] +ipython = ["ipython (>=2.0)"] +plot = ["matplotlib"] +s3 = ["boto3 (>=1.2.4)"] +test = ["boto3 (>=1.2.4)", "hypothesis", "packaging", "pytest (>=2.8.2)", "pytest-cov (>=2.2.0)", "shapely"] + [[package]] name = "requests" version = "2.32.3" @@ -2195,6 +2388,31 @@ urllib3 = ">=1.21.1,<3" socks = ["PySocks (>=1.5.6,!=1.5.7)"] use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"] +[[package]] +name = "rioxarray" +version = "0.13.4" +description = "geospatial xarray extension powered by rasterio" +optional = false +python-versions = ">=3.8" +files = [ + {file = "rioxarray-0.13.4-py3-none-any.whl", hash = "sha256:56eef711d9817d3c729c1a267c940e7dff66bfc874a0b24ed3604ea2f958dfb2"}, + {file = "rioxarray-0.13.4.tar.gz", hash = "sha256:0cad24ad2c3c5ee181a0cfad2b8c2152a609b7eb118a3430034aec171e9cf14f"}, +] + +[package.dependencies] +numpy = ">=1.21" +packaging = "*" +pyproj = ">=2.2" +rasterio = ">=1.1.1" +xarray = ">=0.17" + +[package.extras] +all = ["dask", "mypy", "nbsphinx", "netcdf4", "pre-commit", "pylint", "pytest (>=3.6)", "pytest-cov", "pytest-timeout", "scipy", "sphinx-click", "sphinx-rtd-theme"] +dev = ["dask", "mypy", "nbsphinx", "netcdf4", "pre-commit", "pylint", "pytest (>=3.6)", "pytest-cov", "pytest-timeout", "scipy", "sphinx-click", "sphinx-rtd-theme"] +doc = ["nbsphinx", "sphinx-click", "sphinx-rtd-theme"] +interp = ["scipy"] +test = ["dask", "netcdf4", "pytest (>=3.6)", "pytest-cov", "pytest-timeout"] + [[package]] name = "s3fs" version = "2022.5.0" @@ -2274,6 +2492,26 @@ files = [ [package.dependencies] numpy = ">=1.16.5" +[[package]] +name = "setuptools" +version = "75.1.0" +description = "Easily download, build, install, upgrade, and uninstall Python packages" +optional = false +python-versions = ">=3.8" +files = [ + {file = "setuptools-75.1.0-py3-none-any.whl", hash = "sha256:35ab7fd3bcd95e6b7fd704e4a1539513edad446c097797f2985e0e4b960772f2"}, + {file = "setuptools-75.1.0.tar.gz", hash = "sha256:d59a21b17a275fb872a9c3dae73963160ae079f1049ed956880cd7c09b120538"}, +] + +[package.extras] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)", "ruff (>=0.5.2)"] +core = ["importlib-metadata (>=6)", "importlib-resources (>=5.10.2)", "jaraco.collections", "jaraco.functools", "jaraco.text (>=3.7)", "more-itertools", "more-itertools (>=8.8)", "packaging", "packaging (>=24)", "platformdirs (>=2.6.2)", "tomli (>=2.0.1)", "wheel (>=0.43.0)"] +cover = ["pytest-cov"] +doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "pygments-github-lexers (==0.0.5)", "pyproject-hooks (!=1.1)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-favicon", "sphinx-inline-tabs", "sphinx-lint", "sphinx-notfound-page (>=1,<2)", "sphinx-reredirects", "sphinxcontrib-towncrier", "towncrier (<24.7)"] +enabler = ["pytest-enabler (>=2.2)"] +test = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "ini2toml[lite] (>=0.14)", "jaraco.develop (>=7.21)", "jaraco.envs (>=2.2)", "jaraco.path (>=3.2.0)", "jaraco.test", "packaging (>=23.2)", "pip (>=19.1)", "pyproject-hooks (!=1.1)", "pytest (>=6,!=8.1.*)", "pytest-home (>=0.5)", "pytest-perf", "pytest-subprocess", "pytest-timeout", "pytest-xdist (>=3)", "tomli-w (>=1.0.0)", "virtualenv (>=13.0.0)", "wheel (>=0.44.0)"] +type = ["importlib-metadata (>=7.0.2)", "jaraco.develop (>=7.21)", "mypy (==1.11.*)", "pytest-mypy"] + [[package]] name = "shapely" version = "1.7.1" @@ -2322,15 +2560,33 @@ files = [ {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"}, ] +[[package]] +name = "snuggs" +version = "1.4.7" +description = "Snuggs are s-expressions for Numpy" +optional = false +python-versions = "*" +files = [ + {file = "snuggs-1.4.7-py3-none-any.whl", hash = "sha256:988dde5d4db88e9d71c99457404773dabcc7a1c45971bfbe81900999942d9f07"}, + {file = "snuggs-1.4.7.tar.gz", hash = "sha256:501cf113fe3892e14e2fee76da5cd0606b7e149c411c271898e6259ebde2617b"}, +] + +[package.dependencies] +numpy = "*" +pyparsing = ">=2.1.6" + +[package.extras] +test = ["hypothesis", "pytest"] + [[package]] name = "tomli" -version = "2.0.1" +version = "2.0.2" description = "A lil' TOML parser" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "tomli-2.0.1-py3-none-any.whl", hash = "sha256:939de3e7a6161af0c887ef91b7d41a53e7c5a1ca976325f429cb46ea9bc30ecc"}, - {file = "tomli-2.0.1.tar.gz", hash = "sha256:de526c12914f0c550d15924c62d72abc48d6fe7364aa87328337a31007fe8a4f"}, + {file = "tomli-2.0.2-py3-none-any.whl", hash = "sha256:2ebe24485c53d303f690b0ec092806a085f07af5a5aa1464f3931eec36caaa38"}, + {file = "tomli-2.0.2.tar.gz", hash = "sha256:d46d457a85337051c36524bc5349dd91b1877838e2979ac5ced3e710ed8a60ed"}, ] [[package]] @@ -2346,13 +2602,13 @@ files = [ [[package]] name = "toolz" -version = "0.12.1" +version = "1.0.0" description = "List processing tools and functional utilities" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "toolz-0.12.1-py3-none-any.whl", hash = "sha256:d22731364c07d72eea0a0ad45bafb2c2937ab6fd38a3507bf55eae8744aa7d85"}, - {file = "toolz-0.12.1.tar.gz", hash = "sha256:ecca342664893f177a13dac0e6b41cbd8ac25a358e5f215316d43e2100224f4d"}, + {file = "toolz-1.0.0-py3-none-any.whl", hash = "sha256:292c8f1c4e7516bf9086f8850935c799a874039c8bcf959d47b600e4c44a6236"}, + {file = "toolz-1.0.0.tar.gz", hash = "sha256:2c86e3d9a04798ac556793bced838816296a2f085017664e4995cb40a1047a02"}, ] [[package]] @@ -2418,13 +2674,13 @@ files = [ [[package]] name = "tzdata" -version = "2024.1" +version = "2024.2" description = "Provider of IANA time zone data" optional = false python-versions = ">=2" files = [ - {file = "tzdata-2024.1-py2.py3-none-any.whl", hash = "sha256:9068bc196136463f5245e51efda838afa15aaeca9903f49050dfa2679db4d252"}, - {file = "tzdata-2024.1.tar.gz", hash = "sha256:2674120f8d891909751c38abcdfd386ac0a5a1127954fbc332af6b5ceae07efd"}, + {file = "tzdata-2024.2-py2.py3-none-any.whl", hash = "sha256:a48093786cdcde33cad18c2555e8532f34422074448fbc874186f0abd79565cd"}, + {file = "tzdata-2024.2.tar.gz", hash = "sha256:7d85cc416e9382e69095b7bdf4afd9e3880418a2413feec7069d533d6b4e31cc"}, ] [[package]] @@ -2558,108 +2814,109 @@ viz = ["matplotlib", "nc-time-axis", "seaborn"] [[package]] name = "yarl" -version = "1.11.1" +version = "1.14.0" description = "Yet another URL library" optional = false python-versions = ">=3.8" files = [ - {file = "yarl-1.11.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:400cd42185f92de559d29eeb529e71d80dfbd2f45c36844914a4a34297ca6f00"}, - {file = "yarl-1.11.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:8258c86f47e080a258993eed877d579c71da7bda26af86ce6c2d2d072c11320d"}, - {file = "yarl-1.11.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2164cd9725092761fed26f299e3f276bb4b537ca58e6ff6b252eae9631b5c96e"}, - {file = "yarl-1.11.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a08ea567c16f140af8ddc7cb58e27e9138a1386e3e6e53982abaa6f2377b38cc"}, - {file = "yarl-1.11.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:768ecc550096b028754ea28bf90fde071c379c62c43afa574edc6f33ee5daaec"}, - {file = "yarl-1.11.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2909fa3a7d249ef64eeb2faa04b7957e34fefb6ec9966506312349ed8a7e77bf"}, - {file = "yarl-1.11.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:01a8697ec24f17c349c4f655763c4db70eebc56a5f82995e5e26e837c6eb0e49"}, - {file = "yarl-1.11.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e286580b6511aac7c3268a78cdb861ec739d3e5a2a53b4809faef6b49778eaff"}, - {file = "yarl-1.11.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:4179522dc0305c3fc9782549175c8e8849252fefeb077c92a73889ccbcd508ad"}, - {file = "yarl-1.11.1-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:27fcb271a41b746bd0e2a92182df507e1c204759f460ff784ca614e12dd85145"}, - {file = "yarl-1.11.1-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:f61db3b7e870914dbd9434b560075e0366771eecbe6d2b5561f5bc7485f39efd"}, - {file = "yarl-1.11.1-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:c92261eb2ad367629dc437536463dc934030c9e7caca861cc51990fe6c565f26"}, - {file = "yarl-1.11.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:d95b52fbef190ca87d8c42f49e314eace4fc52070f3dfa5f87a6594b0c1c6e46"}, - {file = "yarl-1.11.1-cp310-cp310-win32.whl", hash = "sha256:489fa8bde4f1244ad6c5f6d11bb33e09cf0d1d0367edb197619c3e3fc06f3d91"}, - {file = "yarl-1.11.1-cp310-cp310-win_amd64.whl", hash = "sha256:476e20c433b356e16e9a141449f25161e6b69984fb4cdbd7cd4bd54c17844998"}, - {file = "yarl-1.11.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:946eedc12895873891aaceb39bceb484b4977f70373e0122da483f6c38faaa68"}, - {file = "yarl-1.11.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:21a7c12321436b066c11ec19c7e3cb9aec18884fe0d5b25d03d756a9e654edfe"}, - {file = "yarl-1.11.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:c35f493b867912f6fda721a59cc7c4766d382040bdf1ddaeeaa7fa4d072f4675"}, - {file = "yarl-1.11.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:25861303e0be76b60fddc1250ec5986c42f0a5c0c50ff57cc30b1be199c00e63"}, - {file = "yarl-1.11.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e4b53f73077e839b3f89c992223f15b1d2ab314bdbdf502afdc7bb18e95eae27"}, - {file = "yarl-1.11.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:327c724b01b8641a1bf1ab3b232fb638706e50f76c0b5bf16051ab65c868fac5"}, - {file = "yarl-1.11.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4307d9a3417eea87715c9736d050c83e8c1904e9b7aada6ce61b46361b733d92"}, - {file = "yarl-1.11.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:48a28bed68ab8fb7e380775f0029a079f08a17799cb3387a65d14ace16c12e2b"}, - {file = "yarl-1.11.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:067b961853c8e62725ff2893226fef3d0da060656a9827f3f520fb1d19b2b68a"}, - {file = "yarl-1.11.1-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:8215f6f21394d1f46e222abeb06316e77ef328d628f593502d8fc2a9117bde83"}, - {file = "yarl-1.11.1-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:498442e3af2a860a663baa14fbf23fb04b0dd758039c0e7c8f91cb9279799bff"}, - {file = "yarl-1.11.1-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:69721b8effdb588cb055cc22f7c5105ca6fdaa5aeb3ea09021d517882c4a904c"}, - {file = "yarl-1.11.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:1e969fa4c1e0b1a391f3fcbcb9ec31e84440253325b534519be0d28f4b6b533e"}, - {file = "yarl-1.11.1-cp311-cp311-win32.whl", hash = "sha256:7d51324a04fc4b0e097ff8a153e9276c2593106a811704025bbc1d6916f45ca6"}, - {file = "yarl-1.11.1-cp311-cp311-win_amd64.whl", hash = "sha256:15061ce6584ece023457fb8b7a7a69ec40bf7114d781a8c4f5dcd68e28b5c53b"}, - {file = "yarl-1.11.1-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:a4264515f9117be204935cd230fb2a052dd3792789cc94c101c535d349b3dab0"}, - {file = "yarl-1.11.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:f41fa79114a1d2eddb5eea7b912d6160508f57440bd302ce96eaa384914cd265"}, - {file = "yarl-1.11.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:02da8759b47d964f9173c8675710720b468aa1c1693be0c9c64abb9d8d9a4867"}, - {file = "yarl-1.11.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9361628f28f48dcf8b2f528420d4d68102f593f9c2e592bfc842f5fb337e44fd"}, - {file = "yarl-1.11.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b91044952da03b6f95fdba398d7993dd983b64d3c31c358a4c89e3c19b6f7aef"}, - {file = "yarl-1.11.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:74db2ef03b442276d25951749a803ddb6e270d02dda1d1c556f6ae595a0d76a8"}, - {file = "yarl-1.11.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7e975a2211952a8a083d1b9d9ba26472981ae338e720b419eb50535de3c02870"}, - {file = "yarl-1.11.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8aef97ba1dd2138112890ef848e17d8526fe80b21f743b4ee65947ea184f07a2"}, - {file = "yarl-1.11.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:a7915ea49b0c113641dc4d9338efa9bd66b6a9a485ffe75b9907e8573ca94b84"}, - {file = "yarl-1.11.1-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:504cf0d4c5e4579a51261d6091267f9fd997ef58558c4ffa7a3e1460bd2336fa"}, - {file = "yarl-1.11.1-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:3de5292f9f0ee285e6bd168b2a77b2a00d74cbcfa420ed078456d3023d2f6dff"}, - {file = "yarl-1.11.1-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:a34e1e30f1774fa35d37202bbeae62423e9a79d78d0874e5556a593479fdf239"}, - {file = "yarl-1.11.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:66b63c504d2ca43bf7221a1f72fbe981ff56ecb39004c70a94485d13e37ebf45"}, - {file = "yarl-1.11.1-cp312-cp312-win32.whl", hash = "sha256:a28b70c9e2213de425d9cba5ab2e7f7a1c8ca23a99c4b5159bf77b9c31251447"}, - {file = "yarl-1.11.1-cp312-cp312-win_amd64.whl", hash = "sha256:17b5a386d0d36fb828e2fb3ef08c8829c1ebf977eef88e5367d1c8c94b454639"}, - {file = "yarl-1.11.1-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:1fa2e7a406fbd45b61b4433e3aa254a2c3e14c4b3186f6e952d08a730807fa0c"}, - {file = "yarl-1.11.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:750f656832d7d3cb0c76be137ee79405cc17e792f31e0a01eee390e383b2936e"}, - {file = "yarl-1.11.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0b8486f322d8f6a38539136a22c55f94d269addb24db5cb6f61adc61eabc9d93"}, - {file = "yarl-1.11.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3fce4da3703ee6048ad4138fe74619c50874afe98b1ad87b2698ef95bf92c96d"}, - {file = "yarl-1.11.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8ed653638ef669e0efc6fe2acb792275cb419bf9cb5c5049399f3556995f23c7"}, - {file = "yarl-1.11.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:18ac56c9dd70941ecad42b5a906820824ca72ff84ad6fa18db33c2537ae2e089"}, - {file = "yarl-1.11.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:688654f8507464745ab563b041d1fb7dab5d9912ca6b06e61d1c4708366832f5"}, - {file = "yarl-1.11.1-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4973eac1e2ff63cf187073cd4e1f1148dcd119314ab79b88e1b3fad74a18c9d5"}, - {file = "yarl-1.11.1-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:964a428132227edff96d6f3cf261573cb0f1a60c9a764ce28cda9525f18f7786"}, - {file = "yarl-1.11.1-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:6d23754b9939cbab02c63434776df1170e43b09c6a517585c7ce2b3d449b7318"}, - {file = "yarl-1.11.1-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:c2dc4250fe94d8cd864d66018f8344d4af50e3758e9d725e94fecfa27588ff82"}, - {file = "yarl-1.11.1-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:09696438cb43ea6f9492ef237761b043f9179f455f405279e609f2bc9100212a"}, - {file = "yarl-1.11.1-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:999bfee0a5b7385a0af5ffb606393509cfde70ecca4f01c36985be6d33e336da"}, - {file = "yarl-1.11.1-cp313-cp313-win32.whl", hash = "sha256:ce928c9c6409c79e10f39604a7e214b3cb69552952fbda8d836c052832e6a979"}, - {file = "yarl-1.11.1-cp313-cp313-win_amd64.whl", hash = "sha256:501c503eed2bb306638ccb60c174f856cc3246c861829ff40eaa80e2f0330367"}, - {file = "yarl-1.11.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:dae7bd0daeb33aa3e79e72877d3d51052e8b19c9025ecf0374f542ea8ec120e4"}, - {file = "yarl-1.11.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:3ff6b1617aa39279fe18a76c8d165469c48b159931d9b48239065767ee455b2b"}, - {file = "yarl-1.11.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:3257978c870728a52dcce8c2902bf01f6c53b65094b457bf87b2644ee6238ddc"}, - {file = "yarl-1.11.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0f351fa31234699d6084ff98283cb1e852270fe9e250a3b3bf7804eb493bd937"}, - {file = "yarl-1.11.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8aef1b64da41d18026632d99a06b3fefe1d08e85dd81d849fa7c96301ed22f1b"}, - {file = "yarl-1.11.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:7175a87ab8f7fbde37160a15e58e138ba3b2b0e05492d7351314a250d61b1591"}, - {file = "yarl-1.11.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ba444bdd4caa2a94456ef67a2f383710928820dd0117aae6650a4d17029fa25e"}, - {file = "yarl-1.11.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0ea9682124fc062e3d931c6911934a678cb28453f957ddccf51f568c2f2b5e05"}, - {file = "yarl-1.11.1-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:8418c053aeb236b20b0ab8fa6bacfc2feaaf7d4683dd96528610989c99723d5f"}, - {file = "yarl-1.11.1-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:61a5f2c14d0a1adfdd82258f756b23a550c13ba4c86c84106be4c111a3a4e413"}, - {file = "yarl-1.11.1-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:f3a6d90cab0bdf07df8f176eae3a07127daafcf7457b997b2bf46776da2c7eb7"}, - {file = "yarl-1.11.1-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:077da604852be488c9a05a524068cdae1e972b7dc02438161c32420fb4ec5e14"}, - {file = "yarl-1.11.1-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:15439f3c5c72686b6c3ff235279630d08936ace67d0fe5c8d5bbc3ef06f5a420"}, - {file = "yarl-1.11.1-cp38-cp38-win32.whl", hash = "sha256:238a21849dd7554cb4d25a14ffbfa0ef380bb7ba201f45b144a14454a72ffa5a"}, - {file = "yarl-1.11.1-cp38-cp38-win_amd64.whl", hash = "sha256:67459cf8cf31da0e2cbdb4b040507e535d25cfbb1604ca76396a3a66b8ba37a6"}, - {file = "yarl-1.11.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:884eab2ce97cbaf89f264372eae58388862c33c4f551c15680dd80f53c89a269"}, - {file = "yarl-1.11.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8a336eaa7ee7e87cdece3cedb395c9657d227bfceb6781295cf56abcd3386a26"}, - {file = "yarl-1.11.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:87f020d010ba80a247c4abc335fc13421037800ca20b42af5ae40e5fd75e7909"}, - {file = "yarl-1.11.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:637c7ddb585a62d4469f843dac221f23eec3cbad31693b23abbc2c366ad41ff4"}, - {file = "yarl-1.11.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:48dfd117ab93f0129084577a07287376cc69c08138694396f305636e229caa1a"}, - {file = "yarl-1.11.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:75e0ae31fb5ccab6eda09ba1494e87eb226dcbd2372dae96b87800e1dcc98804"}, - {file = "yarl-1.11.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f46f81501160c28d0c0b7333b4f7be8983dbbc161983b6fb814024d1b4952f79"}, - {file = "yarl-1.11.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:04293941646647b3bfb1719d1d11ff1028e9c30199509a844da3c0f5919dc520"}, - {file = "yarl-1.11.1-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:250e888fa62d73e721f3041e3a9abf427788a1934b426b45e1b92f62c1f68366"}, - {file = "yarl-1.11.1-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:e8f63904df26d1a66aabc141bfd258bf738b9bc7bc6bdef22713b4f5ef789a4c"}, - {file = "yarl-1.11.1-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:aac44097d838dda26526cffb63bdd8737a2dbdf5f2c68efb72ad83aec6673c7e"}, - {file = "yarl-1.11.1-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:267b24f891e74eccbdff42241c5fb4f974de2d6271dcc7d7e0c9ae1079a560d9"}, - {file = "yarl-1.11.1-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:6907daa4b9d7a688063ed098c472f96e8181733c525e03e866fb5db480a424df"}, - {file = "yarl-1.11.1-cp39-cp39-win32.whl", hash = "sha256:14438dfc5015661f75f85bc5adad0743678eefee266ff0c9a8e32969d5d69f74"}, - {file = "yarl-1.11.1-cp39-cp39-win_amd64.whl", hash = "sha256:94d0caaa912bfcdc702a4204cd5e2bb01eb917fc4f5ea2315aa23962549561b0"}, - {file = "yarl-1.11.1-py3-none-any.whl", hash = "sha256:72bf26f66456baa0584eff63e44545c9f0eaed9b73cb6601b647c91f14c11f38"}, - {file = "yarl-1.11.1.tar.gz", hash = "sha256:1bb2d9e212fb7449b8fb73bc461b51eaa17cc8430b4a87d87be7b25052d92f53"}, + {file = "yarl-1.14.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:1bfc25aa6a7c99cf86564210f79a0b7d4484159c67e01232b116e445b3036547"}, + {file = "yarl-1.14.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0cf21f46a15d445417de8fc89f2568852cf57fe8ca1ab3d19ddb24d45c0383ae"}, + {file = "yarl-1.14.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:1dda53508df0de87b6e6b0a52d6718ff6c62a5aca8f5552748404963df639269"}, + {file = "yarl-1.14.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:587c3cc59bc148a9b1c07a019346eda2549bc9f468acd2f9824d185749acf0a6"}, + {file = "yarl-1.14.0-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3007a5b75cb50140708420fe688c393e71139324df599434633019314ceb8b59"}, + {file = "yarl-1.14.0-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:06ff23462398333c78b6f4f8d3d70410d657a471c2c5bbe6086133be43fc8f1a"}, + {file = "yarl-1.14.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:689a99a42ee4583fcb0d3a67a0204664aa1539684aed72bdafcbd505197a91c4"}, + {file = "yarl-1.14.0-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b0547ab1e9345dc468cac8368d88ea4c5bd473ebc1d8d755347d7401982b5dd8"}, + {file = "yarl-1.14.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:742aef0a99844faaac200564ea6f5e08facb285d37ea18bd1a5acf2771f3255a"}, + {file = "yarl-1.14.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:176110bff341b6730f64a1eb3a7070e12b373cf1c910a9337e7c3240497db76f"}, + {file = "yarl-1.14.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:46a9772a1efa93f9cd170ad33101c1817c77e0e9914d4fe33e2da299d7cf0f9b"}, + {file = "yarl-1.14.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:ee2c68e4f2dd1b1c15b849ba1c96fac105fca6ffdb7c1e8be51da6fabbdeafb9"}, + {file = "yarl-1.14.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:047b258e00b99091b6f90355521f026238c63bd76dcf996d93527bb13320eefd"}, + {file = "yarl-1.14.0-cp310-cp310-win32.whl", hash = "sha256:0aa92e3e30a04f9462a25077db689c4ac5ea9ab6cc68a2e563881b987d42f16d"}, + {file = "yarl-1.14.0-cp310-cp310-win_amd64.whl", hash = "sha256:d9baec588f015d0ee564057aa7574313c53a530662ffad930b7886becc85abdf"}, + {file = "yarl-1.14.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:07f9eaf57719d6721ab15805d85f4b01a5b509a0868d7320134371bcb652152d"}, + {file = "yarl-1.14.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:c14b504a74e58e2deb0378b3eca10f3d076635c100f45b113c18c770b4a47a50"}, + {file = "yarl-1.14.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:16a682a127930f3fc4e42583becca6049e1d7214bcad23520c590edd741d2114"}, + {file = "yarl-1.14.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:73bedd2be05f48af19f0f2e9e1353921ce0c83f4a1c9e8556ecdcf1f1eae4892"}, + {file = "yarl-1.14.0-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f3ab950f8814f3b7b5e3eebc117986f817ec933676f68f0a6c5b2137dd7c9c69"}, + {file = "yarl-1.14.0-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b693c63e7e64b524f54aa4888403c680342d1ad0d97be1707c531584d6aeeb4f"}, + {file = "yarl-1.14.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:85cb3e40eaa98489f1e2e8b29f5ad02ee1ee40d6ce6b88d50cf0f205de1d9d2c"}, + {file = "yarl-1.14.0-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4f24f08b6c9b9818fd80612c97857d28f9779f0d1211653ece9844fc7b414df2"}, + {file = "yarl-1.14.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:29a84a46ec3ebae7a1c024c055612b11e9363a8a23238b3e905552d77a2bc51b"}, + {file = "yarl-1.14.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:5cd5dad8366e0168e0fd23d10705a603790484a6dbb9eb272b33673b8f2cce72"}, + {file = "yarl-1.14.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:a152751af7ef7b5d5fa6d215756e508dd05eb07d0cf2ba51f3e740076aa74373"}, + {file = "yarl-1.14.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:3d569f877ed9a708e4c71a2d13d2940cb0791da309f70bd970ac1a5c088a0a92"}, + {file = "yarl-1.14.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:6a615cad11ec3428020fb3c5a88d85ce1b5c69fd66e9fcb91a7daa5e855325dd"}, + {file = "yarl-1.14.0-cp311-cp311-win32.whl", hash = "sha256:bab03192091681d54e8225c53f270b0517637915d9297028409a2a5114ff4634"}, + {file = "yarl-1.14.0-cp311-cp311-win_amd64.whl", hash = "sha256:985623575e5c4ea763056ffe0e2d63836f771a8c294b3de06d09480538316b13"}, + {file = "yarl-1.14.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:fc2c80bc87fba076e6cbb926216c27fba274dae7100a7b9a0983b53132dd99f2"}, + {file = "yarl-1.14.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:55c144d363ad4626ca744556c049c94e2b95096041ac87098bb363dcc8635e8d"}, + {file = "yarl-1.14.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:b03384eed107dbeb5f625a99dc3a7de8be04fc8480c9ad42fccbc73434170b20"}, + {file = "yarl-1.14.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f72a0d746d38cb299b79ce3d4d60ba0892c84bbc905d0d49c13df5bace1b65f8"}, + {file = "yarl-1.14.0-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8648180b34faaea4aa5b5ca7e871d9eb1277033fa439693855cf0ea9195f85f1"}, + {file = "yarl-1.14.0-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9557c9322aaa33174d285b0c1961fb32499d65ad1866155b7845edc876c3c835"}, + {file = "yarl-1.14.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f50eb3837012a937a2b649ec872b66ba9541ad9d6f103ddcafb8231cfcafd22"}, + {file = "yarl-1.14.0-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8892fa575ac9b1b25fae7b221bc4792a273877b9b56a99ee2d8d03eeb3dbb1d2"}, + {file = "yarl-1.14.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:e6a2c5c5bb2556dfbfffffc2bcfb9c235fd2b566d5006dfb2a37afc7e3278a07"}, + {file = "yarl-1.14.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:ab3abc0b78a5dfaa4795a6afbe7b282b6aa88d81cf8c1bb5e394993d7cae3457"}, + {file = "yarl-1.14.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:47eede5d11d669ab3759b63afb70d28d5328c14744b8edba3323e27dc52d298d"}, + {file = "yarl-1.14.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:fe4d2536c827f508348d7b40c08767e8c7071614250927233bf0c92170451c0a"}, + {file = "yarl-1.14.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:0fd7b941dd1b00b5f0acb97455fea2c4b7aac2dd31ea43fb9d155e9bc7b78664"}, + {file = "yarl-1.14.0-cp312-cp312-win32.whl", hash = "sha256:99ff3744f5fe48288be6bc402533b38e89749623a43208e1d57091fc96b783b9"}, + {file = "yarl-1.14.0-cp312-cp312-win_amd64.whl", hash = "sha256:1ca3894e9e9f72da93544f64988d9c052254a338a9f855165f37f51edb6591de"}, + {file = "yarl-1.14.0-cp313-cp313-macosx_10_13_universal2.whl", hash = "sha256:5d02d700705d67e09e1f57681f758f0b9d4412eeb70b2eb8d96ca6200b486db3"}, + {file = "yarl-1.14.0-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:30600ba5db60f7c0820ef38a2568bb7379e1418ecc947a0f76fd8b2ff4257a97"}, + {file = "yarl-1.14.0-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:e85d86527baebb41a214cc3b45c17177177d900a2ad5783dbe6f291642d4906f"}, + {file = "yarl-1.14.0-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:37001e5d4621cef710c8dc1429ca04e189e572f128ab12312eab4e04cf007132"}, + {file = "yarl-1.14.0-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f4f4547944d4f5cfcdc03f3f097d6f05bbbc915eaaf80a2ee120d0e756de377d"}, + {file = "yarl-1.14.0-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:75ff4c819757f9bdb35de049a509814d6ce851fe26f06eb95a392a5640052482"}, + {file = "yarl-1.14.0-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:68ac1a09392ed6e3fd14be880d39b951d7b981fd135416db7d18a6208c536561"}, + {file = "yarl-1.14.0-cp313-cp313-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:96952f642ac69075e44c7d0284528938fdff39422a1d90d3e45ce40b72e5e2d9"}, + {file = "yarl-1.14.0-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:a56fbe3d7f3bce1d060ea18d2413a2ca9ca814eea7cedc4d247b5f338d54844e"}, + {file = "yarl-1.14.0-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:7e2637d75e92763d1322cb5041573279ec43a80c0f7fbbd2d64f5aee98447b17"}, + {file = "yarl-1.14.0-cp313-cp313-musllinux_1_2_ppc64le.whl", hash = "sha256:9abe80ae2c9d37c17599557b712e6515f4100a80efb2cda15f5f070306477cd2"}, + {file = "yarl-1.14.0-cp313-cp313-musllinux_1_2_s390x.whl", hash = "sha256:217a782020b875538eebf3948fac3a7f9bbbd0fd9bf8538f7c2ad7489e80f4e8"}, + {file = "yarl-1.14.0-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:b9cfef3f14f75bf6aba73a76caf61f9d00865912a04a4393c468a7ce0981b519"}, + {file = "yarl-1.14.0-cp313-cp313-win32.whl", hash = "sha256:d8361c7d04e6a264481f0b802e395f647cd3f8bbe27acfa7c12049efea675bd1"}, + {file = "yarl-1.14.0-cp313-cp313-win_amd64.whl", hash = "sha256:bc24f968b82455f336b79bf37dbb243b7d76cd40897489888d663d4e028f5069"}, + {file = "yarl-1.14.0-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:91d875f75fabf76b3018c5f196bf3d308ed2b49ddcb46c1576d6b075754a1393"}, + {file = "yarl-1.14.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:4009def9be3a7e5175db20aa2d7307ecd00bbf50f7f0f989300710eee1d0b0b9"}, + {file = "yarl-1.14.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:582cedde49603f139be572252a318b30dc41039bc0b8165f070f279e5d12187f"}, + {file = "yarl-1.14.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dbd9ff43a04f8ffe8a959a944c2dca10d22f5f99fc6a459f49c3ebfb409309d9"}, + {file = "yarl-1.14.0-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b9f805e37ed16cc212fdc538a608422d7517e7faf539bedea4fe69425bc55d76"}, + {file = "yarl-1.14.0-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:95e16e9eaa2d7f5d87421b8fe694dd71606aa61d74b824c8d17fc85cc51983d1"}, + {file = "yarl-1.14.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:816d24f584edefcc5ca63428f0b38fee00b39fe64e3c5e558f895a18983efe96"}, + {file = "yarl-1.14.0-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:cd2660c01367eb3ef081b8fa0a5da7fe767f9427aa82023a961a5f28f0d4af6c"}, + {file = "yarl-1.14.0-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:94b2bb9bcfd5be9d27004ea4398fb640373dd0c1a9e219084f42c08f77a720ab"}, + {file = "yarl-1.14.0-cp38-cp38-musllinux_1_2_i686.whl", hash = "sha256:c2089a9afef887664115f7fa6d3c0edd6454adaca5488dba836ca91f60401075"}, + {file = "yarl-1.14.0-cp38-cp38-musllinux_1_2_ppc64le.whl", hash = "sha256:2192f718db4a8509f63dd6d950f143279211fa7e6a2c612edc17d85bf043d36e"}, + {file = "yarl-1.14.0-cp38-cp38-musllinux_1_2_s390x.whl", hash = "sha256:8385ab36bf812e9d37cf7613999a87715f27ef67a53f0687d28c44b819df7cb0"}, + {file = "yarl-1.14.0-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:b4c1ecba93e7826dc71ddba75fb7740cdb52e7bd0be9f03136b83f54e6a1f511"}, + {file = "yarl-1.14.0-cp38-cp38-win32.whl", hash = "sha256:e749af6c912a7bb441d105c50c1a3da720474e8acb91c89350080dd600228f0e"}, + {file = "yarl-1.14.0-cp38-cp38-win_amd64.whl", hash = "sha256:147e36331f6f63e08a14640acf12369e041e0751bb70d9362df68c2d9dcf0c87"}, + {file = "yarl-1.14.0-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:7a9f917966d27f7ce30039fe8d900f913c5304134096554fd9bea0774bcda6d1"}, + {file = "yarl-1.14.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8a2f8fb7f944bcdfecd4e8d855f84c703804a594da5123dd206f75036e536d4d"}, + {file = "yarl-1.14.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:8f4e475f29a9122f908d0f1f706e1f2fc3656536ffd21014ff8a6f2e1b14d1d8"}, + {file = "yarl-1.14.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8089d4634d8fa2b1806ce44fefa4979b1ab2c12c0bc7ef3dfa45c8a374811348"}, + {file = "yarl-1.14.0-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1b16f6c75cffc2dc0616ea295abb0e1967601bd1fb1e0af6a1de1c6c887f3439"}, + {file = "yarl-1.14.0-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:498b3c55087b9d762636bca9b45f60d37e51d24341786dc01b81253f9552a607"}, + {file = "yarl-1.14.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e3f8bfc1db82589ef965ed234b87de30d140db8b6dc50ada9e33951ccd8ec07a"}, + {file = "yarl-1.14.0-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:625f207b1799e95e7c823f42f473c1e9dbfb6192bd56bba8695656d92be4535f"}, + {file = "yarl-1.14.0-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:781e2495e408a81e4eaeedeb41ba32b63b1980dddf8b60dbbeff6036bcd35049"}, + {file = "yarl-1.14.0-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:659603d26d40dd4463200df9bfbc339fbfaed3fe32e5c432fe1dc2b5d4aa94b4"}, + {file = "yarl-1.14.0-cp39-cp39-musllinux_1_2_ppc64le.whl", hash = "sha256:4e0d45ebf975634468682c8bec021618b3ad52c37619e5c938f8f831fa1ac5c0"}, + {file = "yarl-1.14.0-cp39-cp39-musllinux_1_2_s390x.whl", hash = "sha256:a2e4725a08cb2b4794db09e350c86dee18202bb8286527210e13a1514dc9a59a"}, + {file = "yarl-1.14.0-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:19268b4fec1d7760134f2de46ef2608c2920134fb1fa61e451f679e41356dc55"}, + {file = "yarl-1.14.0-cp39-cp39-win32.whl", hash = "sha256:337912bcdcf193ade64b9aae5a4017a0a1950caf8ca140362e361543c6773f21"}, + {file = "yarl-1.14.0-cp39-cp39-win_amd64.whl", hash = "sha256:b6d0147574ce2e7b812c989e50fa72bbc5338045411a836bd066ce5fc8ac0bce"}, + {file = "yarl-1.14.0-py3-none-any.whl", hash = "sha256:c8ed4034f0765f8861620c1f2f2364d2e58520ea288497084dae880424fc0d9f"}, + {file = "yarl-1.14.0.tar.gz", hash = "sha256:88c7d9d58aab0724b979ab5617330acb1c7030b79379c8138c1c8c94e121d1b3"}, ] [package.dependencies] idna = ">=2.0" multidict = ">=4.0" +propcache = ">=0.2.0" [[package]] name = "zarr" @@ -2703,4 +2960,4 @@ type = ["pytest-mypy"] [metadata] lock-version = "2.0" python-versions = "~=3.8" -content-hash = "2bdf3a4ef2a7f9b982ad43dd9806fcc8fa3c450c994fa48c8d776c3a963af56b" +content-hash = "092a8232d65317934fe8aef1cd75b46977932d787222dce8540855ca47625e51" diff --git a/pyproject.toml b/pyproject.toml index cc6c4f60..0f3795d7 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -69,6 +69,8 @@ zarr = "2.11.3" pandas = "<2.1.0rc0" matplotlib = "^3.7" psutil = "*" +rasterio = "1.2.8" +rioxarray = "*" [tool.poetry.dev-dependencies]