diff --git a/piker/brokers/deribit/api.py b/piker/brokers/deribit/api.py index 3b19ff59..f846a5c0 100644 --- a/piker/brokers/deribit/api.py +++ b/piker/brokers/deribit/api.py @@ -28,6 +28,8 @@ from decimal import ( Decimal, ) from functools import partial +from pathlib import Path +from pprint import pformat import time from typing import ( Any, @@ -37,8 +39,6 @@ from typing import ( from pendulum import now import trio -from trio_typing import TaskStatus -from rapidfuzz import process as fuzzy import numpy as np from tractor.trionics import ( broadcast_receiver, @@ -55,8 +55,11 @@ from cryptofeed.defines import ( OPTION, CALL, PUT ) from cryptofeed.symbols import Symbol -# types for managing the cb callbacks. -# from cryptofeed.types import L1Book +from cryptofeed.types import ( + L1Book, + Trade, +) +from piker.brokers import SymbolNotFound from .venues import ( _ws_url, MarketType, @@ -64,9 +67,7 @@ from .venues import ( Pair, OptionPair, JSONRPCResult, - JSONRPCChannel, KLinesResult, - Trade, LastTradesResult, ) from piker.accounting import ( @@ -77,7 +78,7 @@ from piker.accounting import ( from piker.data import ( def_iohlcv_fields, match_from_pairs, - Struct, + # Struct, ) from piker.data._web_bs import ( open_jsonrpc_session @@ -96,9 +97,17 @@ _spawn_kwargs = { } -# convert datetime obj timestamp to unixtime in milliseconds -def deribit_timestamp(when): - return int((when.timestamp() * 1000) + (when.microsecond / 1000)) +def deribit_timestamp(when: datetime) -> int: + ''' + Convert conventional epoch timestamp, in secs, to unixtime in + milliseconds. + + ''' + return int( + (when.timestamp() * 1000) + + + (when.microsecond / 1000) + ) def str_to_cb_sym(name: str) -> Symbol: @@ -121,14 +130,20 @@ def str_to_cb_sym(name: str) -> Symbol: type=OPTION, strike_price=strike_price, option_type=option_type, - expiry_date=new_expiry_date) + expiry_date=new_expiry_date + ) def piker_sym_to_cb_sym(name: str) -> Symbol: - base, expiry_date, strike_price, option_type = tuple( + ( + base, + expiry_date, + strike_price, + option_type, + )= tuple( name.upper().split('-')) - quote = base + quote: str = base if option_type == 'P': option_type = PUT @@ -143,14 +158,32 @@ def piker_sym_to_cb_sym(name: str) -> Symbol: type=OPTION, strike_price=strike_price, option_type=option_type, - expiry_date=expiry_date) + expiry_date=expiry_date + ) -def cb_sym_to_deribit_inst(sym: Symbol): +# TODO, instead can't we just lookup the `MktPair` directly +# and pass it upward to `stream_quotes()`?? +def cb_sym_to_deribit_inst(sym: Symbol) -> str: + ''' + Generate our own internal `str`-repr for a `cryptofeed.Symbol` + uniquely from its fields. + + This is the equiv of generating a `Pair.fmqe` from `cryptofeed` + for now i suppose..? + + ''' new_expiry_date = get_values_from_cb_normalized_date(sym.expiry_date) - otype = 'C' if sym.option_type == CALL else 'P' - - return f'{sym.base}-{new_expiry_date}-{sym.strike_price}-{otype}' + otype = ( + 'C' if sym.option_type == CALL + else 'P' + ) + return ( + f'{sym.base}-' + f'{new_expiry_date}-' + f'{sym.strike_price}-' + f'{otype}' + ) def get_values_from_cb_normalized_date(expiry_date: str) -> str: @@ -179,16 +212,18 @@ def get_config() -> dict[str, Any]: conf: dict path: Path - conf, path = config.load( conf_name='brokers', touch_if_dne=True, ) - section: dict = {} - section = conf.get('deribit') + section: dict|None = conf.get('deribit') if section is None: - log.warning(f'No config section found for deribit in {path}') - return {} + raise ValueError( + f'No `[deribit]` section found in\n' + f'{path!r}\n\n' + f'See the template config from the core repo for samples..\n' + # f'' + ) conf_option = section.get('option', {}) section.clear # clear the dict to reuse it @@ -204,7 +239,10 @@ def get_config() -> dict[str, Any]: class Client: + ''' + Hi-level interface for the jsron-RPC over websocket API. + ''' def __init__( self, @@ -223,8 +261,12 @@ class Client: self._auth_ts = None self._auth_renew_ts = 5 # seconds to renew auth - async def _json_rpc_auth_wrapper(self, *args, **kwargs) -> JSONRPCResult: - + async def _json_rpc_auth_wrapper( + self, + *args, + **kwargs, + ) -> JSONRPCResult: + """Background task that adquires a first access token and then will refresh the access token. @@ -250,9 +292,6 @@ class Client: return await self.json_rpc(*args, **kwargs) - - - async def get_balances( self, kind: str = 'option' @@ -277,23 +316,29 @@ class Client: venue: str | None = None, ) -> dict[str, Asset]: - """Return the set of asset balances for this account - by symbol. - """ + ''' + Return the set of asset balances for this account + by (deribit's) symbol. + + + ''' assets = {} resp = await self._json_rpc_auth_wrapper( 'public/get_currencies', params={} ) - currencies = resp.result + currencies: list[dict] = resp.result for currency in currencies: - name = currency['currency'] - tx_tick = digits_to_dec(currency['fee_precision']) - atype='crypto_currency' + name: str = currency['currency'] + tx_tick: Decimal = digits_to_dec(currency['fee_precision']) + + # TODO, handling of options, futures, perps etc. more + # specifically with diff `.atype`s? assets[name] = Asset( name=name, - atype=atype, - tx_tick=tx_tick) + atype='crypto_currency', + tx_tick=tx_tick, + ) instruments = await self.symbol_info(currency=name) for instrument in instruments: @@ -301,9 +346,10 @@ class Client: assets[pair.symbol] = Asset( name=pair.symbol, atype=pair.venue, - tx_tick=pair.size_tick) + tx_tick=pair.size_tick, + ) - return assets + return assets async def get_mkt_pairs(self) -> dict[str, Pair]: flat: dict[str, Pair] = {} @@ -358,6 +404,19 @@ class Client: return cached_pair if sym: + opt: OptionPair|None = pair_table.get(sym) + if not opt: + closest_matches: dict[str, Pair] = match_from_pairs( + pairs=pair_table, + query=sym, + score_cutoff=40, + ) + closest_syms: list[str] = list(closest_matches.keys()) + raise ValueError( + f'No contract found for {sym!r}\n\n' + f'Closest {len(closest_syms)} available contracts:\n\n' + f'{pformat(closest_syms)}\n' + ) return pair_table[sym] else: return self._pairs @@ -381,7 +440,7 @@ class Client: params: dict[str, str] = { 'currency': currency.upper(), 'kind': kind, - 'expired': str(expired).lower() + 'expired': expired, } resp: JSONRPCResult = await self._json_rpc_auth_wrapper( @@ -389,9 +448,9 @@ class Client: params, ) # convert to symbol-keyed table - pair_type: Type = PAIRTYPES[kind] + pair_type: Pair = PAIRTYPES[kind] results: list[dict] | None = resp.result - + instruments: dict[str, Pair] = {} for item in results: symbol=item['instrument_name'].lower() @@ -427,12 +486,15 @@ class Client: mkt_pairs = await self.symbol_info() if not mkt_pairs: - raise SymbolNotFound(f'No market pairs found!?:\n{resp}') + raise SymbolNotFound( + f'No market pairs found!?:\n' + f'{mkt_pairs}' + ) pairs_view_subtable: dict[str, Pair] = {} for instrument in mkt_pairs: - pair_type: Type = PAIRTYPES[venue] + pair_type: Pair|OptionPair = PAIRTYPES[venue] pair: Pair = pair_type(**mkt_pairs[instrument].to_dict()) @@ -480,12 +542,14 @@ class Client: if end_dt is None: end_dt = now('UTC') + _orig_start_dt = start_dt if start_dt is None: start_dt = end_dt.start_of( - 'minute').subtract(minutes=limit) + 'minute' + ).subtract(minutes=limit) - start_time = deribit_timestamp(start_dt) - end_time = deribit_timestamp(end_dt) + start_time: int = deribit_timestamp(start_dt) + end_time: int = deribit_timestamp(end_dt) # https://docs.deribit.com/#public-get_tradingview_chart_data resp = await self._json_rpc_auth_wrapper( @@ -499,9 +563,13 @@ class Client: result = KLinesResult(**resp.result) new_bars: list[tuple] = [] - for i in range(len(result.close)): + # if _orig_start_dt is None: + # if not new_bars: + # import tractor + # await tractor.pause() - row = [ + for i in range(len(result.close)): + row = [ (start_time + (i * (60 * 1000))) / 1000.0, # time result.open[i], result.high[i], @@ -554,7 +622,7 @@ async def get_client( @acm -async def open_feed_handler(): +async def open_feed_handler() -> FeedHandler: fh = FeedHandler(config=get_config()) yield fh await to_asyncio.run_task(fh.stop_async) @@ -575,43 +643,37 @@ async def aio_price_feed_relay( from_trio: asyncio.Queue, to_trio: trio.abc.SendChannel, ) -> None: - async def _trade(data: dict, receipt_timestamp): - to_trio.send_nowait(('trade', { - 'symbol': cb_sym_to_deribit_inst( - str_to_cb_sym(data.symbol)).lower(), - 'last': data, - 'broker_ts': time.time(), - 'data': data.to_dict(), - 'receipt': receipt_timestamp - })) + ''' + Relay price feed quotes from the `cryptofeed.FeedHandler` to + the `piker`-side `trio.task` consumers for delivery to consumer + sub-actors for various subsystems. + + ''' + async def _trade( + trade: Trade, # cryptofeed, NOT ours from `.venues`! + receipt_timestamp: int, + ) -> None: + ''' + Proxy-thru `cryptofeed.FeedHandler` "trades" to `piker`-side. + + ''' + to_trio.send_nowait(('trade', trade)) + + async def _l1( + book: L1Book, + receipt_timestamp: int, + ) -> None: + ''' + Relay-thru "l1 book" updates. + + ''' + + to_trio.send_nowait(('l1', book)) + + # TODO, make this work! + # -[ ] why isn't this working in `tractor.pause_from_sync()`?? + # breakpoint() - async def _l1(data: dict, receipt_timestamp): - to_trio.send_nowait(('l1', { - 'symbol': cb_sym_to_deribit_inst( - str_to_cb_sym(data.symbol)).lower(), - 'ticks': [ - { - 'type': 'bid', - 'price': float(data.bid_price), - 'size': float(data.bid_size) - }, - { - 'type': 'bsize', - 'price': float(data.bid_price), - 'size': float(data.bid_size) - }, - { - 'type': 'ask', - 'price': float(data.ask_price), - 'size': float(data.ask_size) - }, - { - 'type': 'asize', - 'price': float(data.ask_price), - 'size': float(data.ask_size) - } - ] - })) sym: Symbol = piker_sym_to_cb_sym(instrument) fh.add_feed( DERIBIT, @@ -625,27 +687,35 @@ async def aio_price_feed_relay( if not fh.running: fh.run( start_loop=False, - install_signal_handlers=False) + install_signal_handlers=False + ) # sync with trio to_trio.send_nowait(None) + # run until cancelled await asyncio.sleep(float('inf')) @acm async def open_price_feed( instrument: str -) -> trio.abc.ReceiveStream: - async with maybe_open_feed_handler() as fh: - async with to_asyncio.open_channel_from( +) -> to_asyncio.LinkedTaskChannel: + + fh: FeedHandler + first: None + chan: to_asyncio.LinkedTaskChannel + async with ( + maybe_open_feed_handler() as fh, + to_asyncio.open_channel_from( partial( aio_price_feed_relay, fh, instrument ) - ) as (first, chan): - yield chan + ) as (first, chan) + ): + yield chan @acm @@ -654,6 +724,7 @@ async def maybe_open_price_feed( ) -> trio.abc.ReceiveStream: # TODO: add a predicate to maybe_open_context + feed: to_asyncio.LinkedTaskChannel async with maybe_open_context( acm_func=open_price_feed, kwargs={ @@ -668,68 +739,69 @@ async def maybe_open_price_feed( -async def aio_order_feed_relay( - fh: FeedHandler, - instrument: Symbol, - from_trio: asyncio.Queue, - to_trio: trio.abc.SendChannel, -) -> None: - async def _fill(data: dict, receipt_timestamp): - breakpoint() +# TODO, move all to `.broker` submod! +# async def aio_order_feed_relay( +# fh: FeedHandler, +# instrument: Symbol, +# from_trio: asyncio.Queue, +# to_trio: trio.abc.SendChannel, +# ) -> None: +# async def _fill(data: dict, receipt_timestamp): +# breakpoint() - async def _order_info(data: dict, receipt_timestamp): - breakpoint() +# async def _order_info(data: dict, receipt_timestamp): +# breakpoint() - fh.add_feed( - DERIBIT, - channels=[FILLS, ORDER_INFO], - symbols=[instrument.upper()], - callbacks={ - FILLS: _fill, - ORDER_INFO: _order_info, - }) +# fh.add_feed( +# DERIBIT, +# channels=[FILLS, ORDER_INFO], +# symbols=[instrument.upper()], +# callbacks={ +# FILLS: _fill, +# ORDER_INFO: _order_info, +# }) - if not fh.running: - fh.run( - start_loop=False, - install_signal_handlers=False) +# if not fh.running: +# fh.run( +# start_loop=False, +# install_signal_handlers=False) - # sync with trio - to_trio.send_nowait(None) +# # sync with trio +# to_trio.send_nowait(None) - await asyncio.sleep(float('inf')) +# await asyncio.sleep(float('inf')) -@acm -async def open_order_feed( - instrument: list[str] -) -> trio.abc.ReceiveStream: - async with maybe_open_feed_handler() as fh: - async with to_asyncio.open_channel_from( - partial( - aio_order_feed_relay, - fh, - instrument - ) - ) as (first, chan): - yield chan +# @acm +# async def open_order_feed( +# instrument: list[str] +# ) -> trio.abc.ReceiveStream: +# async with maybe_open_feed_handler() as fh: +# async with to_asyncio.open_channel_from( +# partial( +# aio_order_feed_relay, +# fh, +# instrument +# ) +# ) as (first, chan): +# yield chan -@acm -async def maybe_open_order_feed( - instrument: str -) -> trio.abc.ReceiveStream: +# @acm +# async def maybe_open_order_feed( +# instrument: str +# ) -> trio.abc.ReceiveStream: - # TODO: add a predicate to maybe_open_context - async with maybe_open_context( - acm_func=open_order_feed, - kwargs={ - 'instrument': instrument.split('.')[0], - 'fh': fh - }, - key=f'{instrument.split('.')[0]}-order', - ) as (cache_hit, feed): - if cache_hit: - yield broadcast_receiver(feed, 10) - else: - yield feed +# # TODO: add a predicate to maybe_open_context +# async with maybe_open_context( +# acm_func=open_order_feed, +# kwargs={ +# 'instrument': instrument.split('.')[0], +# 'fh': fh +# }, +# key=f'{instrument.split('.')[0]}-order', +# ) as (cache_hit, feed): +# if cache_hit: +# yield broadcast_receiver(feed, 10) +# else: +# yield feed diff --git a/piker/brokers/deribit/feed.py b/piker/brokers/deribit/feed.py index e32e31d2..efd43ea5 100644 --- a/piker/brokers/deribit/feed.py +++ b/piker/brokers/deribit/feed.py @@ -18,56 +18,58 @@ Deribit backend. ''' +from __future__ import annotations from contextlib import asynccontextmanager as acm from datetime import datetime -from typing import Any, Optional, Callable -from pprint import pformat +from typing import ( + # Any, + # Optional, + Callable, +) +# from pprint import pformat import time +import cryptofeed import trio from trio_typing import TaskStatus from pendulum import ( from_timestamp, - now, ) -from rapidfuzz import process as fuzzy import numpy as np import tractor from piker.accounting import ( + Asset, MktPair, unpack_fqme, ) from piker.brokers import ( open_cached_client, NoData, + DataUnavailable, ) from piker._cacheables import ( async_lifo_cache, ) -from piker.log import get_logger, get_console_log -from piker.data import ShmArray +from piker.log import ( + get_logger, + mk_repr, +) from piker.data.validate import FeedInit -from piker.brokers._util import ( - BrokerError, - DataUnavailable, -) -from cryptofeed import FeedHandler -from cryptofeed.defines import ( - DERIBIT, L1_BOOK, TRADES, OPTION, CALL, PUT -) -from cryptofeed.symbols import Symbol from .api import ( - Client, Trade, - get_config, - piker_sym_to_cb_sym, cb_sym_to_deribit_inst, + Client, + # get_config, + piker_sym_to_cb_sym, + cb_sym_to_deribit_inst, + str_to_cb_sym, maybe_open_price_feed ) from .venues import ( Pair, OptionPair, + Trade, ) _spawn_kwargs = { @@ -86,6 +88,10 @@ async def open_history_client( # TODO implement history getter for the new storage layer. async with open_cached_client('deribit') as client: + pair: OptionPair = client._pairs[mkt.dst.name] + # XXX NOTE, the cuckers use ms !!! + creation_time_s: int = pair.creation_timestamp/1000 + async def get_ohlc( timeframe: float, end_dt: datetime | None = None, @@ -105,6 +111,31 @@ async def open_history_client( end_dt=end_dt, ) if len(array) == 0: + if ( + end_dt is None + ): + raise DataUnavailable( + 'No history seems to exist yet?\n\n' + f'{mkt}' + ) + elif ( + end_dt + and + end_dt.timestamp() < creation_time_s + ): + # the contract can't have history + # before it was created. + pair_type_str: str = type(pair).__name__ + create_dt: datetime = from_timestamp(creation_time_s) + raise DataUnavailable( + f'No history prior to\n' + f'`{pair_type_str}.creation_timestamp: int = ' + f'{pair.creation_timestamp}\n\n' + f'------ deribit sux ------\n' + f'WHICH IN "NORMAL PEOPLE WHO USE EPOCH TIME" form is,\n' + f'creation_time_s: {creation_time_s}\n' + f'create_dt: {create_dt}\n' + ) raise NoData( f'No frame for {start_dt} -> {end_dt}\n' ) @@ -126,14 +157,20 @@ async def open_history_client( return array, start_dt, end_dt - yield get_ohlc, {'erlangs': 3, 'rate': 3} + yield ( + get_ohlc, + { # backfill config + 'erlangs': 3, + 'rate': 3, + } + ) @async_lifo_cache() async def get_mkt_info( fqme: str, -) -> tuple[MktPair, Pair] | None: +) -> tuple[MktPair, Pair|OptionPair] | None: # uppercase since kraken bs_mktid is always upper if 'deribit' not in fqme.lower(): @@ -149,7 +186,7 @@ async def get_mkt_info( # returns, always! expiry: str = expiry.upper() venue: str = venue.upper() - venue_lower: str = venue.lower() + # venue_lower: str = venue.lower() mkt_mode: str = 'option' @@ -175,64 +212,88 @@ async def get_mkt_info( price_tick=pair.price_tick, size_tick=pair.size_tick, bs_mktid=pair.symbol, - expiry=pair.expiry, venue=mkt_mode, broker='deribit', _atype=mkt_mode, _fqme_without_src=True, + + # expiry=pair.expiry, + # XXX TODO, currently we don't use it since it's + # already "described" in the `OptionPair.symbol: str` + # and if we slap in the ISO repr it's kinda hideous.. + # -[ ] figure out the best either std ) return mkt, pair async def stream_quotes( - send_chan: trio.abc.SendChannel, symbols: list[str], feed_is_live: trio.Event, - loglevel: str = None, # startup sync task_status: TaskStatus[tuple[dict, dict]] = trio.TASK_STATUS_IGNORED, ) -> None: - # XXX: required to propagate ``tractor`` loglevel to piker logging - get_console_log(loglevel or tractor.current_actor().loglevel) + ''' + Open a live quote stream for the market set defined by `symbols`. + Internally this starts a `cryptofeed.FeedHandler` inside an `asyncio`-side + task and relays through L1 and `Trade` msgs here to our `trio.Task`. + + ''' sym = symbols[0].split('.')[0] - init_msgs: list[FeedInit] = [] + # multiline nested `dict` formatter (since rn quote-msgs are + # just that). + pfmt: Callable[[str], str] = mk_repr( + # so we can see `deribit`'s delightfully mega-long bs fields.. + maxstring=100, + ) + async with ( open_cached_client('deribit') as client, send_chan as send_chan ): - + mkt: MktPair + pair: Pair mkt, pair = await get_mkt_info(sym) # build out init msgs according to latest spec init_msgs.append( - FeedInit(mkt_info=mkt) + FeedInit( + mkt_info=mkt, + ) ) - nsym = piker_sym_to_cb_sym(sym) + # build `cryptofeed` feed-handle + cf_sym: cryptofeed.Symbol = piker_sym_to_cb_sym(sym) - async with maybe_open_price_feed(sym) as stream: + from_cf: tractor.to_asyncio.LinkedTaskChannel + async with maybe_open_price_feed(sym) as from_cf: - cache = client._pairs + # load the "last trades" summary + last_trades_res: cryptofeed.LastTradesResult = await client.last_trades( + cb_sym_to_deribit_inst(cf_sym), + count=1, + ) + last_trades: list[Trade] = last_trades_res.trades - last_trades = (await client.last_trades( - cb_sym_to_deribit_inst(nsym), count=1)).trades + # TODO, do we even need this or will the above always + # work? + # if not last_trades: + # await tractor.pause() + # async for typ, quote in from_cf: + # if typ == 'trade': + # last_trade = Trade(**(quote['data'])) + # break - if len(last_trades) == 0: - last_trade = None - async for typ, quote in stream: - if typ == 'trade': - last_trade = Trade(**(quote['data'])) - break + # else: + last_trade = Trade( + **(last_trades[0]) + ) - else: - last_trade = Trade(**(last_trades[0])) - - first_quote = { + first_quote: dict = { 'symbol': sym, 'last': last_trade.price, 'brokerd_ts': last_trade.timestamp, @@ -243,13 +304,84 @@ async def stream_quotes( 'broker_ts': last_trade.timestamp }] } - task_status.started((init_msgs, first_quote)) + task_status.started(( + init_msgs, + first_quote, + )) feed_is_live.set() - async for typ, quote in stream: - topic = quote['symbol'] - await send_chan.send({topic: quote}) + # NOTE XXX, static for now! + # => since this only handles ONE mkt feed at a time we + # don't need a lookup table to map interleaved quotes + # from multiple possible mkt-pairs + topic: str = mkt.bs_fqme + + # deliver until cancelled + async for typ, ref in from_cf: + match typ: + case 'trade': + trade: cryptofeed.types.Trade = ref + + # TODO, re-impl this according to teh ideal + # fqme for opts that we choose!! + bs_fqme: str = cb_sym_to_deribit_inst( + str_to_cb_sym(trade.symbol) + ).lower() + + piker_quote: dict = { + 'symbol': bs_fqme, + 'last': trade.price, + 'broker_ts': time.time(), + # ^TODO, name this `brokerd/datad_ts` and + # use `time.time_ns()` ?? + 'ticks': [{ + 'type': 'trade', + 'price': float(trade.price), + 'size': float(trade.amount), + 'broker_ts': trade.timestamp, + }], + } + log.info( + f'deribit {typ!r} quote for {sym!r}\n\n' + f'{trade}\n\n' + f'{pfmt(piker_quote)}\n' + ) + + case 'l1': + book: cryptofeed.types.L1Book = ref + + # TODO, so this is where we can possibly change things + # and instead lever the `MktPair.bs_fqme: str` output? + bs_fqme: str = cb_sym_to_deribit_inst( + str_to_cb_sym(book.symbol) + ).lower() + + piker_quote: dict = { + 'symbol': bs_fqme, + 'ticks': [ + + {'type': 'bid', + 'price': float(book.bid_price), + 'size': float(book.bid_size)}, + + {'type': 'bsize', + 'price': float(book.bid_price), + 'size': float(book.bid_size),}, + + {'type': 'ask', + 'price': float(book.ask_price), + 'size': float(book.ask_size),}, + + {'type': 'asize', + 'price': float(book.ask_price), + 'size': float(book.ask_size),} + ] + } + + await send_chan.send({ + topic: piker_quote, + }) @tractor.context @@ -259,13 +391,13 @@ async def open_symbol_search( async with open_cached_client('deribit') as client: # load all symbols locally for fast search - cache = client._pairs + # cache = client._pairs await ctx.started() async with ctx.open_stream() as stream: - pattern: str async for pattern in stream: + # NOTE: pattern fuzzy-matching is done within # the methd impl. pairs: dict[str, Pair] = await client.search_symbols( diff --git a/piker/brokers/deribit/venues.py b/piker/brokers/deribit/venues.py index 91a1583f..0dda913e 100644 --- a/piker/brokers/deribit/venues.py +++ b/piker/brokers/deribit/venues.py @@ -22,11 +22,10 @@ from __future__ import annotations import pendulum from typing import ( Literal, + Optional, ) from decimal import Decimal -from msgspec import field - from piker.types import Struct @@ -111,18 +110,21 @@ class OptionPair(Pair, frozen=True): block_trade_min_trade_amount: int # '25' block_trade_commission: float # '0.003' - # NOTE: see `.data._symcache.SymbologyCache.load()` for why ns_path: str = 'piker.brokers.deribit:OptionPair' + # TODO, impl this without the MM:SS part of + # the `'THH:MM:SS..'` etc.. @property def expiry(self) -> str: - iso_date = pendulum.from_timestamp(self.expiration_timestamp / 1000).isoformat() + iso_date = pendulum.from_timestamp( + self.expiration_timestamp / 1000 + ).isoformat() return iso_date @property def venue(self) -> str: - return 'option' + return f'{self.instrument_type}_option' @property def bs_fqme(self) -> str: @@ -152,6 +154,7 @@ class JSONRPCResult(Struct): error: Optional[dict] = None result: Optional[list[dict]] = None + class JSONRPCChannel(Struct): method: str params: dict @@ -168,6 +171,7 @@ class KLinesResult(Struct): status: str volume: list[float] + class Trade(Struct): iv: float price: float @@ -186,6 +190,7 @@ class Trade(Struct): block_trade_id: Optional[str] = '', block_trade_leg_count: Optional[int] = 0, + class LastTradesResult(Struct): trades: list[Trade] has_more: bool diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index 83cd363d..74e6410a 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -30,6 +30,7 @@ import time from typing import ( Any, AsyncIterator, + Callable, TYPE_CHECKING, ) @@ -54,6 +55,9 @@ from ._util import ( get_console_log, ) from ..service import maybe_spawn_daemon +from piker.log import ( + mk_repr, +) if TYPE_CHECKING: from ._sharedmem import ( @@ -575,7 +579,6 @@ async def open_sample_stream( async def sample_and_broadcast( - bus: _FeedsBus, # noqa rt_shm: ShmArray, hist_shm: ShmArray, @@ -596,11 +599,22 @@ async def sample_and_broadcast( overruns = Counter() + # multiline nested `dict` formatter (since rn quote-msgs are + # just that). + pfmt: Callable[[str], str] = mk_repr() + # iterate stream delivered by broker async for quotes in quote_stream: - # print(quotes) - # TODO: ``numba`` this! + # XXX WARNING XXX only enable for debugging bc ow can cost + # ALOT of perf with HF-feedz!!! + # + # log.info( + # 'Rx live quotes:\n' + # f'{pfmt(quotes)}' + # ) + + # TODO: `numba` this! for broker_symbol, quote in quotes.items(): # TODO: in theory you can send the IPC msg *before* writing # to the sharedmem array to decrease latency, however, that @@ -673,6 +687,18 @@ async def sample_and_broadcast( sub_key: str = broker_symbol.lower() subs: set[Sub] = bus.get_subs(sub_key) + if not subs: + all_bs_fqmes: list[str] = list( + bus._subscribers.keys() + ) + log.warning( + f'No subscribers for {brokername!r} live-quote ??\n' + f'broker_symbol: {broker_symbol}\n\n' + + f'Maybe the backend-sys symbol does not match one of,\n' + f'{pfmt(all_bs_fqmes)}\n' + ) + # NOTE: by default the broker backend doesn't append # it's own "name" into the fqme schema (but maybe it # should?) so we have to manually generate the correct diff --git a/piker/data/_web_bs.py b/piker/data/_web_bs.py index a401588d..2f781571 100644 --- a/piker/data/_web_bs.py +++ b/piker/data/_web_bs.py @@ -360,7 +360,7 @@ async def open_autorecon_ws( ''' JSONRPC response-request style machinery for transparent multiplexing -of msgs over a NoBsWs. +of msgs over a `NoBsWs`. ''' @@ -377,6 +377,16 @@ async def open_jsonrpc_session( url: str, start_id: int = 0, response_type: type = JSONRPCResult, + msg_recv_timeout: float = float('inf'), + # ^NOTE, since only `deribit` is using this jsonrpc stuff atm + # and options mkts are generally "slow moving".. + # + # FURTHER if we break the underlying ws connection then since we + # don't pass a `fixture` to the task that manages `NoBsWs`, i.e. + # `_reconnect_forever()`, the jsonrpc "transport pipe" get's + # broken and never restored with wtv init sequence is required to + # re-establish a working req-resp session. + # request_type: Optional[type] = None, # request_hook: Optional[Callable] = None, # error_hook: Optional[Callable] = None, @@ -388,12 +398,18 @@ async def open_jsonrpc_session( async with ( trio.open_nursery() as n, - open_autorecon_ws(url) as ws + open_autorecon_ws( + url=url, + msg_recv_timeout=msg_recv_timeout, + ) as ws ): rpc_id: Iterable[int] = count(start_id) rpc_results: dict[int, dict] = {} - async def json_rpc(method: str, params: dict) -> dict: + async def json_rpc( + method: str, + params: dict, + ) -> dict: ''' perform a json rpc call and wait for the result, raise exception in case of error field present on response diff --git a/piker/data/feed.py b/piker/data/feed.py index 7264c8e6..20f7da30 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -540,7 +540,10 @@ async def open_feed_bus( # subscription since the backend isn't (yet) expected to # append it's own name to the fqme, so we filter on keys # which *do not* include that name (e.g .ib) . - bus._subscribers.setdefault(bs_fqme, set()) + bus._subscribers.setdefault( + bs_fqme, + set(), + ) # sync feed subscribers with flume handles await ctx.started( diff --git a/piker/log.py b/piker/log.py index 56776e1e..7f554f16 100644 --- a/piker/log.py +++ b/piker/log.py @@ -18,7 +18,11 @@ Log like a forester! """ import logging +import reprlib import json +from typing import ( + Callable, +) import tractor from pygments import ( @@ -84,3 +88,27 @@ def colorize_json( # likeable styles: algol_nu, tango, monokai formatters.TerminalTrueColorFormatter(style=style) ) + + +def mk_repr( + **repr_kws, +) -> Callable[[str], str]: + ''' + Allocate and deliver a `repr.Repr` instance with provided input + settings using the std-lib's `reprlib` mod, + * https://docs.python.org/3/library/reprlib.html + + ------ Ex. ------ + An up to 6-layer-nested `dict` as multi-line: + - https://stackoverflow.com/a/79102479 + - https://docs.python.org/3/library/reprlib.html#reprlib.Repr.maxlevel + + ''' + def_kws: dict[str, int] = dict( + indent=2, + maxlevel=6, # recursion levels + maxstring=66, # match editor line-len limit + ) + def_kws |= repr_kws + reprr = reprlib.Repr(**def_kws) + return reprr.repr diff --git a/piker/storage/nativedb.py b/piker/storage/nativedb.py index bc7f10e3..941b519f 100644 --- a/piker/storage/nativedb.py +++ b/piker/storage/nativedb.py @@ -161,7 +161,12 @@ class NativeStorageClient: def index_files(self): for path in self._datadir.iterdir(): - if path.name in {'borked', 'expired',}: + if ( + path.name in {'borked', 'expired',} + or + '.parquet' not in str(path) + ): + # ignore all non-apache files (for now) continue key: str = path.name.rstrip('.parquet') diff --git a/piker/tsp/__init__.py b/piker/tsp/__init__.py index 7e0b4953..554f0199 100644 --- a/piker/tsp/__init__.py +++ b/piker/tsp/__init__.py @@ -44,8 +44,10 @@ import trio from trio_typing import TaskStatus import tractor from pendulum import ( + Interval, DateTime, Duration, + duration as mk_duration, from_timestamp, ) import numpy as np @@ -214,7 +216,8 @@ async def maybe_fill_null_segments( # pair, immediately stop backfilling? if ( start_dt - and end_dt < start_dt + and + end_dt < start_dt ): await tractor.pause() break @@ -262,6 +265,7 @@ async def maybe_fill_null_segments( except tractor.ContextCancelled: # log.exception await tractor.pause() + raise null_segs_detected.set() # RECHECK for more null-gaps @@ -349,7 +353,7 @@ async def maybe_fill_null_segments( async def start_backfill( get_hist, - frame_types: dict[str, Duration] | None, + def_frame_duration: Duration, mod: ModuleType, mkt: MktPair, shm: ShmArray, @@ -379,22 +383,23 @@ async def start_backfill( update_start_on_prepend: bool = False if backfill_until_dt is None: - # TODO: drop this right and just expose the backfill - # limits inside a [storage] section in conf.toml? - # when no tsdb "last datum" is provided, we just load - # some near-term history. - # periods = { - # 1: {'days': 1}, - # 60: {'days': 14}, - # } - - # do a decently sized backfill and load it into storage. + # TODO: per-provider default history-durations? + # -[ ] inside the `open_history_client()` config allow + # declaring the history duration limits instead of + # guessing and/or applying the same limits to all? + # + # -[ ] allow declaring (default) per-provider backfill + # limits inside a [storage] sub-section in conf.toml? + # + # NOTE, when no tsdb "last datum" is provided, we just + # load some near-term history by presuming a "decently + # large" 60s duration limit and a much shorter 1s range. periods = { 1: {'days': 2}, 60: {'years': 6}, } period_duration: int = periods[timeframe] - update_start_on_prepend = True + update_start_on_prepend: bool = True # NOTE: manually set the "latest" datetime which we intend to # backfill history "until" so as to adhere to the history @@ -416,7 +421,6 @@ async def start_backfill( f'backfill_until_dt: {backfill_until_dt}\n' f'last_start_dt: {last_start_dt}\n' ) - try: ( array, @@ -426,37 +430,58 @@ async def start_backfill( timeframe, end_dt=last_start_dt, ) - except NoData as _daterr: - # 3 cases: - # - frame in the middle of a legit venue gap - # - history actually began at the `last_start_dt` - # - some other unknown error (ib blocking the - # history bc they don't want you seeing how they - # cucked all the tinas..) - if dur := frame_types.get(timeframe): - # decrement by a frame's worth of duration and - # retry a few times. - last_start_dt.subtract( - seconds=dur.total_seconds() + orig_last_start_dt: datetime = last_start_dt + gap_report: str = ( + f'EMPTY FRAME for `end_dt: {last_start_dt}`?\n' + f'{mod.name} -> tf@fqme: {timeframe}@{mkt.fqme}\n' + f'last_start_dt: {orig_last_start_dt}\n\n' + f'bf_until: {backfill_until_dt}\n' + ) + # EMPTY FRAME signal with 3 (likely) causes: + # + # 1. range contains legit gap in venue history + # 2. history actually (edge case) **began** at the + # value `last_start_dt` + # 3. some other unknown error (ib blocking the + # history-query bc they don't want you seeing how + # they cucked all the tinas.. like with options + # hist) + # + if def_frame_duration: + # decrement by a duration's (frame) worth of time + # as maybe indicated by the backend to see if we + # can get older data before this possible + # "history gap". + last_start_dt: datetime = last_start_dt.subtract( + seconds=def_frame_duration.total_seconds() ) - log.warning( - f'{mod.name} -> EMPTY FRAME for end_dt?\n' - f'tf@fqme: {timeframe}@{mkt.fqme}\n' - 'bf_until <- last_start_dt:\n' - f'{backfill_until_dt} <- {last_start_dt}\n' - f'Decrementing `end_dt` by {dur} and retry..\n' + gap_report += ( + f'Decrementing `end_dt` and retrying with,\n' + f'def_frame_duration: {def_frame_duration}\n' + f'(new) last_start_dt: {last_start_dt}\n' ) + log.warning(gap_report) + # skip writing to shm/tsdb and try the next + # duration's worth of prior history. continue + else: + # await tractor.pause() + raise DataUnavailable(gap_report) + # broker says there never was or is no more history to pull - except DataUnavailable: + except DataUnavailable as due: + message: str = due.args[0] log.warning( - f'NO-MORE-DATA in range?\n' - f'`{mod.name}` halted history:\n' - f'tf@fqme: {timeframe}@{mkt.fqme}\n' - 'bf_until <- last_start_dt:\n' - f'{backfill_until_dt} <- {last_start_dt}\n' + f'Provider {mod.name!r} halted backfill due to,\n\n' + + f'{message}\n' + + f'fqme: {mkt.fqme}\n' + f'timeframe: {timeframe}\n' + f'last_start_dt: {last_start_dt}\n' + f'bf_until: {backfill_until_dt}\n' ) # UGH: what's a better way? # TODO: backends are responsible for being correct on @@ -465,34 +490,54 @@ async def start_backfill( # to halt the request loop until the condition is # resolved or should the backend be entirely in # charge of solving such faults? yes, right? - # if timeframe > 1: - # await tractor.pause() return + time: np.ndarray = array['time'] assert ( - array['time'][0] + time[0] == next_start_dt.timestamp() ) - diff = last_start_dt - next_start_dt - frame_time_diff_s = diff.seconds + assert time[-1] == next_end_dt.timestamp() + + expected_dur: Interval = last_start_dt - next_start_dt # frame's worth of sample-period-steps, in seconds frame_size_s: float = len(array) * timeframe - expected_frame_size_s: float = frame_size_s + timeframe - if frame_time_diff_s > expected_frame_size_s: - + recv_frame_dur: Duration = ( + from_timestamp(array[-1]['time']) + - + from_timestamp(array[0]['time']) + ) + if ( + (lt_frame := (recv_frame_dur < expected_dur)) + or + (null_frame := (frame_size_s == 0)) + # ^XXX, should NEVER hit now! + ): # XXX: query result includes a start point prior to our # expected "frame size" and thus is likely some kind of # history gap (eg. market closed period, outage, etc.) # so just report it to console for now. + if lt_frame: + reason = 'Possible GAP (or first-datum)' + else: + assert null_frame + reason = 'NULL-FRAME' + + missing_dur: Interval = expected_dur.end - recv_frame_dur.end log.warning( - 'GAP DETECTED:\n' - f'last_start_dt: {last_start_dt}\n' - f'diff: {diff}\n' - f'frame_time_diff_s: {frame_time_diff_s}\n' + f'{timeframe}s-series {reason} detected!\n' + f'fqme: {mkt.fqme}\n' + f'last_start_dt: {last_start_dt}\n\n' + f'recv interval: {recv_frame_dur}\n' + f'expected interval: {expected_dur}\n\n' + + f'Missing duration of history of {missing_dur.in_words()!r}\n' + f'{missing_dur}\n' ) + # await tractor.pause() to_push = diff_history( array, @@ -567,7 +612,8 @@ async def start_backfill( # long-term storage. if ( storage is not None - and write_tsdb + and + write_tsdb ): log.info( f'Writing {ln} frame to storage:\n' @@ -688,7 +734,7 @@ async def back_load_from_tsdb( last_tsdb_dt and latest_start_dt ): - backfilled_size_s = ( + backfilled_size_s: Duration = ( latest_start_dt - last_tsdb_dt ).seconds # if the shm buffer len is not large enough to contain @@ -911,6 +957,8 @@ async def tsdb_backfill( f'{pformat(config)}\n' ) + # concurrently load the provider's most-recent-frame AND any + # pre-existing tsdb history already saved in `piker` storage. dt_eps: list[DateTime, DateTime] = [] async with trio.open_nursery() as tn: tn.start_soon( @@ -921,7 +969,6 @@ async def tsdb_backfill( timeframe, config, ) - tsdb_entry: tuple = await load_tsdb_hist( storage, mkt, @@ -950,6 +997,25 @@ async def tsdb_backfill( mr_end_dt, ) = dt_eps + first_frame_dur_s: Duration = (mr_end_dt - mr_start_dt).seconds + calced_frame_size: Duration = mk_duration( + seconds=first_frame_dur_s, + ) + # NOTE, attempt to use the backend declared default frame + # sizing (as allowed by their time-series query APIs) and + # if not provided try to construct a default from the + # first frame received above. + def_frame_durs: dict[ + int, + Duration, + ]|None = config.get('frame_types', None) + if def_frame_durs: + def_frame_size: Duration = def_frame_durs[timeframe] + assert def_frame_size == calced_frame_size + else: + # use what we calced from first frame above. + def_frame_size = calced_frame_size + # NOTE: when there's no offline data, there's 2 cases: # - data backend doesn't support timeframe/sample # period (in which case `dt_eps` should be `None` and @@ -980,7 +1046,7 @@ async def tsdb_backfill( partial( start_backfill, get_hist=get_hist, - frame_types=config.get('frame_types', None), + def_frame_duration=def_frame_size, mod=mod, mkt=mkt, shm=shm, diff --git a/poetry.lock b/poetry.lock index c9b5ced6..72f22a3f 100644 --- a/poetry.lock +++ b/poetry.lock @@ -2,13 +2,13 @@ [[package]] name = "anyio" -version = "3.7.1" +version = "4.6.2.post1" description = "High level compatibility layer for multiple asynchronous event loop implementations" optional = false -python-versions = ">=3.7" +python-versions = ">=3.9" files = [ - {file = "anyio-3.7.1-py3-none-any.whl", hash = "sha256:91dee416e570e92c64041bd18b900d1d6fa78dff7048769ce5ac5ddad004fbb5"}, - {file = "anyio-3.7.1.tar.gz", hash = "sha256:44a3c9aba0f5defa43261a8b3efb97891f2bd7d804e0e1f56419befa1adfc780"}, + {file = "anyio-4.6.2.post1-py3-none-any.whl", hash = "sha256:6d170c36fba3bdd840c73d3868c1e777e33676a69c3a72cf0a0d5d6d8009b61d"}, + {file = "anyio-4.6.2.post1.tar.gz", hash = "sha256:4c8bc31ccdb51c7f7bd251f51c609e038d63e34219b44aa86e47576389880b4c"}, ] [package.dependencies] @@ -50,7 +50,7 @@ files = [ [[package]] name = "asyncvnc" version = "1.1.0" -description = "" +description = "Asynchronous VNC for Python" optional = false python-versions = ">= 3.7" files = [] @@ -69,21 +69,22 @@ resolved_reference = "825447564e3af6b0d4a0996793f1ca7fb360c48f" [[package]] name = "attrs" -version = "23.1.0" +version = "23.2.0" description = "Classes Without Boilerplate" optional = false python-versions = ">=3.7" files = [ - {file = "attrs-23.1.0-py3-none-any.whl", hash = "sha256:1f28b4522cdc2fb4256ac1a020c78acf9cba2c6b461ccd2c126f3aa8e8335d04"}, - {file = "attrs-23.1.0.tar.gz", hash = "sha256:6279836d581513a26f1bf235f9acd333bc9115683f14f7e8fae46c98fc50e015"}, + {file = "attrs-23.2.0-py3-none-any.whl", hash = "sha256:99b87a485a5820b23b879f04c2305b44b951b502fd64be915879d77a7e8fc6f1"}, + {file = "attrs-23.2.0.tar.gz", hash = "sha256:935dc3b529c262f6cf76e50877d35a4bd3c1de194fd41f47a2b7ae8f19971f30"}, ] [package.extras] cov = ["attrs[tests]", "coverage[toml] (>=5.3)"] -dev = ["attrs[docs,tests]", "pre-commit"] +dev = ["attrs[tests]", "pre-commit"] docs = ["furo", "myst-parser", "sphinx", "sphinx-notfound-page", "sphinxcontrib-towncrier", "towncrier", "zope-interface"] tests = ["attrs[tests-no-zope]", "zope-interface"] -tests-no-zope = ["cloudpickle", "hypothesis", "mypy (>=1.1.1)", "pympler", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-xdist[psutil]"] +tests-mypy = ["mypy (>=1.6)", "pytest-mypy-plugins"] +tests-no-zope = ["attrs[tests-mypy]", "cloudpickle", "hypothesis", "pympler", "pytest (>=4.3.0)", "pytest-xdist[psutil]"] [[package]] name = "bidict" @@ -103,75 +104,78 @@ test = ["hypothesis", "pytest", "pytest-benchmark[histogram]", "pytest-cov", "py [[package]] name = "cffi" -version = "1.15.1" +version = "1.17.1" description = "Foreign Function Interface for Python calling C code." optional = false -python-versions = "*" +python-versions = ">=3.8" files = [ - {file = "cffi-1.15.1-cp27-cp27m-macosx_10_9_x86_64.whl", hash = "sha256:a66d3508133af6e8548451b25058d5812812ec3798c886bf38ed24a98216fab2"}, - {file = "cffi-1.15.1-cp27-cp27m-manylinux1_i686.whl", hash = "sha256:470c103ae716238bbe698d67ad020e1db9d9dba34fa5a899b5e21577e6d52ed2"}, - {file = "cffi-1.15.1-cp27-cp27m-manylinux1_x86_64.whl", hash = "sha256:9ad5db27f9cabae298d151c85cf2bad1d359a1b9c686a275df03385758e2f914"}, - {file = "cffi-1.15.1-cp27-cp27m-win32.whl", hash = "sha256:b3bbeb01c2b273cca1e1e0c5df57f12dce9a4dd331b4fa1635b8bec26350bde3"}, - {file = "cffi-1.15.1-cp27-cp27m-win_amd64.whl", hash = "sha256:e00b098126fd45523dd056d2efba6c5a63b71ffe9f2bbe1a4fe1716e1d0c331e"}, - {file = "cffi-1.15.1-cp27-cp27mu-manylinux1_i686.whl", hash = "sha256:d61f4695e6c866a23a21acab0509af1cdfd2c013cf256bbf5b6b5e2695827162"}, - {file = "cffi-1.15.1-cp27-cp27mu-manylinux1_x86_64.whl", hash = "sha256:ed9cb427ba5504c1dc15ede7d516b84757c3e3d7868ccc85121d9310d27eed0b"}, - {file = "cffi-1.15.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:39d39875251ca8f612b6f33e6b1195af86d1b3e60086068be9cc053aa4376e21"}, - {file = "cffi-1.15.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:285d29981935eb726a4399badae8f0ffdff4f5050eaa6d0cfc3f64b857b77185"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3eb6971dcff08619f8d91607cfc726518b6fa2a9eba42856be181c6d0d9515fd"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:21157295583fe8943475029ed5abdcf71eb3911894724e360acff1d61c1d54bc"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5635bd9cb9731e6d4a1132a498dd34f764034a8ce60cef4f5319c0541159392f"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2012c72d854c2d03e45d06ae57f40d78e5770d252f195b93f581acf3ba44496e"}, - {file = "cffi-1.15.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd86c085fae2efd48ac91dd7ccffcfc0571387fe1193d33b6394db7ef31fe2a4"}, - {file = "cffi-1.15.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:fa6693661a4c91757f4412306191b6dc88c1703f780c8234035eac011922bc01"}, - {file = "cffi-1.15.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:59c0b02d0a6c384d453fece7566d1c7e6b7bae4fc5874ef2ef46d56776d61c9e"}, - {file = "cffi-1.15.1-cp310-cp310-win32.whl", hash = "sha256:cba9d6b9a7d64d4bd46167096fc9d2f835e25d7e4c121fb2ddfc6528fb0413b2"}, - {file = "cffi-1.15.1-cp310-cp310-win_amd64.whl", hash = "sha256:ce4bcc037df4fc5e3d184794f27bdaab018943698f4ca31630bc7f84a7b69c6d"}, - {file = "cffi-1.15.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3d08afd128ddaa624a48cf2b859afef385b720bb4b43df214f85616922e6a5ac"}, - {file = "cffi-1.15.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:3799aecf2e17cf585d977b780ce79ff0dc9b78d799fc694221ce814c2c19db83"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a591fe9e525846e4d154205572a029f653ada1a78b93697f3b5a8f1f2bc055b9"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3548db281cd7d2561c9ad9984681c95f7b0e38881201e157833a2342c30d5e8c"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:91fc98adde3d7881af9b59ed0294046f3806221863722ba7d8d120c575314325"}, - {file = "cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:94411f22c3985acaec6f83c6df553f2dbe17b698cc7f8ae751ff2237d96b9e3c"}, - {file = "cffi-1.15.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:03425bdae262c76aad70202debd780501fabeaca237cdfddc008987c0e0f59ef"}, - {file = "cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:cc4d65aeeaa04136a12677d3dd0b1c0c94dc43abac5860ab33cceb42b801c1e8"}, - {file = "cffi-1.15.1-cp311-cp311-win32.whl", hash = "sha256:a0f100c8912c114ff53e1202d0078b425bee3649ae34d7b070e9697f93c5d52d"}, - {file = "cffi-1.15.1-cp311-cp311-win_amd64.whl", hash = "sha256:04ed324bda3cda42b9b695d51bb7d54b680b9719cfab04227cdd1e04e5de3104"}, - {file = "cffi-1.15.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:50a74364d85fd319352182ef59c5c790484a336f6db772c1a9231f1c3ed0cbd7"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e263d77ee3dd201c3a142934a086a4450861778baaeeb45db4591ef65550b0a6"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:cec7d9412a9102bdc577382c3929b337320c4c4c4849f2c5cdd14d7368c5562d"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4289fc34b2f5316fbb762d75362931e351941fa95fa18789191b33fc4cf9504a"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:173379135477dc8cac4bc58f45db08ab45d228b3363adb7af79436135d028405"}, - {file = "cffi-1.15.1-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:6975a3fac6bc83c4a65c9f9fcab9e47019a11d3d2cf7f3c0d03431bf145a941e"}, - {file = "cffi-1.15.1-cp36-cp36m-win32.whl", hash = "sha256:2470043b93ff09bf8fb1d46d1cb756ce6132c54826661a32d4e4d132e1977adf"}, - {file = "cffi-1.15.1-cp36-cp36m-win_amd64.whl", hash = "sha256:30d78fbc8ebf9c92c9b7823ee18eb92f2e6ef79b45ac84db507f52fbe3ec4497"}, - {file = "cffi-1.15.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:198caafb44239b60e252492445da556afafc7d1e3ab7a1fb3f0584ef6d742375"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5ef34d190326c3b1f822a5b7a45f6c4535e2f47ed06fec77d3d799c450b2651e"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8102eaf27e1e448db915d08afa8b41d6c7ca7a04b7d73af6514df10a3e74bd82"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:5df2768244d19ab7f60546d0c7c63ce1581f7af8b5de3eb3004b9b6fc8a9f84b"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a8c4917bd7ad33e8eb21e9a5bbba979b49d9a97acb3a803092cbc1133e20343c"}, - {file = "cffi-1.15.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e2642fe3142e4cc4af0799748233ad6da94c62a8bec3a6648bf8ee68b1c7426"}, - {file = "cffi-1.15.1-cp37-cp37m-win32.whl", hash = "sha256:e229a521186c75c8ad9490854fd8bbdd9a0c9aa3a524326b55be83b54d4e0ad9"}, - {file = "cffi-1.15.1-cp37-cp37m-win_amd64.whl", hash = "sha256:a0b71b1b8fbf2b96e41c4d990244165e2c9be83d54962a9a1d118fd8657d2045"}, - {file = "cffi-1.15.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:320dab6e7cb2eacdf0e658569d2575c4dad258c0fcc794f46215e1e39f90f2c3"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1e74c6b51a9ed6589199c787bf5f9875612ca4a8a0785fb2d4a84429badaf22a"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5c84c68147988265e60416b57fc83425a78058853509c1b0629c180094904a5"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3b926aa83d1edb5aa5b427b4053dc420ec295a08e40911296b9eb1b6170f6cca"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:87c450779d0914f2861b8526e035c5e6da0a3199d8f1add1a665e1cbc6fc6d02"}, - {file = "cffi-1.15.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f2c9f67e9821cad2e5f480bc8d83b8742896f1242dba247911072d4fa94c192"}, - {file = "cffi-1.15.1-cp38-cp38-win32.whl", hash = "sha256:8b7ee99e510d7b66cdb6c593f21c043c248537a32e0bedf02e01e9553a172314"}, - {file = "cffi-1.15.1-cp38-cp38-win_amd64.whl", hash = "sha256:00a9ed42e88df81ffae7a8ab6d9356b371399b91dbdf0c3cb1e84c03a13aceb5"}, - {file = "cffi-1.15.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:54a2db7b78338edd780e7ef7f9f6c442500fb0d41a5a4ea24fff1c929d5af585"}, - {file = "cffi-1.15.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:fcd131dd944808b5bdb38e6f5b53013c5aa4f334c5cad0c72742f6eba4b73db0"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7473e861101c9e72452f9bf8acb984947aa1661a7704553a9f6e4baa5ba64415"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6c9a799e985904922a4d207a94eae35c78ebae90e128f0c4e521ce339396be9d"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3bcde07039e586f91b45c88f8583ea7cf7a0770df3a1649627bf598332cb6984"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:33ab79603146aace82c2427da5ca6e58f2b3f2fb5da893ceac0c42218a40be35"}, - {file = "cffi-1.15.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5d598b938678ebf3c67377cdd45e09d431369c3b1a5b331058c338e201f12b27"}, - {file = "cffi-1.15.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:db0fbb9c62743ce59a9ff687eb5f4afbe77e5e8403d6697f7446e5f609976f76"}, - {file = "cffi-1.15.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:98d85c6a2bef81588d9227dde12db8a7f47f639f4a17c9ae08e773aa9c697bf3"}, - {file = "cffi-1.15.1-cp39-cp39-win32.whl", hash = "sha256:40f4774f5a9d4f5e344f31a32b5096977b5d48560c5592e2f3d2c4374bd543ee"}, - {file = "cffi-1.15.1-cp39-cp39-win_amd64.whl", hash = "sha256:70df4e3b545a17496c9b3f41f5115e69a4f2e77e94e1d2a8e1070bc0c38c8a3c"}, - {file = "cffi-1.15.1.tar.gz", hash = "sha256:d400bfb9a37b1351253cb402671cea7e89bdecc294e8016a707f6d1d8ac934f9"}, + {file = "cffi-1.17.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:df8b1c11f177bc2313ec4b2d46baec87a5f3e71fc8b45dab2ee7cae86d9aba14"}, + {file = "cffi-1.17.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8f2cdc858323644ab277e9bb925ad72ae0e67f69e804f4898c070998d50b1a67"}, + {file = "cffi-1.17.1-cp310-cp310-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:edae79245293e15384b51f88b00613ba9f7198016a5948b5dddf4917d4d26382"}, + {file = "cffi-1.17.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:45398b671ac6d70e67da8e4224a065cec6a93541bb7aebe1b198a61b58c7b702"}, + {file = "cffi-1.17.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad9413ccdeda48c5afdae7e4fa2192157e991ff761e7ab8fdd8926f40b160cc3"}, + {file = "cffi-1.17.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5da5719280082ac6bd9aa7becb3938dc9f9cbd57fac7d2871717b1feb0902ab6"}, + {file = "cffi-1.17.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bb1a08b8008b281856e5971307cc386a8e9c5b625ac297e853d36da6efe9c17"}, + {file = "cffi-1.17.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:045d61c734659cc045141be4bae381a41d89b741f795af1dd018bfb532fd0df8"}, + {file = "cffi-1.17.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:6883e737d7d9e4899a8a695e00ec36bd4e5e4f18fabe0aca0efe0a4b44cdb13e"}, + {file = "cffi-1.17.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:6b8b4a92e1c65048ff98cfe1f735ef8f1ceb72e3d5f0c25fdb12087a23da22be"}, + {file = "cffi-1.17.1-cp310-cp310-win32.whl", hash = "sha256:c9c3d058ebabb74db66e431095118094d06abf53284d9c81f27300d0e0d8bc7c"}, + {file = "cffi-1.17.1-cp310-cp310-win_amd64.whl", hash = "sha256:0f048dcf80db46f0098ccac01132761580d28e28bc0f78ae0d58048063317e15"}, + {file = "cffi-1.17.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a45e3c6913c5b87b3ff120dcdc03f6131fa0065027d0ed7ee6190736a74cd401"}, + {file = "cffi-1.17.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:30c5e0cb5ae493c04c8b42916e52ca38079f1b235c2f8ae5f4527b963c401caf"}, + {file = "cffi-1.17.1-cp311-cp311-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f75c7ab1f9e4aca5414ed4d8e5c0e303a34f4421f8a0d47a4d019ceff0ab6af4"}, + {file = "cffi-1.17.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a1ed2dd2972641495a3ec98445e09766f077aee98a1c896dcb4ad0d303628e41"}, + {file = "cffi-1.17.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:46bf43160c1a35f7ec506d254e5c890f3c03648a4dbac12d624e4490a7046cd1"}, + {file = "cffi-1.17.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:a24ed04c8ffd54b0729c07cee15a81d964e6fee0e3d4d342a27b020d22959dc6"}, + {file = "cffi-1.17.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:610faea79c43e44c71e1ec53a554553fa22321b65fae24889706c0a84d4ad86d"}, + {file = "cffi-1.17.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:a9b15d491f3ad5d692e11f6b71f7857e7835eb677955c00cc0aefcd0669adaf6"}, + {file = "cffi-1.17.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:de2ea4b5833625383e464549fec1bc395c1bdeeb5f25c4a3a82b5a8c756ec22f"}, + {file = "cffi-1.17.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:fc48c783f9c87e60831201f2cce7f3b2e4846bf4d8728eabe54d60700b318a0b"}, + {file = "cffi-1.17.1-cp311-cp311-win32.whl", hash = "sha256:85a950a4ac9c359340d5963966e3e0a94a676bd6245a4b55bc43949eee26a655"}, + {file = "cffi-1.17.1-cp311-cp311-win_amd64.whl", hash = "sha256:caaf0640ef5f5517f49bc275eca1406b0ffa6aa184892812030f04c2abf589a0"}, + {file = "cffi-1.17.1-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:805b4371bf7197c329fcb3ead37e710d1bca9da5d583f5073b799d5c5bd1eee4"}, + {file = "cffi-1.17.1-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:733e99bc2df47476e3848417c5a4540522f234dfd4ef3ab7fafdf555b082ec0c"}, + {file = "cffi-1.17.1-cp312-cp312-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1257bdabf294dceb59f5e70c64a3e2f462c30c7ad68092d01bbbfb1c16b1ba36"}, + {file = "cffi-1.17.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da95af8214998d77a98cc14e3a3bd00aa191526343078b530ceb0bd710fb48a5"}, + {file = "cffi-1.17.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:d63afe322132c194cf832bfec0dc69a99fb9bb6bbd550f161a49e9e855cc78ff"}, + {file = "cffi-1.17.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f79fc4fc25f1c8698ff97788206bb3c2598949bfe0fef03d299eb1b5356ada99"}, + {file = "cffi-1.17.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b62ce867176a75d03a665bad002af8e6d54644fad99a3c70905c543130e39d93"}, + {file = "cffi-1.17.1-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:386c8bf53c502fff58903061338ce4f4950cbdcb23e2902d86c0f722b786bbe3"}, + {file = "cffi-1.17.1-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:4ceb10419a9adf4460ea14cfd6bc43d08701f0835e979bf821052f1805850fe8"}, + {file = "cffi-1.17.1-cp312-cp312-win32.whl", hash = "sha256:a08d7e755f8ed21095a310a693525137cfe756ce62d066e53f502a83dc550f65"}, + {file = "cffi-1.17.1-cp312-cp312-win_amd64.whl", hash = "sha256:51392eae71afec0d0c8fb1a53b204dbb3bcabcb3c9b807eedf3e1e6ccf2de903"}, + {file = "cffi-1.17.1-cp313-cp313-macosx_10_13_x86_64.whl", hash = "sha256:f3a2b4222ce6b60e2e8b337bb9596923045681d71e5a082783484d845390938e"}, + {file = "cffi-1.17.1-cp313-cp313-macosx_11_0_arm64.whl", hash = "sha256:0984a4925a435b1da406122d4d7968dd861c1385afe3b45ba82b750f229811e2"}, + {file = "cffi-1.17.1-cp313-cp313-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d01b12eeeb4427d3110de311e1774046ad344f5b1a7403101878976ecd7a10f3"}, + {file = "cffi-1.17.1-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:706510fe141c86a69c8ddc029c7910003a17353970cff3b904ff0686a5927683"}, + {file = "cffi-1.17.1-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:de55b766c7aa2e2a3092c51e0483d700341182f08e67c63630d5b6f200bb28e5"}, + {file = "cffi-1.17.1-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c59d6e989d07460165cc5ad3c61f9fd8f1b4796eacbd81cee78957842b834af4"}, + {file = "cffi-1.17.1-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd398dbc6773384a17fe0d3e7eeb8d1a21c2200473ee6806bb5e6a8e62bb73dd"}, + {file = "cffi-1.17.1-cp313-cp313-musllinux_1_1_aarch64.whl", hash = "sha256:3edc8d958eb099c634dace3c7e16560ae474aa3803a5df240542b305d14e14ed"}, + {file = "cffi-1.17.1-cp313-cp313-musllinux_1_1_x86_64.whl", hash = "sha256:72e72408cad3d5419375fc87d289076ee319835bdfa2caad331e377589aebba9"}, + {file = "cffi-1.17.1-cp313-cp313-win32.whl", hash = "sha256:e03eab0a8677fa80d646b5ddece1cbeaf556c313dcfac435ba11f107ba117b5d"}, + {file = "cffi-1.17.1-cp313-cp313-win_amd64.whl", hash = "sha256:f6a16c31041f09ead72d69f583767292f750d24913dadacf5756b966aacb3f1a"}, + {file = "cffi-1.17.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:636062ea65bd0195bc012fea9321aca499c0504409f413dc88af450b57ffd03b"}, + {file = "cffi-1.17.1-cp38-cp38-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c7eac2ef9b63c79431bc4b25f1cd649d7f061a28808cbc6c47b534bd789ef964"}, + {file = "cffi-1.17.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e221cf152cff04059d011ee126477f0d9588303eb57e88923578ace7baad17f9"}, + {file = "cffi-1.17.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:31000ec67d4221a71bd3f67df918b1f88f676f1c3b535a7eb473255fdc0b83fc"}, + {file = "cffi-1.17.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:6f17be4345073b0a7b8ea599688f692ac3ef23ce28e5df79c04de519dbc4912c"}, + {file = "cffi-1.17.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e2b1fac190ae3ebfe37b979cc1ce69c81f4e4fe5746bb401dca63a9062cdaf1"}, + {file = "cffi-1.17.1-cp38-cp38-win32.whl", hash = "sha256:7596d6620d3fa590f677e9ee430df2958d2d6d6de2feeae5b20e82c00b76fbf8"}, + {file = "cffi-1.17.1-cp38-cp38-win_amd64.whl", hash = "sha256:78122be759c3f8a014ce010908ae03364d00a1f81ab5c7f4a7a5120607ea56e1"}, + {file = "cffi-1.17.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:b2ab587605f4ba0bf81dc0cb08a41bd1c0a5906bd59243d56bad7668a6fc6c16"}, + {file = "cffi-1.17.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:28b16024becceed8c6dfbc75629e27788d8a3f9030691a1dbf9821a128b22c36"}, + {file = "cffi-1.17.1-cp39-cp39-manylinux_2_12_i686.manylinux2010_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1d599671f396c4723d016dbddb72fe8e0397082b0a77a4fab8028923bec050e8"}, + {file = "cffi-1.17.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ca74b8dbe6e8e8263c0ffd60277de77dcee6c837a3d0881d8c1ead7268c9e576"}, + {file = "cffi-1.17.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f7f5baafcc48261359e14bcd6d9bff6d4b28d9103847c9e136694cb0501aef87"}, + {file = "cffi-1.17.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:98e3969bcff97cae1b2def8ba499ea3d6f31ddfdb7635374834cf89a1a08ecf0"}, + {file = "cffi-1.17.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cdf5ce3acdfd1661132f2a9c19cac174758dc2352bfe37d98aa7512c6b7178b3"}, + {file = "cffi-1.17.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:9755e4345d1ec879e3849e62222a18c7174d65a6a92d5b346b1863912168b595"}, + {file = "cffi-1.17.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:f1e22e8c4419538cb197e4dd60acc919d7696e5ef98ee4da4e01d3f8cfa4cc5a"}, + {file = "cffi-1.17.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:c03e868a0b3bc35839ba98e74211ed2b05d2119be4e8a0f224fba9384f1fe02e"}, + {file = "cffi-1.17.1-cp39-cp39-win32.whl", hash = "sha256:e31ae45bc2e29f6b2abd0de1cc3b9d5205aa847cafaecb8af1476a609a2f6eb7"}, + {file = "cffi-1.17.1-cp39-cp39-win_amd64.whl", hash = "sha256:d016c76bdd850f3c626af19b0542c9677ba156e4ee4fccfdd7848803533ef662"}, + {file = "cffi-1.17.1.tar.gz", hash = "sha256:1c39c6016c32bc48dd54561950ebd6836e1670f2ae46128f67cf49e789c52824"}, ] [package.dependencies] @@ -221,47 +225,51 @@ development = ["black", "flake8", "mypy", "pytest", "types-colorama"] [[package]] name = "cryptography" -version = "41.0.3" +version = "43.0.3" description = "cryptography is a package which provides cryptographic recipes and primitives to Python developers." optional = false python-versions = ">=3.7" files = [ - {file = "cryptography-41.0.3-cp37-abi3-macosx_10_12_universal2.whl", hash = "sha256:652627a055cb52a84f8c448185922241dd5217443ca194d5739b44612c5e6507"}, - {file = "cryptography-41.0.3-cp37-abi3-macosx_10_12_x86_64.whl", hash = "sha256:8f09daa483aedea50d249ef98ed500569841d6498aa9c9f4b0531b9964658922"}, - {file = "cryptography-41.0.3-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4fd871184321100fb400d759ad0cddddf284c4b696568204d281c902fc7b0d81"}, - {file = "cryptography-41.0.3-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:84537453d57f55a50a5b6835622ee405816999a7113267739a1b4581f83535bd"}, - {file = "cryptography-41.0.3-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:3fb248989b6363906827284cd20cca63bb1a757e0a2864d4c1682a985e3dca47"}, - {file = "cryptography-41.0.3-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:42cb413e01a5d36da9929baa9d70ca90d90b969269e5a12d39c1e0d475010116"}, - {file = "cryptography-41.0.3-cp37-abi3-musllinux_1_1_aarch64.whl", hash = "sha256:aeb57c421b34af8f9fe830e1955bf493a86a7996cc1338fe41b30047d16e962c"}, - {file = "cryptography-41.0.3-cp37-abi3-musllinux_1_1_x86_64.whl", hash = "sha256:6af1c6387c531cd364b72c28daa29232162010d952ceb7e5ca8e2827526aceae"}, - {file = "cryptography-41.0.3-cp37-abi3-win32.whl", hash = "sha256:0d09fb5356f975974dbcb595ad2d178305e5050656affb7890a1583f5e02a306"}, - {file = "cryptography-41.0.3-cp37-abi3-win_amd64.whl", hash = "sha256:a983e441a00a9d57a4d7c91b3116a37ae602907a7618b882c8013b5762e80574"}, - {file = "cryptography-41.0.3-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:5259cb659aa43005eb55a0e4ff2c825ca111a0da1814202c64d28a985d33b087"}, - {file = "cryptography-41.0.3-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:67e120e9a577c64fe1f611e53b30b3e69744e5910ff3b6e97e935aeb96005858"}, - {file = "cryptography-41.0.3-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:7efe8041897fe7a50863e51b77789b657a133c75c3b094e51b5e4b5cec7bf906"}, - {file = "cryptography-41.0.3-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:ce785cf81a7bdade534297ef9e490ddff800d956625020ab2ec2780a556c313e"}, - {file = "cryptography-41.0.3-pp38-pypy38_pp73-macosx_10_12_x86_64.whl", hash = "sha256:57a51b89f954f216a81c9d057bf1a24e2f36e764a1ca9a501a6964eb4a6800dd"}, - {file = "cryptography-41.0.3-pp38-pypy38_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:4c2f0d35703d61002a2bbdcf15548ebb701cfdd83cdc12471d2bae80878a4207"}, - {file = "cryptography-41.0.3-pp38-pypy38_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:23c2d778cf829f7d0ae180600b17e9fceea3c2ef8b31a99e3c694cbbf3a24b84"}, - {file = "cryptography-41.0.3-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:95dd7f261bb76948b52a5330ba5202b91a26fbac13ad0e9fc8a3ac04752058c7"}, - {file = "cryptography-41.0.3-pp39-pypy39_pp73-macosx_10_12_x86_64.whl", hash = "sha256:41d7aa7cdfded09b3d73a47f429c298e80796c8e825ddfadc84c8a7f12df212d"}, - {file = "cryptography-41.0.3-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:d0d651aa754ef58d75cec6edfbd21259d93810b73f6ec246436a21b7841908de"}, - {file = "cryptography-41.0.3-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:ab8de0d091acbf778f74286f4989cf3d1528336af1b59f3e5d2ebca8b5fe49e1"}, - {file = "cryptography-41.0.3-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:a74fbcdb2a0d46fe00504f571a2a540532f4c188e6ccf26f1f178480117b33c4"}, - {file = "cryptography-41.0.3.tar.gz", hash = "sha256:6d192741113ef5e30d89dcb5b956ef4e1578f304708701b8b73d38e3e1461f34"}, + {file = "cryptography-43.0.3-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:bf7a1932ac4176486eab36a19ed4c0492da5d97123f1406cf15e41b05e787d2e"}, + {file = "cryptography-43.0.3-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:63efa177ff54aec6e1c0aefaa1a241232dcd37413835a9b674b6e3f0ae2bfd3e"}, + {file = "cryptography-43.0.3-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7e1ce50266f4f70bf41a2c6dc4358afadae90e2a1e5342d3c08883df1675374f"}, + {file = "cryptography-43.0.3-cp37-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:443c4a81bb10daed9a8f334365fe52542771f25aedaf889fd323a853ce7377d6"}, + {file = "cryptography-43.0.3-cp37-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:74f57f24754fe349223792466a709f8e0c093205ff0dca557af51072ff47ab18"}, + {file = "cryptography-43.0.3-cp37-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:9762ea51a8fc2a88b70cf2995e5675b38d93bf36bd67d91721c309df184f49bd"}, + {file = "cryptography-43.0.3-cp37-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:81ef806b1fef6b06dcebad789f988d3b37ccaee225695cf3e07648eee0fc6b73"}, + {file = "cryptography-43.0.3-cp37-abi3-win32.whl", hash = "sha256:cbeb489927bd7af4aa98d4b261af9a5bc025bd87f0e3547e11584be9e9427be2"}, + {file = "cryptography-43.0.3-cp37-abi3-win_amd64.whl", hash = "sha256:f46304d6f0c6ab8e52770addfa2fc41e6629495548862279641972b6215451cd"}, + {file = "cryptography-43.0.3-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:8ac43ae87929a5982f5948ceda07001ee5e83227fd69cf55b109144938d96984"}, + {file = "cryptography-43.0.3-cp39-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:846da004a5804145a5f441b8530b4bf35afbf7da70f82409f151695b127213d5"}, + {file = "cryptography-43.0.3-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0f996e7268af62598f2fc1204afa98a3b5712313a55c4c9d434aef49cadc91d4"}, + {file = "cryptography-43.0.3-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:f7b178f11ed3664fd0e995a47ed2b5ff0a12d893e41dd0494f406d1cf555cab7"}, + {file = "cryptography-43.0.3-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:c2e6fc39c4ab499049df3bdf567f768a723a5e8464816e8f009f121a5a9f4405"}, + {file = "cryptography-43.0.3-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:e1be4655c7ef6e1bbe6b5d0403526601323420bcf414598955968c9ef3eb7d16"}, + {file = "cryptography-43.0.3-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:df6b6c6d742395dd77a23ea3728ab62f98379eff8fb61be2744d4679ab678f73"}, + {file = "cryptography-43.0.3-cp39-abi3-win32.whl", hash = "sha256:d56e96520b1020449bbace2b78b603442e7e378a9b3bd68de65c782db1507995"}, + {file = "cryptography-43.0.3-cp39-abi3-win_amd64.whl", hash = "sha256:0c580952eef9bf68c4747774cde7ec1d85a6e61de97281f2dba83c7d2c806362"}, + {file = "cryptography-43.0.3-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:d03b5621a135bffecad2c73e9f4deb1a0f977b9a8ffe6f8e002bf6c9d07b918c"}, + {file = "cryptography-43.0.3-pp310-pypy310_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:a2a431ee15799d6db9fe80c82b055bae5a752bef645bba795e8e52687c69efe3"}, + {file = "cryptography-43.0.3-pp310-pypy310_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:281c945d0e28c92ca5e5930664c1cefd85efe80e5c0d2bc58dd63383fda29f83"}, + {file = "cryptography-43.0.3-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:f18c716be16bc1fea8e95def49edf46b82fccaa88587a45f8dc0ff6ab5d8e0a7"}, + {file = "cryptography-43.0.3-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:4a02ded6cd4f0a5562a8887df8b3bd14e822a90f97ac5e544c162899bc467664"}, + {file = "cryptography-43.0.3-pp39-pypy39_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:53a583b6637ab4c4e3591a15bc9db855b8d9dee9a669b550f311480acab6eb08"}, + {file = "cryptography-43.0.3-pp39-pypy39_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:1ec0bcf7e17c0c5669d881b1cd38c4972fade441b27bda1051665faaa89bdcaa"}, + {file = "cryptography-43.0.3-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:2ce6fae5bdad59577b44e4dfed356944fbf1d925269114c28be377692643b4ff"}, + {file = "cryptography-43.0.3.tar.gz", hash = "sha256:315b9001266a492a6ff443b61238f956b214dbec9910a081ba5b6646a055a805"}, ] [package.dependencies] -cffi = ">=1.12" +cffi = {version = ">=1.12", markers = "platform_python_implementation != \"PyPy\""} [package.extras] docs = ["sphinx (>=5.3.0)", "sphinx-rtd-theme (>=1.1.1)"] -docstest = ["pyenchant (>=1.6.11)", "sphinxcontrib-spelling (>=4.0.1)", "twine (>=1.12.0)"] +docstest = ["pyenchant (>=1.6.11)", "readme-renderer", "sphinxcontrib-spelling (>=4.0.1)"] nox = ["nox"] -pep8test = ["black", "check-sdist", "mypy", "ruff"] +pep8test = ["check-sdist", "click", "mypy", "ruff"] sdist = ["build"] ssh = ["bcrypt (>=3.1.5)"] -test = ["pretend", "pytest (>=6.2.0)", "pytest-benchmark", "pytest-cov", "pytest-xdist"] +test = ["certifi", "cryptography-vectors (==43.0.3)", "pretend", "pytest (>=6.2.0)", "pytest-benchmark", "pytest-cov", "pytest-xdist"] test-randomorder = ["pytest-randomly"] [[package]] @@ -333,13 +341,13 @@ files = [ [[package]] name = "eventkit" -version = "1.0.1" +version = "1.0.3" description = "Event-driven data pipelines" optional = false python-versions = "*" files = [ - {file = "eventkit-1.0.1-py3-none-any.whl", hash = "sha256:6060a6aa04d5c5d20f2e55b7c17e2a22e8d31f88f2c2791d60eab3301aa040da"}, - {file = "eventkit-1.0.1.tar.gz", hash = "sha256:56b99a6205f61cd995aa5e0036e37bd61f052f7d32560e60b6fe45e319a7ef3a"}, + {file = "eventkit-1.0.3-py3-none-any.whl", hash = "sha256:0e199527a89aff9d195b9671ad45d2cc9f79ecda0900de8ecfb4c864d67ad6a2"}, + {file = "eventkit-1.0.3.tar.gz", hash = "sha256:99497f6f3c638a50ff7616f2f8cd887b18bbff3765dc1bd8681554db1467c933"}, ] [package.dependencies] @@ -347,13 +355,13 @@ numpy = "*" [[package]] name = "exceptiongroup" -version = "1.1.3" +version = "1.2.2" description = "Backport of PEP 654 (exception groups)" optional = false python-versions = ">=3.7" files = [ - {file = "exceptiongroup-1.1.3-py3-none-any.whl", hash = "sha256:343280667a4585d195ca1cf9cef84a4e178c4b6cf2274caef9859782b567d5e3"}, - {file = "exceptiongroup-1.1.3.tar.gz", hash = "sha256:097acd85d473d75af5bb98e41b61ff7fe35efe6675e4f9370ec6ec5126d160e9"}, + {file = "exceptiongroup-1.2.2-py3-none-any.whl", hash = "sha256:3111b9d131c238bec2f8f516e123e14ba243563fb135d3fe885990585aa7795b"}, + {file = "exceptiongroup-1.2.2.tar.gz", hash = "sha256:47c2edf7c6738fafb49fd34290706d1a1a2f4d1c6df275526b62cbb4aa5393cc"}, ] [package.extras] @@ -480,31 +488,37 @@ nest-asyncio = "*" [[package]] name = "idna" -version = "3.4" +version = "3.10" description = "Internationalized Domain Names in Applications (IDNA)" optional = false -python-versions = ">=3.5" +python-versions = ">=3.6" files = [ - {file = "idna-3.4-py3-none-any.whl", hash = "sha256:90b77e79eaa3eba6de819a0c442c0b4ceefc341a7a2ab77d7562bf49f425c5c2"}, - {file = "idna-3.4.tar.gz", hash = "sha256:814f528e8dead7d329833b91c5faa87d60bf71824cd12a7530b5526063d02cb4"}, + {file = "idna-3.10-py3-none-any.whl", hash = "sha256:946d195a0d259cbba61165e88e65941f16e9b36ea6ddb97f00452bae8b1287d3"}, + {file = "idna-3.10.tar.gz", hash = "sha256:12f65c9b470abda6dc35cf8e63cc574b1c52b11df2c86030af0ac09b01b13ea9"}, ] +[package.extras] +all = ["flake8 (>=7.1.1)", "mypy (>=1.11.2)", "pytest (>=8.3.2)", "ruff (>=0.6.2)"] + [[package]] name = "importlib-metadata" -version = "6.8.0" +version = "8.5.0" description = "Read metadata from Python packages" optional = false python-versions = ">=3.8" files = [ - {file = "importlib_metadata-6.8.0-py3-none-any.whl", hash = "sha256:3ebb78df84a805d7698245025b975d9d67053cd94c79245ba4b3eb694abe68bb"}, - {file = "importlib_metadata-6.8.0.tar.gz", hash = "sha256:dbace7892d8c0c4ac1ad096662232f831d4e64f4c4545bd53016a3e9d4654743"}, + {file = "importlib_metadata-8.5.0-py3-none-any.whl", hash = "sha256:45e54197d28b7a7f1559e60b95e7c567032b602131fbd588f1497f47880aa68b"}, + {file = "importlib_metadata-8.5.0.tar.gz", hash = "sha256:71522656f0abace1d072b9e5481a48f07c138e00f079c38c8f883823f9c26bd7"}, ] [package.dependencies] -zipp = ">=0.5" +zipp = ">=3.20" [package.extras] -docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)"] +cover = ["pytest-cov"] +doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +enabler = ["pytest-enabler (>=2.2)"] perf = ["ipython"] testing = ["flufl.flake8", "importlib-resources (>=1.3)", "packaging", "pyfakefs", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-mypy (>=0.9.1)", "pytest-perf (>=0.9.2)", "pytest-ruff"] @@ -644,13 +658,13 @@ files = [ [[package]] name = "nest-asyncio" -version = "1.5.7" +version = "1.6.0" description = "Patch asyncio to allow nested event loops" optional = false python-versions = ">=3.5" files = [ - {file = "nest_asyncio-1.5.7-py3-none-any.whl", hash = "sha256:5301c82941b550b3123a1ea772ba9a1c80bad3a182be8c1a5ae6ad3be57a9657"}, - {file = "nest_asyncio-1.5.7.tar.gz", hash = "sha256:6a80f7b98f24d9083ed24608977c09dd608d83f91cccc24c9d2cba6d10e01c10"}, + {file = "nest_asyncio-1.6.0-py3-none-any.whl", hash = "sha256:87af6efd6b5e897c81050477ef65c62e2b2f35d51703cae01aff2905b1852e1c"}, + {file = "nest_asyncio-1.6.0.tar.gz", hash = "sha256:6f172d5449aca15afd6c646851f4e31e02c598d553a667e38cafa997cfec55fe"}, ] [[package]] @@ -729,13 +743,13 @@ files = [ [[package]] name = "outcome" -version = "1.2.0" +version = "1.3.0.post0" description = "Capture the outcome of Python function calls." optional = false python-versions = ">=3.7" files = [ - {file = "outcome-1.2.0-py2.py3-none-any.whl", hash = "sha256:c4ab89a56575d6d38a05aa16daeaa333109c1f96167aba8901ab18b6b5e0f7f5"}, - {file = "outcome-1.2.0.tar.gz", hash = "sha256:6f82bd3de45da303cf1f771ecafa1633750a358436a8bb60e06a1ceb745d2672"}, + {file = "outcome-1.3.0.post0-py2.py3-none-any.whl", hash = "sha256:e771c5ce06d1415e356078d3bdd68523f284b4ce5419828922b6871e65eda82b"}, + {file = "outcome-1.3.0.post0.tar.gz", hash = "sha256:9dcf02e65f2971b80047b377468e72a268e15c0af3cf1238e6ff14f7f91143b8"}, ] [package.dependencies] @@ -743,13 +757,13 @@ attrs = ">=19.2.0" [[package]] name = "packaging" -version = "23.1" +version = "24.2" description = "Core utilities for Python packages" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "packaging-23.1-py3-none-any.whl", hash = "sha256:994793af429502c4ea2ebf6bf664629d07c1a9fe974af92966e4b8d2df7edc61"}, - {file = "packaging-23.1.tar.gz", hash = "sha256:a392980d2b6cffa644431898be54b0045151319d1e7ec34f0cfed48767dd334f"}, + {file = "packaging-24.2-py3-none-any.whl", hash = "sha256:09abb1bccd265c01f4a3aa3f7a7db064b36514d2cba19a2f694fe6150451a759"}, + {file = "packaging-24.2.tar.gz", hash = "sha256:c228a6dc5e932d346bc5739379109d49e8853dd8223571c7c5b55260edc0b97f"}, ] [[package]] @@ -836,24 +850,24 @@ xlsxwriter = ["xlsxwriter"] [[package]] name = "pycparser" -version = "2.21" +version = "2.22" description = "C parser in Python" optional = false -python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" +python-versions = ">=3.8" files = [ - {file = "pycparser-2.21-py2.py3-none-any.whl", hash = "sha256:8ee45429555515e1f6b185e78100aea234072576aa43ab53aefcae078162fca9"}, - {file = "pycparser-2.21.tar.gz", hash = "sha256:e644fdec12f7872f86c58ff790da456218b10f863970249516d60a5eaca77206"}, + {file = "pycparser-2.22-py3-none-any.whl", hash = "sha256:c3702b6d3dd8c7abc1afa565d7e63d53a1d0bd86cdc24edd75470f4de499cfcc"}, + {file = "pycparser-2.22.tar.gz", hash = "sha256:491c8be9c040f5390f5bf44a5b07752bd07f56edf992381b05c701439eec10f6"}, ] [[package]] name = "pygments" -version = "2.16.1" +version = "2.18.0" description = "Pygments is a syntax highlighting package written in Python." optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "Pygments-2.16.1-py3-none-any.whl", hash = "sha256:13fc09fa63bc8d8671a6d247e1eb303c4b343eaee81d861f3404db2935653692"}, - {file = "Pygments-2.16.1.tar.gz", hash = "sha256:1daff0494820c69bc8941e407aa20f577374ee88364ee10a98fdbe0aece96e29"}, + {file = "pygments-2.18.0-py3-none-any.whl", hash = "sha256:b8e6aca0523f3ab76fee51799c488e38782ac06eafcf95e7ba832985c8e7b13a"}, + {file = "pygments-2.18.0.tar.gz", hash = "sha256:786ff802f32e91311bff3889f6e9a86e81505fe99f2735bb6d60ae0c5004f199"}, ] [package.extras] @@ -861,24 +875,24 @@ plugins = ["importlib-metadata"] [[package]] name = "pyreadline3" -version = "3.4.1" +version = "3.5.4" description = "A python implementation of GNU readline." optional = false -python-versions = "*" +python-versions = ">=3.8" files = [ - {file = "pyreadline3-3.4.1-py3-none-any.whl", hash = "sha256:b0efb6516fd4fb07b45949053826a62fa4cb353db5be2bbb4a7aa1fdd1e345fb"}, - {file = "pyreadline3-3.4.1.tar.gz", hash = "sha256:6f3d1f7b8a31ba32b73917cefc1f28cc660562f39aea8646d30bd6eff21f7bae"}, + {file = "pyreadline3-3.5.4-py3-none-any.whl", hash = "sha256:eaf8e6cc3c49bcccf145fc6067ba8643d1df34d604a1ec0eccbf7a18e6d3fae6"}, + {file = "pyreadline3-3.5.4.tar.gz", hash = "sha256:8d57d53039a1c75adba8e50dd3d992b28143480816187ea5efbd5c78e6c885b7"}, ] [[package]] name = "python-dateutil" -version = "2.8.2" +version = "2.9.0.post0" description = "Extensions to the standard Python datetime module" optional = false python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,>=2.7" files = [ - {file = "python-dateutil-2.8.2.tar.gz", hash = "sha256:0123cacc1627ae19ddf3c27a5de5bd67ee4586fbdd6440d9748f8abb483d3e86"}, - {file = "python_dateutil-2.8.2-py2.py3-none-any.whl", hash = "sha256:961d03dc3453ebbc59dbdea9e4e11c5651520a876d0f4db161e8674aae935da9"}, + {file = "python-dateutil-2.9.0.post0.tar.gz", hash = "sha256:37dd54208da7e1cd875388217d5e00ebd4179249f90fb72437e91a35459a0ad3"}, + {file = "python_dateutil-2.9.0.post0-py2.py3-none-any.whl", hash = "sha256:a8b2bc7bffae282281c8140a97d3aa9c14da0b136dfe83f850eea9a5f7470427"}, ] [package.dependencies] @@ -897,13 +911,13 @@ files = [ [[package]] name = "rich" -version = "13.5.2" +version = "13.9.4" description = "Render rich text, tables, progress bars, syntax highlighting, markdown and more to the terminal" optional = false -python-versions = ">=3.7.0" +python-versions = ">=3.8.0" files = [ - {file = "rich-13.5.2-py3-none-any.whl", hash = "sha256:146a90b3b6b47cac4a73c12866a499e9817426423f57c5a66949c086191a8808"}, - {file = "rich-13.5.2.tar.gz", hash = "sha256:fb9d6c0a0f643c99eed3875b5377a184132ba9be4d61516a55273d3554d75a39"}, + {file = "rich-13.9.4-py3-none-any.whl", hash = "sha256:6049d5e6ec054bf2779ab3358186963bac2ea89175919d699e378b99738c2a90"}, + {file = "rich-13.9.4.tar.gz", hash = "sha256:439594978a49a09530cff7ebc4b5c7103ef57baf48d5ea3184f21d9a2befa098"}, ] [package.dependencies] @@ -926,13 +940,13 @@ files = [ [[package]] name = "sniffio" -version = "1.3.0" +version = "1.3.1" description = "Sniff out which async library your code is running under" optional = false python-versions = ">=3.7" files = [ - {file = "sniffio-1.3.0-py3-none-any.whl", hash = "sha256:eecefdce1e5bbfb7ad2eeaabf7c1eeb404d7757c379bd1f7e5cce9d8bf425384"}, - {file = "sniffio-1.3.0.tar.gz", hash = "sha256:e60305c5e5d314f5389259b7f22aaa33d8f7dee49763119234af3755c55b9101"}, + {file = "sniffio-1.3.1-py3-none-any.whl", hash = "sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2"}, + {file = "sniffio-1.3.1.tar.gz", hash = "sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc"}, ] [[package]] @@ -962,24 +976,24 @@ pyreadline3 = {version = "*", markers = "platform_system == \"Windows\""} [[package]] name = "tomli" -version = "2.0.1" +version = "2.1.0" 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.1.0-py3-none-any.whl", hash = "sha256:a5c57c3d1c56f5ccdf89f6523458f60ef716e210fc47c4cfb188c5ba473e0391"}, + {file = "tomli-2.1.0.tar.gz", hash = "sha256:3f646cae2aec94e17d04973e4249548320197cfabdf130015d023de4b74d8ab8"}, ] [[package]] name = "tomli-w" -version = "1.0.0" +version = "1.1.0" description = "A lil' TOML writer" optional = false -python-versions = ">=3.7" +python-versions = ">=3.9" files = [ - {file = "tomli_w-1.0.0-py3-none-any.whl", hash = "sha256:9f2a07e8be30a0729e533ec968016807069991ae2fd921a78d42f429ae5f4463"}, - {file = "tomli_w-1.0.0.tar.gz", hash = "sha256:f463434305e0336248cac9c2dc8076b707d8a12d019dd349f5c1e382dd1ae1b9"}, + {file = "tomli_w-1.1.0-py3-none-any.whl", hash = "sha256:1403179c78193e3184bfaade390ddbd071cba48a32a2e62ba11aae47490c63f7"}, + {file = "tomli_w-1.1.0.tar.gz", hash = "sha256:49e847a3a304d516a169a601184932ef0f6b61623fe680f836a2aa7128ed0d33"}, ] [[package]] @@ -1114,13 +1128,13 @@ wsproto = ">=0.14" [[package]] name = "typer" -version = "0.9.0" +version = "0.9.4" description = "Typer, build great CLIs. Easy to code. Based on Python type hints." optional = false python-versions = ">=3.6" files = [ - {file = "typer-0.9.0-py3-none-any.whl", hash = "sha256:5d96d986a21493606a358cae4461bd8cdf83cbf33a5aa950ae629ca3b51467ee"}, - {file = "typer-0.9.0.tar.gz", hash = "sha256:50922fd79aea2f4751a8e0408ff10d2662bd0c8bbfa84755a699f3bada2978b2"}, + {file = "typer-0.9.4-py3-none-any.whl", hash = "sha256:aa6c4a4e2329d868b80ecbaf16f807f2b54e192209d7ac9dd42691d63f7a54eb"}, + {file = "typer-0.9.4.tar.gz", hash = "sha256:f714c2d90afae3a7929fcd72a3abb08df305e1ff61719381384211c4070af57f"}, ] [package.dependencies] @@ -1131,17 +1145,17 @@ typing-extensions = ">=3.7.4.3" all = ["colorama (>=0.4.3,<0.5.0)", "rich (>=10.11.0,<14.0.0)", "shellingham (>=1.3.0,<2.0.0)"] dev = ["autoflake (>=1.3.1,<2.0.0)", "flake8 (>=3.8.3,<4.0.0)", "pre-commit (>=2.17.0,<3.0.0)"] doc = ["cairosvg (>=2.5.2,<3.0.0)", "mdx-include (>=1.4.1,<2.0.0)", "mkdocs (>=1.1.2,<2.0.0)", "mkdocs-material (>=8.1.4,<9.0.0)", "pillow (>=9.3.0,<10.0.0)"] -test = ["black (>=22.3.0,<23.0.0)", "coverage (>=6.2,<7.0)", "isort (>=5.0.6,<6.0.0)", "mypy (==0.910)", "pytest (>=4.4.0,<8.0.0)", "pytest-cov (>=2.10.0,<5.0.0)", "pytest-sugar (>=0.9.4,<0.10.0)", "pytest-xdist (>=1.32.0,<4.0.0)", "rich (>=10.11.0,<14.0.0)", "shellingham (>=1.3.0,<2.0.0)"] +test = ["black (>=22.3.0,<23.0.0)", "coverage (>=6.2,<7.0)", "isort (>=5.0.6,<6.0.0)", "mypy (==0.971)", "pytest (>=4.4.0,<8.0.0)", "pytest-cov (>=2.10.0,<5.0.0)", "pytest-sugar (>=0.9.4,<0.10.0)", "pytest-xdist (>=1.32.0,<4.0.0)", "rich (>=10.11.0,<14.0.0)", "shellingham (>=1.3.0,<2.0.0)"] [[package]] name = "typing-extensions" -version = "4.7.1" -description = "Backported and Experimental Type Hints for Python 3.7+" +version = "4.12.2" +description = "Backported and Experimental Type Hints for Python 3.8+" optional = false -python-versions = ">=3.7" +python-versions = ">=3.8" files = [ - {file = "typing_extensions-4.7.1-py3-none-any.whl", hash = "sha256:440d5dd3af93b060174bf433bccd69b0babc3b15b1a8dca43789fd7f61514b36"}, - {file = "typing_extensions-4.7.1.tar.gz", hash = "sha256:b75ddc264f0ba5615db7ba217daeb99701ad295353c45f9e95963337ceeeffb2"}, + {file = "typing_extensions-4.12.2-py3-none-any.whl", hash = "sha256:04e5ca0351e0f3f85c6853954072df659d0d13fac324d0072316b67d7794700d"}, + {file = "typing_extensions-4.12.2.tar.gz", hash = "sha256:1a7ead55c7e559dd4dee8856e3a88b41225abfe1ce8df57b7c13915fe121ffb8"}, ] [[package]] @@ -1244,18 +1258,22 @@ h11 = ">=0.9.0,<1" [[package]] name = "zipp" -version = "3.16.2" +version = "3.21.0" description = "Backport of pathlib-compatible object wrapper for zip files" optional = false -python-versions = ">=3.8" +python-versions = ">=3.9" files = [ - {file = "zipp-3.16.2-py3-none-any.whl", hash = "sha256:679e51dd4403591b2d6838a48de3d283f3d188412a9782faadf845f298736ba0"}, - {file = "zipp-3.16.2.tar.gz", hash = "sha256:ebc15946aa78bd63458992fc81ec3b6f7b1e92d51c35e6de1c3804e73b799147"}, + {file = "zipp-3.21.0-py3-none-any.whl", hash = "sha256:ac1bbe05fd2991f160ebce24ffbac5f6d11d83dc90891255885223d42b3cd931"}, + {file = "zipp-3.21.0.tar.gz", hash = "sha256:2c9958f6430a2040341a52eb608ed6dd93ef4392e02ffe219417c1b28b5dd1f4"}, ] [package.extras] -docs = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] -testing = ["big-O", "jaraco.functools", "jaraco.itertools", "more-itertools", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=2.2)", "pytest-ignore-flaky", "pytest-mypy (>=0.9.1)", "pytest-ruff"] +check = ["pytest-checkdocs (>=2.4)", "pytest-ruff (>=0.2.1)"] +cover = ["pytest-cov"] +doc = ["furo", "jaraco.packaging (>=9.3)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)", "sphinx-lint"] +enabler = ["pytest-enabler (>=2.2)"] +test = ["big-O", "importlib-resources", "jaraco.functools", "jaraco.itertools", "jaraco.test", "more-itertools", "pytest (>=6,!=8.1.*)", "pytest-ignore-flaky"] +type = ["pytest-mypy"] [metadata] lock-version = "2.0" diff --git a/pyproject.toml b/pyproject.toml index e242ef00..1d73f459 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -25,11 +25,11 @@ build-backend = "hatchling.build" ignore = [] # https://docs.astral.sh/ruff/settings/#lint_per-file-ignores -"piker/ui/qt.py" = [ - "E402", - 'F401', # unused imports (without __all__ or blah as blah) - # "F841", # unused variable rules -] +# "piker/ui/qt.py" = [ +# "E402", +# 'F401', # unused imports (without __all__ or blah as blah) +# # "F841", # unused variable rules +# ] # ignore-init-module-imports = false # ------ - ------