From 430047078610015b35243df534b4bb88d425f826 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 15 Nov 2022 18:10:51 -0500 Subject: [PATCH 01/54] Fix for empty tsdb query result case --- piker/data/feed.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index 93630a13..f291c512 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -690,14 +690,13 @@ async def tsdb_backfill( timeframe=timeframe, ) + # empty query + if not len(tsdb_history): + break + next_start = tsdb_history['Epoch'][0] - if ( - not len(tsdb_history) # empty query - + if next_start >= tsdb_last_frame_start: # no earlier data detected - or next_start >= tsdb_last_frame_start - - ): break else: tsdb_last_frame_start = next_start From 43717c92d9e4a472286e241105b33c4c76ad5e52 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 16 Nov 2022 13:41:20 -0500 Subject: [PATCH 03/54] Type annot-declare fsp-engine data `Feed` --- piker/fsp/_engine.py | 2 ++ piker/ui/_fsp.py | 5 ++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index 5d389e29..c47455e3 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -36,6 +36,7 @@ from .. import data from ..data import attach_shm_array from ..data.feed import ( Flume, + Feed, ) from ..data._sharedmem import ShmArray from ..data._sampling import _default_delay_s @@ -302,6 +303,7 @@ async def cascade( raise ValueError(f'Unknown fsp target: {ns_path}') # open a data feed stream with requested broker + feed: Feed async with data.feed.maybe_open_feed( [fqsn], diff --git a/piker/ui/_fsp.py b/piker/ui/_fsp.py index dfbd8b9a..9e05f545 100644 --- a/piker/ui/_fsp.py +++ b/piker/ui/_fsp.py @@ -51,7 +51,10 @@ from ._forms import ( mk_form, open_form_input_handling, ) -from ..fsp._api import maybe_mk_fsp_shm, Fsp +from ..fsp._api import ( + maybe_mk_fsp_shm, + Fsp, +) from ..fsp import cascade from ..fsp._volume import ( # tina_vwap, From 715e693564773789fdb3760e6a71de9a51ae7e07 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 16 Nov 2022 17:39:54 -0500 Subject: [PATCH 04/54] Improved clearing-tick-burst-oriented throttling Instead of uniformly distributing the msg send rate for a given aggregate subscription, choose to be more bursty around clearing ticks so as to avoid saturating the consumer with L1 book updates and vs. delivering real trade data as-fast-as-possible. Presuming the consumer is in the "UI land of slow" (eg. modern display frame rates) such an approach serves more useful for seeing "material changes" in the market as-bursty-as-possible (i.e. more short lived fast changes in last clearing price vs. many slower changes in the bid-ask spread queues). Such an approach also lends better to multi-feed overlays which in aggregate tend to scale linearly with the number of feeds/overlays; centralization of bursty arrival rates allows for a higher overall throttle rate if used cleverly with framing. --- piker/data/_sampling.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index f8230bd7..a2017780 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -383,6 +383,7 @@ async def sample_and_broadcast( trio.ClosedResourceError, trio.EndOfChannel, ): + ctx = stream._ctx chan = ctx.chan if ctx: log.warning( @@ -427,6 +428,7 @@ async def uniform_rate_send( diff = 0 task_status.started() + types: set[str] = set() while True: @@ -473,6 +475,7 @@ async def uniform_rate_send( # at it's display rate. if ticks: first_quote['ticks'].extend(ticks) + types.update(item['type'] for item in ticks) # send cycle isn't due yet so continue waiting continue @@ -491,6 +494,20 @@ async def uniform_rate_send( # we have a quote already so send it now. + with trio.move_on_after(1/60) as cs: + while ( + not types.intersection({'trade', 'utrade', 'last'}) + ): + try: + sym, last_quote = await quote_stream.receive() + except trio.EndOfChannel: + log.exception(f"feed for {stream} ended?") + break + + ticks = last_quote.get('ticks') + first_quote['ticks'].extend(ticks) + types.update(item['type'] for item in ticks) + # measured_rate = 1 / (time.time() - last_send) # log.info( # f'`{sym}` throttled send hz: {round(measured_rate, ndigits=1)}' @@ -520,3 +537,4 @@ async def uniform_rate_send( first_quote = last_quote = None diff = 0 last_send = time.time() + types.clear() From 125e31dbf37b58835fb320cbf43757cc9d387666 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 17 Nov 2022 17:28:26 -0500 Subject: [PATCH 05/54] Implement by-type tick-framing in throttler loop This has been an outstanding idea for a while and changes the framing format of tick events into a `dict[str, list[dict]]` wherein for each tick "type" (eg. 'bid', 'ask', 'trade', 'asize'..etc) we create an FIFO ordered `list` of events (data) and then pack this table into each (throttled) send. This gives an additional implied downsample reduction (in terms of iteration on the consumer side) from `N` tick-events to a (max) `T` tick-types presuming the rx side only needs the latest tick event. Drop the `types: set` and adjust clearing event test to use the new `ticks_by_type` map's keys. --- piker/data/_sampling.py | 75 ++++++++++++++++++++++++++++++----------- 1 file changed, 56 insertions(+), 19 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index a2017780..32b753a2 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -20,7 +20,10 @@ financial data flows. """ from __future__ import annotations -from collections import Counter +from collections import ( + Counter, + defaultdict, +) import time from typing import ( TYPE_CHECKING, @@ -405,10 +408,17 @@ async def sample_and_broadcast( ) +# a working tick-type-classes template +_tick_groups = { + 'clears': {'trade', 'utrade', 'last'}, + 'bids': {'bid', 'bsize'}, + 'asks': {'ask', 'asize'}, +} + + # TODO: a less naive throttler, here's some snippets: # token bucket by njs: # https://gist.github.com/njsmith/7ea44ec07e901cb78ebe1dd8dd846cb9 - async def uniform_rate_send( rate: float, @@ -428,7 +438,12 @@ async def uniform_rate_send( diff = 0 task_status.started() - types: set[str] = set() + ticks_by_type: defaultdict[ + str, + list[dict], + ] = defaultdict(list) + + clear_types = _tick_groups['clears'] while True: @@ -457,25 +472,41 @@ async def uniform_rate_send( # tick array/buffer. ticks = last_quote.get('ticks') - # XXX: idea for frame type data structure we could - # use on the wire instead of a simple list? - # frames = { - # 'index': ['type_a', 'type_c', 'type_n', 'type_n'], - - # 'type_a': [tick0, tick1, tick2, .., tickn], - # 'type_b': [tick0, tick1, tick2, .., tickn], - # 'type_c': [tick0, tick1, tick2, .., tickn], - # ... - # 'type_n': [tick0, tick1, tick2, .., tickn], - # } - # TODO: once we decide to get fancy really we should # have a shared mem tick buffer that is just # continually filled and the UI just ready from it # at it's display rate. if ticks: + # TODO: do we need this any more or can we just + # expect the receiver to unwind the below + # `ticks_by_type: dict`? + # => undwinding would potentially require a + # `dict[str, set | list]` instead with an + # included `'types' field which is an (ordered) + # set of tick type fields in the order which + # types arrived? first_quote['ticks'].extend(ticks) - types.update(item['type'] for item in ticks) + + # XXX: build a tick-by-type table of lists + # of tick messages. This allows for less + # iteration on the receiver side by allowing for + # a single "latest tick event" look up by + # indexing the last entry in each sub-list. + # tbt = { + # 'types': ['bid', 'asize', 'last', .. ''], + + # 'bid': [tick0, tick1, tick2, .., tickn], + # 'asize': [tick0, tick1, tick2, .., tickn], + # 'last': [tick0, tick1, tick2, .., tickn], + # ... + # '': [tick0, tick1, tick2, .., tickn], + # } + for tick in ticks: + # append in reverse FIFO order for in-order + # iteration on receiver side. + ticks_by_type[tick['type']].append(tick) + + first_quote['tbt'] = ticks_by_type # send cycle isn't due yet so continue waiting continue @@ -496,7 +527,7 @@ async def uniform_rate_send( with trio.move_on_after(1/60) as cs: while ( - not types.intersection({'trade', 'utrade', 'last'}) + not set(ticks_by_type).intersection(clear_types) ): try: sym, last_quote = await quote_stream.receive() @@ -506,7 +537,13 @@ async def uniform_rate_send( ticks = last_quote.get('ticks') first_quote['ticks'].extend(ticks) - types.update(item['type'] for item in ticks) + if ticks: + for tick in ticks: + # append in reverse FIFO order for in-order + # iteration on receiver side. + ticks_by_type[tick['type']].append(tick) + + first_quote['tbt'] = ticks_by_type # measured_rate = 1 / (time.time() - last_send) # log.info( @@ -537,4 +574,4 @@ async def uniform_rate_send( first_quote = last_quote = None diff = 0 last_send = time.time() - types.clear() + ticks_by_type.clear() From f2df32a673cd0c0ff5d78e43a4326fc004442265 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 17 Nov 2022 18:25:46 -0500 Subject: [PATCH 06/54] Use throttle period for wait-on-clearing-event timeout --- piker/data/_sampling.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index 32b753a2..143f1c19 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -525,7 +525,7 @@ async def uniform_rate_send( # we have a quote already so send it now. - with trio.move_on_after(1/60) as cs: + with trio.move_on_after(throttle_period) as cs: while ( not set(ticks_by_type).intersection(clear_types) ): From 1ee49df31d166dcfcd3f33d57a45dc85415d843c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 21 Nov 2022 20:25:33 -0500 Subject: [PATCH 07/54] Ensure a rt shm buffer without backfill has correct epoch timestamping --- piker/data/feed.py | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index f291c512..2f5313e1 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -25,6 +25,7 @@ from collections import defaultdict from contextlib import asynccontextmanager as acm from datetime import datetime from functools import partial +import time from types import ModuleType from typing import ( Any, @@ -678,6 +679,10 @@ async def tsdb_backfill( tsdb_last_frame_start = tsdb_history['Epoch'][0] + if timeframe == 1: + times = shm.array['time'] + assert (times[1] - times[0]) == 1 + # load as much from storage into shm possible (depends on # user's shm size settings). while ( @@ -1204,7 +1209,12 @@ async def allocate_persistent_feed( rt_shm.push(hist_shm.array[-3:-1]) ohlckeys = ['open', 'high', 'low', 'close'] rt_shm.array[ohlckeys][-2:] = hist_shm.array['close'][-1] - rt_shm.array['volume'][-2] = 0 + rt_shm.array['volume'][-2:] = 0 + + # set fast buffer time step to 1s + ts = round(time.time()) + rt_shm.array['time'][0] = ts + rt_shm.array['time'][1] = ts + 1 # wait the spawning parent task to register its subscriber # send-stream entry before we start the sample loop. From 7da5c2b238942331ffe592d64aafe4d7852fa79b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 23 Nov 2022 15:06:42 -0500 Subject: [PATCH 08/54] Add epoch time index to fsp buffers --- piker/fsp/_api.py | 5 ++++- piker/fsp/_engine.py | 32 ++++++++++++++++++++++++++------ 2 files changed, 30 insertions(+), 7 deletions(-) diff --git a/piker/fsp/_api.py b/piker/fsp/_api.py index f4e42bc1..9654a2a1 100644 --- a/piker/fsp/_api.py +++ b/piker/fsp/_api.py @@ -199,7 +199,10 @@ def maybe_mk_fsp_shm( # TODO: load output types from `Fsp` # - should `index` be a required internal field? fsp_dtype = np.dtype( - [('index', int)] + + [('index', int)] + + + [('time', float)] + + [(field_name, float) for field_name in target.outputs] ) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index c47455e3..edef7219 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -132,7 +132,10 @@ async def fsp_compute( fields.pop('index') history: Optional[np.ndarray] = None # TODO: nptyping here! - if fields and len(fields) > 1 and fields: + if ( + fields and + len(fields) > 1 + ): if not isinstance(history_output, dict): raise ValueError( f'`{func_name}` is a multi-output FSP and should yield a ' @@ -217,8 +220,14 @@ async def fsp_compute( log.debug(f"{func_name}: {processed}") key, output = processed - index = src.index - dst.array[-1][key] = output + # dst.array[-1][key] = output + dst.array[[key, 'time']][-1] = ( + output, + # TODO: what about pushing ``time.time_ns()`` + # in which case we'll need to round at the graphics + # processing / sampling layer? + src.array[-1]['time'] + ) # NOTE: for now we aren't streaming this to the consumer # stream latest array index entry which basically just acts @@ -229,6 +238,7 @@ async def fsp_compute( # N-consumers who subscribe for the real-time output, # which we'll likely want to implement using local-mem # chans for the fan out? + # index = src.index # if attach_stream: # await client_stream.send(index) @@ -388,7 +398,8 @@ async def cascade( src: ShmArray, dst: ShmArray ) -> tuple[bool, int, int]: - '''Predicate to dertmine if a destination FSP + ''' + Predicate to dertmine if a destination FSP output array is aligned to its source array. ''' @@ -406,11 +417,9 @@ async def cascade( ), step_diff, len_diff async def poll_and_sync_to_step( - tracker: TaskTracker, src: ShmArray, dst: ShmArray, - ) -> tuple[TaskTracker, int]: synced, step_diff, _ = is_synced(src, dst) @@ -469,4 +478,15 @@ async def cascade( else: last = array[-1:].copy() + # sync with source time step + src_t = src.array['time'][-1] + last['time'] = src_t + dst.push(last) + + # dst_t = dst.array['time'][-1] + # print( + # f'{dst.token}\n' + # f'src: {src_t}\n' + # f'dst: {dst_t}\n' + # ) From e5e70a6011d5ae7af6c1bdb3983e81b72866d72a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 24 Nov 2022 11:51:56 -0500 Subject: [PATCH 09/54] Extend `Flume` methods Add some (untested) data slicing util methods for mapping time ranges to source data indices: - `.get_index()` which maps a single input epoch time to an equiv array (int) index. - add `slice_from_time()` which returns a view of the shm data from an input epoch range presuming the underlying struct array contains a `'time'` field with epoch stamps. - `.view_data()` which slices out the "in view" data according to the current state of the passed in `pg.PlotItem`'s view box. --- piker/data/feed.py | 108 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 108 insertions(+) diff --git a/piker/data/feed.py b/piker/data/feed.py index 2f5313e1..e51b5f6d 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -87,6 +87,7 @@ from ..brokers._util import ( if TYPE_CHECKING: from .marketstore import Storage + from pyqtgraph import PlotItem log = get_logger(__name__) @@ -1037,6 +1038,113 @@ class Flume(Struct): **msg, ) + def get_index( + self, + time_s: float, + + ) -> int: + ''' + Return array shm-buffer index for for epoch time. + + ''' + array = self.rt_shm.array + times = array['time'] + mask = (times >= time_s) + + if any(mask): + return array['index'][mask][0] + + # just the latest index + array['index'][-1] + + def slice_from_time( + self, + array: np.ndarray, + start_t: float, + stop_t: float, + timeframe_s: int = 1, + return_data: bool = False, + + ) -> np.ndarray: + ''' + Slice an input struct array providing only datums + "in view" of this chart. + + ''' + arr = { + 1: self.rt_shm.array, + 60: self.hist_shm.arry, + }[timeframe_s] + + times = arr['time'] + index = array['index'] + + # use advanced indexing to map the + # time range to the index range. + mask = ( + (times >= start_t) + & + (times < stop_t) + ) + + # TODO: if we can ensure each time field has a uniform + # step we can instead do some arithmetic to determine + # the equivalent index like we used to? + # return array[ + # lbar - ifirst: + # (rbar - ifirst) + 1 + # ] + + i_by_t = index[mask] + i_0 = i_by_t[0] + + abs_slc = slice( + i_0, + i_by_t[-1], + ) + # slice data by offset from the first index + # available in the passed datum set. + read_slc = slice( + 0, + i_by_t[-1] - i_0, + ) + if not return_data: + return ( + abs_slc, + read_slc, + ) + + # also return the readable data from the timerange + return ( + abs_slc, + read_slc, + arr[mask], + ) + + def view_data( + self, + plot: PlotItem, + timeframe_s: int = 1, + + ) -> np.ndarray: + + # get far-side x-indices plot view + vr = plot.viewRect() + l = vr.left() + r = vr.right() + + ( + abs_slc, + buf_slc, + iv_arr, + ) = self.slice_from_time( + start_t=l, + stop_t=r, + timeframe_s=timeframe_s, + return_data=True, + ) + return iv_arr + async def allocate_persistent_feed( bus: _FeedsBus, From eacd44dd65dd25b7b5242333dcb37069e618af44 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 24 Nov 2022 13:02:12 -0500 Subject: [PATCH 10/54] Move `Flume` to a new `.data._flumes` module --- piker/data/_flumes.py | 304 ++++++++++++++++++++++++++++++++++++++++++ piker/data/feed.py | 258 +---------------------------------- 2 files changed, 305 insertions(+), 257 deletions(-) create mode 100644 piker/data/_flumes.py diff --git a/piker/data/_flumes.py b/piker/data/_flumes.py new file mode 100644 index 00000000..82ce2947 --- /dev/null +++ b/piker/data/_flumes.py @@ -0,0 +1,304 @@ +# piker: trading gear for hackers +# Copyright (C) Tyler Goodlet (in stewardship for pikers) + +# This program is free software: you can redistribute it and/or modify +# it under the terms of the GNU Affero General Public License as published by +# the Free Software Foundation, either version 3 of the License, or +# (at your option) any later version. + +# This program is distributed in the hope that it will be useful, +# but WITHOUT ANY WARRANTY; without even the implied warranty of +# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +# GNU Affero General Public License for more details. + +# You should have received a copy of the GNU Affero General Public License +# along with this program. If not, see . + +""" +Real-time data flow abstractions. + + + +""" +from contextlib import asynccontextmanager as acm +from functools import partial +from typing import ( + AsyncIterator, + TYPE_CHECKING, +) + +import tractor +from tractor.trionics import ( + maybe_open_context, +) +import pendulum +import numpy as np + +from .types import Struct +from ._source import ( + Symbol, +) +from ._sharedmem import ( + attach_shm_array, + ShmArray, + _Token, +) +from ._sampling import ( + iter_ohlc_periods, +) + +if TYPE_CHECKING: + from pyqtgraph import PlotItem + from .feed import Feed + + +class Flume(Struct): + ''' + Composite reference type which points to all the addressing handles + and other meta-data necessary for the read, measure and management + of a set of real-time updated data flows. + + Can be thought of as a "flow descriptor" or "flow frame" which + describes the high level properties of a set of data flows that can + be used seamlessly across process-memory boundaries. + + Each instance's sub-components normally includes: + - a msg oriented quote stream provided via an IPC transport + - history and real-time shm buffers which are both real-time + updated and backfilled. + - associated startup indexing information related to both buffer + real-time-append and historical prepend addresses. + - low level APIs to read and measure the updated data and manage + queuing properties. + + ''' + symbol: Symbol + first_quote: dict + _hist_shm_token: _Token + _rt_shm_token: _Token + + # private shm refs loaded dynamically from tokens + _hist_shm: ShmArray | None = None + _rt_shm: ShmArray | None = None + + stream: tractor.MsgStream | None = None + izero_hist: int = 0 + izero_rt: int = 0 + throttle_rate: int | None = None + + # TODO: do we need this really if we can pull the `Portal` from + # ``tractor``'s internals? + feed: Feed | None = None + + @property + def rt_shm(self) -> ShmArray: + + if self._rt_shm is None: + self._rt_shm = attach_shm_array( + token=self._rt_shm_token, + readonly=True, + ) + + return self._rt_shm + + @property + def hist_shm(self) -> ShmArray: + + if self._hist_shm is None: + self._hist_shm = attach_shm_array( + token=self._hist_shm_token, + readonly=True, + ) + + return self._hist_shm + + async def receive(self) -> dict: + return await self.stream.receive() + + @acm + async def index_stream( + self, + delay_s: int = 1, + + ) -> AsyncIterator[int]: + + if not self.feed: + raise RuntimeError('This flume is not part of any ``Feed``?') + + # TODO: maybe a public (property) API for this in ``tractor``? + portal = self.stream._ctx._portal + assert portal + + # XXX: this should be singleton on a host, + # a lone broker-daemon per provider should be + # created for all practical purposes + async with maybe_open_context( + acm_func=partial( + portal.open_context, + iter_ohlc_periods, + ), + kwargs={'delay_s': delay_s}, + ) as (cache_hit, (ctx, first)): + async with ctx.open_stream() as istream: + if cache_hit: + # add a new broadcast subscription for the quote stream + # if this feed is likely already in use + async with istream.subscribe() as bistream: + yield bistream + else: + yield istream + + def get_ds_info( + self, + ) -> tuple[float, float, float]: + ''' + Compute the "downsampling" ratio info between the historical shm + buffer and the real-time (HFT) one. + + Return a tuple of the fast sample period, historical sample + period and ratio between them. + + ''' + times = self.hist_shm.array['time'] + end = pendulum.from_timestamp(times[-1]) + start = pendulum.from_timestamp(times[times != times[-1]][-1]) + hist_step_size_s = (end - start).seconds + + times = self.rt_shm.array['time'] + end = pendulum.from_timestamp(times[-1]) + start = pendulum.from_timestamp(times[times != times[-1]][-1]) + rt_step_size_s = (end - start).seconds + + ratio = hist_step_size_s / rt_step_size_s + return ( + rt_step_size_s, + hist_step_size_s, + ratio, + ) + + # TODO: get native msgspec decoding for these workinn + def to_msg(self) -> dict: + msg = self.to_dict() + msg['symbol'] = msg['symbol'].to_dict() + + # can't serialize the stream or feed objects, it's expected + # you'll have a ref to it since this msg should be rxed on + # a stream on whatever far end IPC.. + msg.pop('stream') + msg.pop('feed') + return msg + + @classmethod + def from_msg(cls, msg: dict) -> dict: + symbol = Symbol(**msg.pop('symbol')) + return cls( + symbol=symbol, + **msg, + ) + + def get_index( + self, + time_s: float, + + ) -> int: + ''' + Return array shm-buffer index for for epoch time. + + ''' + array = self.rt_shm.array + times = array['time'] + mask = (times >= time_s) + + if any(mask): + return array['index'][mask][0] + + # just the latest index + array['index'][-1] + + def slice_from_time( + self, + array: np.ndarray, + start_t: float, + stop_t: float, + timeframe_s: int = 1, + return_data: bool = False, + + ) -> np.ndarray: + ''' + Slice an input struct array providing only datums + "in view" of this chart. + + ''' + arr = { + 1: self.rt_shm.array, + 60: self.hist_shm.arry, + }[timeframe_s] + + times = arr['time'] + index = array['index'] + + # use advanced indexing to map the + # time range to the index range. + mask = ( + (times >= start_t) + & + (times < stop_t) + ) + + # TODO: if we can ensure each time field has a uniform + # step we can instead do some arithmetic to determine + # the equivalent index like we used to? + # return array[ + # lbar - ifirst: + # (rbar - ifirst) + 1 + # ] + + i_by_t = index[mask] + i_0 = i_by_t[0] + + abs_slc = slice( + i_0, + i_by_t[-1], + ) + # slice data by offset from the first index + # available in the passed datum set. + read_slc = slice( + 0, + i_by_t[-1] - i_0, + ) + if not return_data: + return ( + abs_slc, + read_slc, + ) + + # also return the readable data from the timerange + return ( + abs_slc, + read_slc, + arr[mask], + ) + + def view_data( + self, + plot: PlotItem, + timeframe_s: int = 1, + + ) -> np.ndarray: + + # get far-side x-indices plot view + vr = plot.viewRect() + l = vr.left() + r = vr.right() + + ( + abs_slc, + buf_slc, + iv_arr, + ) = self.slice_from_time( + start_t=l, + stop_t=r, + timeframe_s=timeframe_s, + return_data=True, + ) + return iv_arr diff --git a/piker/data/feed.py b/piker/data/feed.py index e51b5f6d..59a7b303 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -29,7 +29,6 @@ import time from types import ModuleType from typing import ( Any, - AsyncIterator, AsyncContextManager, Callable, Optional, @@ -57,11 +56,10 @@ from .._daemon import ( maybe_spawn_brokerd, check_for_service, ) +from ._flumes import Flume from ._sharedmem import ( maybe_open_shm_array, - attach_shm_array, ShmArray, - _Token, _secs_in_day, ) from .ingest import get_ingestormod @@ -76,7 +74,6 @@ from ._sampling import ( sampler, broadcast, increment_ohlc_buffer, - iter_ohlc_periods, sample_and_broadcast, uniform_rate_send, _default_delay_s, @@ -87,7 +84,6 @@ from ..brokers._util import ( if TYPE_CHECKING: from .marketstore import Storage - from pyqtgraph import PlotItem log = get_logger(__name__) @@ -894,258 +890,6 @@ async def manage_history( await trio.sleep_forever() -class Flume(Struct): - ''' - Composite reference type which points to all the addressing handles - and other meta-data necessary for the read, measure and management - of a set of real-time updated data flows. - - Can be thought of as a "flow descriptor" or "flow frame" which - describes the high level properties of a set of data flows that can - be used seamlessly across process-memory boundaries. - - Each instance's sub-components normally includes: - - a msg oriented quote stream provided via an IPC transport - - history and real-time shm buffers which are both real-time - updated and backfilled. - - associated startup indexing information related to both buffer - real-time-append and historical prepend addresses. - - low level APIs to read and measure the updated data and manage - queuing properties. - - ''' - symbol: Symbol - first_quote: dict - _hist_shm_token: _Token - _rt_shm_token: _Token - - # private shm refs loaded dynamically from tokens - _hist_shm: ShmArray | None = None - _rt_shm: ShmArray | None = None - - stream: tractor.MsgStream | None = None - izero_hist: int = 0 - izero_rt: int = 0 - throttle_rate: int | None = None - - # TODO: do we need this really if we can pull the `Portal` from - # ``tractor``'s internals? - feed: Feed | None = None - - @property - def rt_shm(self) -> ShmArray: - - if self._rt_shm is None: - self._rt_shm = attach_shm_array( - token=self._rt_shm_token, - readonly=True, - ) - - return self._rt_shm - - @property - def hist_shm(self) -> ShmArray: - - if self._hist_shm is None: - self._hist_shm = attach_shm_array( - token=self._hist_shm_token, - readonly=True, - ) - - return self._hist_shm - - async def receive(self) -> dict: - return await self.stream.receive() - - @acm - async def index_stream( - self, - delay_s: int = 1, - - ) -> AsyncIterator[int]: - - if not self.feed: - raise RuntimeError('This flume is not part of any ``Feed``?') - - # TODO: maybe a public (property) API for this in ``tractor``? - portal = self.stream._ctx._portal - assert portal - - # XXX: this should be singleton on a host, - # a lone broker-daemon per provider should be - # created for all practical purposes - async with maybe_open_context( - acm_func=partial( - portal.open_context, - iter_ohlc_periods, - ), - kwargs={'delay_s': delay_s}, - ) as (cache_hit, (ctx, first)): - async with ctx.open_stream() as istream: - if cache_hit: - # add a new broadcast subscription for the quote stream - # if this feed is likely already in use - async with istream.subscribe() as bistream: - yield bistream - else: - yield istream - - def get_ds_info( - self, - ) -> tuple[float, float, float]: - ''' - Compute the "downsampling" ratio info between the historical shm - buffer and the real-time (HFT) one. - - Return a tuple of the fast sample period, historical sample - period and ratio between them. - - ''' - times = self.hist_shm.array['time'] - end = pendulum.from_timestamp(times[-1]) - start = pendulum.from_timestamp(times[times != times[-1]][-1]) - hist_step_size_s = (end - start).seconds - - times = self.rt_shm.array['time'] - end = pendulum.from_timestamp(times[-1]) - start = pendulum.from_timestamp(times[times != times[-1]][-1]) - rt_step_size_s = (end - start).seconds - - ratio = hist_step_size_s / rt_step_size_s - return ( - rt_step_size_s, - hist_step_size_s, - ratio, - ) - - # TODO: get native msgspec decoding for these workinn - def to_msg(self) -> dict: - msg = self.to_dict() - msg['symbol'] = msg['symbol'].to_dict() - - # can't serialize the stream or feed objects, it's expected - # you'll have a ref to it since this msg should be rxed on - # a stream on whatever far end IPC.. - msg.pop('stream') - msg.pop('feed') - return msg - - @classmethod - def from_msg(cls, msg: dict) -> dict: - symbol = Symbol(**msg.pop('symbol')) - return cls( - symbol=symbol, - **msg, - ) - - def get_index( - self, - time_s: float, - - ) -> int: - ''' - Return array shm-buffer index for for epoch time. - - ''' - array = self.rt_shm.array - times = array['time'] - mask = (times >= time_s) - - if any(mask): - return array['index'][mask][0] - - # just the latest index - array['index'][-1] - - def slice_from_time( - self, - array: np.ndarray, - start_t: float, - stop_t: float, - timeframe_s: int = 1, - return_data: bool = False, - - ) -> np.ndarray: - ''' - Slice an input struct array providing only datums - "in view" of this chart. - - ''' - arr = { - 1: self.rt_shm.array, - 60: self.hist_shm.arry, - }[timeframe_s] - - times = arr['time'] - index = array['index'] - - # use advanced indexing to map the - # time range to the index range. - mask = ( - (times >= start_t) - & - (times < stop_t) - ) - - # TODO: if we can ensure each time field has a uniform - # step we can instead do some arithmetic to determine - # the equivalent index like we used to? - # return array[ - # lbar - ifirst: - # (rbar - ifirst) + 1 - # ] - - i_by_t = index[mask] - i_0 = i_by_t[0] - - abs_slc = slice( - i_0, - i_by_t[-1], - ) - # slice data by offset from the first index - # available in the passed datum set. - read_slc = slice( - 0, - i_by_t[-1] - i_0, - ) - if not return_data: - return ( - abs_slc, - read_slc, - ) - - # also return the readable data from the timerange - return ( - abs_slc, - read_slc, - arr[mask], - ) - - def view_data( - self, - plot: PlotItem, - timeframe_s: int = 1, - - ) -> np.ndarray: - - # get far-side x-indices plot view - vr = plot.viewRect() - l = vr.left() - r = vr.right() - - ( - abs_slc, - buf_slc, - iv_arr, - ) = self.slice_from_time( - start_t=l, - stop_t=r, - timeframe_s=timeframe_s, - return_data=True, - ) - return iv_arr - - async def allocate_persistent_feed( bus: _FeedsBus, sub_registered: trio.Event, From 7ec88f8cac611446c137f218c3e7d1373ed9028b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 24 Nov 2022 13:29:37 -0500 Subject: [PATCH 11/54] Make hist shm token optional to allow for FSPs --- piker/data/_flumes.py | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/piker/data/_flumes.py b/piker/data/_flumes.py index 82ce2947..bdb7c786 100644 --- a/piker/data/_flumes.py +++ b/piker/data/_flumes.py @@ -74,9 +74,12 @@ class Flume(Struct): ''' symbol: Symbol first_quote: dict - _hist_shm_token: _Token _rt_shm_token: _Token + # optional since some data flows won't have a "downsampled" history + # buffer/stream (eg. FSPs). + _hist_shm_token: _Token | None = None + # private shm refs loaded dynamically from tokens _hist_shm: ShmArray | None = None _rt_shm: ShmArray | None = None @@ -104,7 +107,14 @@ class Flume(Struct): @property def hist_shm(self) -> ShmArray: - if self._hist_shm is None: + if self._hist_shm_token is None: + raise RuntimeError( + 'No shm token has been set for the history buffer?' + ) + + if ( + self._hist_shm is None + ): self._hist_shm = attach_shm_array( token=self._hist_shm_token, readonly=True, @@ -288,16 +298,14 @@ class Flume(Struct): # get far-side x-indices plot view vr = plot.viewRect() - l = vr.left() - r = vr.right() ( abs_slc, buf_slc, iv_arr, ) = self.slice_from_time( - start_t=l, - stop_t=r, + start_t=vr.left(), + stop_t=vr.right(), timeframe_s=timeframe_s, return_data=True, ) From 2399c618b6dde3fbddc596610a2a3fd287966691 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 24 Nov 2022 15:05:06 -0500 Subject: [PATCH 12/54] Expand sampler loop shm write lines --- piker/data/_sampling.py | 43 ++++++++++++++++++++++++++++++----------- 1 file changed, 32 insertions(+), 11 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index 143f1c19..f3c92d56 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -128,22 +128,43 @@ async def increment_ohlc_buffer( # TODO: ``numba`` this! for shm in shms: + # append new entry to buffer thus "incrementing" the bar + array = shm.array + last = array[-1:][shm._write_fields].copy() + + (t, close) = last[0][[ + 'time', + 'close', + ]] + + # this copies non-std fields (eg. vwap) from the last datum + last[[ + 'time', + + 'open', + 'high', + 'low', + 'close', + + 'volume', + ]][0] = ( + # epoch timestamp + t + this_delay_s, + + # OHLC + close, + close, + close, + close, + + 0, # vlm + ) + # TODO: in theory we could make this faster by copying the # "last" readable value into the underlying larger buffer's # next value and then incrementing the counter instead of # using ``.push()``? - # append new entry to buffer thus "incrementing" the bar - array = shm.array - last = array[-1:][shm._write_fields].copy() - # (index, t, close) = last[0][['index', 'time', 'close']] - (t, close) = last[0][['time', 'close']] - - # this copies non-std fields (eg. vwap) from the last datum - last[ - ['time', 'volume', 'open', 'high', 'low', 'close'] - ][0] = (t + this_delay_s, 0, close, close, close, close) - # write to the buffer shm.push(last) From c85e7790de6c7ae583c557a957a8eae00c365715 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 24 Nov 2022 15:47:11 -0500 Subject: [PATCH 13/54] Rename `._flumes.py` -> `.flows.py` --- piker/data/feed.py | 2 +- piker/data/{_flumes.py => flows.py} | 0 2 files changed, 1 insertion(+), 1 deletion(-) rename piker/data/{_flumes.py => flows.py} (100%) diff --git a/piker/data/feed.py b/piker/data/feed.py index 59a7b303..302b324c 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -56,7 +56,7 @@ from .._daemon import ( maybe_spawn_brokerd, check_for_service, ) -from ._flumes import Flume +from .flows import Flume from ._sharedmem import ( maybe_open_shm_array, ShmArray, diff --git a/piker/data/_flumes.py b/piker/data/flows.py similarity index 100% rename from piker/data/_flumes.py rename to piker/data/flows.py From 8e1ceca43d23dc3bbdac39d47e5af3cdbed06f77 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 24 Nov 2022 16:15:35 -0500 Subject: [PATCH 14/54] Add some data-flows jargon notes (re: #270) --- piker/data/flows.py | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/piker/data/flows.py b/piker/data/flows.py index bdb7c786..23fa4207 100644 --- a/piker/data/flows.py +++ b/piker/data/flows.py @@ -15,9 +15,10 @@ # along with this program. If not, see . """ -Real-time data flow abstractions. +abstractions for organizing, managing and generally operating-on +real-time data processing data-structures. - +"Streams, flumes, cascades and flows.." """ from contextlib import asynccontextmanager as acm @@ -52,6 +53,26 @@ if TYPE_CHECKING: from .feed import Feed +# TODO: ideas for further abstractions as per +# https://github.com/pikers/piker/issues/216 and +# https://github.com/pikers/piker/issues/270: +# - a ``Cascade`` would be the minimal "connection" of 2 ``Flumes`` +# as per circuit parlance: +# https://en.wikipedia.org/wiki/Two-port_network#Cascade_connection +# - could cover the combination of our `FspAdmin` and the +# backend `.fsp._engine` related machinery to "connect" one flume +# to another? +# - a (financial signal) ``Flow`` would be the a "collection" of such +# minmial cascades. Some engineering based jargon concepts: +# - https://en.wikipedia.org/wiki/Signal_chain +# - https://en.wikipedia.org/wiki/Daisy_chain_(electrical_engineering) +# - https://en.wikipedia.org/wiki/Audio_signal_flow +# - https://en.wikipedia.org/wiki/Digital_signal_processing#Implementation +# - https://en.wikipedia.org/wiki/Dataflow_programming +# - https://en.wikipedia.org/wiki/Signal_programming +# - https://en.wikipedia.org/wiki/Incremental_computing + + class Flume(Struct): ''' Composite reference type which points to all the addressing handles From a5bb33b0fff5f84654aac8775ea868e1c121cf9c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 25 Nov 2022 16:40:58 -0500 Subject: [PATCH 15/54] Avoid key error on already popped cancel --- piker/clearing/_ems.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/clearing/_ems.py b/piker/clearing/_ems.py index ad512c08..06ae3b70 100644 --- a/piker/clearing/_ems.py +++ b/piker/clearing/_ems.py @@ -866,7 +866,7 @@ async def translate_and_relay_brokerd_events( elif status == 'canceled': log.cancel(f'Cancellation for {oid} is complete!') - status_msg = book._active.pop(oid) + status_msg = book._active.pop(oid, None) else: # open # relayed from backend but probably not handled so From d1b07c625fc5ac474964e25eb988b4f829280cee Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 26 Nov 2022 15:08:36 -0500 Subject: [PATCH 16/54] Copy timestamps from source to FSP dest buffer Slice up to history's length worth of (latest) time stamps from source series read at the start of the history init phase. --- piker/fsp/_engine.py | 25 ++++++++++++++++--------- piker/fsp/_volume.py | 3 +-- 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index edef7219..357934d8 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -112,8 +112,9 @@ async def fsp_compute( flume.rt_shm, ) - # Conduct a single iteration of fsp with historical bars input - # and get historical output + # HISTORY COMPUTE PHASE + # conduct a single iteration of fsp with historical bars input + # and get historical output, pack into history_output: Union[ dict[str, np.ndarray], # multi-output case np.ndarray, # single output case @@ -130,7 +131,8 @@ async def fsp_compute( # each respective field. fields = getattr(dst.array.dtype, 'fields', None).copy() fields.pop('index') - history: Optional[np.ndarray] = None # TODO: nptyping here! + history_by_field: Optional[np.ndarray] = None + src_time = src.array['time'] if ( fields and @@ -146,7 +148,7 @@ async def fsp_compute( if key in history_output: output = history_output[key] - if history is None: + if history_by_field is None: if output is None: length = len(src.array) @@ -156,7 +158,7 @@ async def fsp_compute( # using the first output, determine # the length of the struct-array that # will be pushed to shm. - history = np.zeros( + history_by_field = np.zeros( length, dtype=dst.array.dtype ) @@ -164,7 +166,7 @@ async def fsp_compute( if output is None: continue - history[key] = output + history_by_field[key] = output # single-key output stream else: @@ -173,11 +175,13 @@ async def fsp_compute( f'`{func_name}` is a single output FSP and should yield an ' '`np.ndarray` for history' ) - history = np.zeros( + history_by_field = np.zeros( len(history_output), dtype=dst.array.dtype ) - history[func_name] = history_output + history_by_field[func_name] = history_output + + history_by_field['time'] = src_time[-len(history_by_field):] # TODO: XXX: # THERE'S A BIG BUG HERE WITH THE `index` field since we're @@ -194,7 +198,10 @@ async def fsp_compute( # TODO: can we use this `start` flag instead of the manual # setting above? - index = dst.push(history, start=first) + index = dst.push( + history_by_field, + start=first, + ) profiler(f'{func_name} pushed history') profiler.finish() diff --git a/piker/fsp/_volume.py b/piker/fsp/_volume.py index b5456fac..b998c67b 100644 --- a/piker/fsp/_volume.py +++ b/piker/fsp/_volume.py @@ -268,8 +268,7 @@ async def flow_rates( 'dark_dvlm_rate': None, } - # TODO: 3.10 do ``anext()`` - quote = await source.__anext__() + quote = await anext(source) # ltr = 0 # lvr = 0 From 04c0d775953d5546199c4401de2fdb5cbb6ddca9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 29 Nov 2022 13:36:33 -0500 Subject: [PATCH 17/54] Frame ticks in helper routine Wow, turns out tick framing was totally borked since we weren't framing on "greater then throttle period long waits" XD This moves all the framing logic into a common func and calls it in every case: - every (normal) "pre throttle period expires" quote receive - each "no new quote before throttle period expires" (slow case) - each "no clearing tick yet received" / only burst on clears case --- piker/data/_sampling.py | 115 +++++++++++++++++++++++----------------- 1 file changed, 65 insertions(+), 50 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index f3c92d56..c4b182ae 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -431,12 +431,58 @@ async def sample_and_broadcast( # a working tick-type-classes template _tick_groups = { - 'clears': {'trade', 'utrade', 'last'}, + 'clears': {'trade', 'dark_trade', 'last'}, 'bids': {'bid', 'bsize'}, 'asks': {'ask', 'asize'}, } +def frame_ticks( + first_quote: dict, + last_quote: dict, + ticks_by_type: dict, +) -> None: + # append quotes since last iteration into the last quote's + # tick array/buffer. + ticks = last_quote.get('ticks') + + # TODO: once we decide to get fancy really we should + # have a shared mem tick buffer that is just + # continually filled and the UI just ready from it + # at it's display rate. + if ticks: + # TODO: do we need this any more or can we just + # expect the receiver to unwind the below + # `ticks_by_type: dict`? + # => undwinding would potentially require a + # `dict[str, set | list]` instead with an + # included `'types' field which is an (ordered) + # set of tick type fields in the order which + # types arrived? + first_quote['ticks'].extend(ticks) + + # XXX: build a tick-by-type table of lists + # of tick messages. This allows for less + # iteration on the receiver side by allowing for + # a single "latest tick event" look up by + # indexing the last entry in each sub-list. + # tbt = { + # 'types': ['bid', 'asize', 'last', .. ''], + + # 'bid': [tick0, tick1, tick2, .., tickn], + # 'asize': [tick0, tick1, tick2, .., tickn], + # 'last': [tick0, tick1, tick2, .., tickn], + # ... + # '': [tick0, tick1, tick2, .., tickn], + # } + + # append in reverse FIFO order for in-order iteration on + # receiver side. + for tick in ticks: + ttype = tick['type'] + ticks_by_type[ttype].append(tick) + + # TODO: a less naive throttler, here's some snippets: # token bucket by njs: # https://gist.github.com/njsmith/7ea44ec07e901cb78ebe1dd8dd846cb9 @@ -483,51 +529,17 @@ async def uniform_rate_send( if not first_quote: first_quote = last_quote + # first_quote['tbt'] = ticks_by_type if (throttle_period - diff) > 0: # received a quote but the send cycle period hasn't yet # expired we aren't supposed to send yet so append # to the tick frame. - - # append quotes since last iteration into the last quote's - # tick array/buffer. - ticks = last_quote.get('ticks') - - # TODO: once we decide to get fancy really we should - # have a shared mem tick buffer that is just - # continually filled and the UI just ready from it - # at it's display rate. - if ticks: - # TODO: do we need this any more or can we just - # expect the receiver to unwind the below - # `ticks_by_type: dict`? - # => undwinding would potentially require a - # `dict[str, set | list]` instead with an - # included `'types' field which is an (ordered) - # set of tick type fields in the order which - # types arrived? - first_quote['ticks'].extend(ticks) - - # XXX: build a tick-by-type table of lists - # of tick messages. This allows for less - # iteration on the receiver side by allowing for - # a single "latest tick event" look up by - # indexing the last entry in each sub-list. - # tbt = { - # 'types': ['bid', 'asize', 'last', .. ''], - - # 'bid': [tick0, tick1, tick2, .., tickn], - # 'asize': [tick0, tick1, tick2, .., tickn], - # 'last': [tick0, tick1, tick2, .., tickn], - # ... - # '': [tick0, tick1, tick2, .., tickn], - # } - for tick in ticks: - # append in reverse FIFO order for in-order - # iteration on receiver side. - ticks_by_type[tick['type']].append(tick) - - first_quote['tbt'] = ticks_by_type + frame_ticks( + first_quote, + last_quote, + ticks_by_type, + ) # send cycle isn't due yet so continue waiting continue @@ -544,6 +556,12 @@ async def uniform_rate_send( # received quote ASAP. sym, first_quote = await quote_stream.receive() + frame_ticks( + first_quote, + first_quote, + ticks_by_type, + ) + # we have a quote already so send it now. with trio.move_on_after(throttle_period) as cs: @@ -556,20 +574,17 @@ async def uniform_rate_send( log.exception(f"feed for {stream} ended?") break - ticks = last_quote.get('ticks') - first_quote['ticks'].extend(ticks) - if ticks: - for tick in ticks: - # append in reverse FIFO order for in-order - # iteration on receiver side. - ticks_by_type[tick['type']].append(tick) - - first_quote['tbt'] = ticks_by_type + frame_ticks( + first_quote, + last_quote, + ticks_by_type, + ) # measured_rate = 1 / (time.time() - last_send) # log.info( # f'`{sym}` throttled send hz: {round(measured_rate, ndigits=1)}' # ) + first_quote['tbt'] = ticks_by_type # TODO: now if only we could sync this to the display # rate timing exactly lul From 89095d4e9ff21643824ec22a3420adcfdde8357a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 8 Dec 2022 15:37:59 -0500 Subject: [PATCH 18/54] Ensure FSPs last 2 times are synced with its source --- piker/fsp/_engine.py | 37 +++++++++++++++++++++++-------------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index 357934d8..93bf0388 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -114,7 +114,7 @@ async def fsp_compute( # HISTORY COMPUTE PHASE # conduct a single iteration of fsp with historical bars input - # and get historical output, pack into + # and get historical output. history_output: Union[ dict[str, np.ndarray], # multi-output case np.ndarray, # single output case @@ -415,12 +415,12 @@ async def cascade( return not ( # the source is likely backfilling and we must # sync history calculations - len_diff > 2 or + len_diff > 2 # we aren't step synced to the source and may be # leading/lagging by a step - step_diff > 1 or - step_diff < 0 + or step_diff > 1 + or step_diff < 0 ), step_diff, len_diff async def poll_and_sync_to_step( @@ -485,15 +485,24 @@ async def cascade( else: last = array[-1:].copy() - # sync with source time step - src_t = src.array['time'][-1] - last['time'] = src_t - dst.push(last) - # dst_t = dst.array['time'][-1] - # print( - # f'{dst.token}\n' - # f'src: {src_t}\n' - # f'dst: {dst_t}\n' - # ) + # sync with source buffer's time step + src_l2 = src.array[-2:] + src_li, src_lt = src_l2[-1][['index', 'time']] + src_2li, src_2lt = src_l2[-2][['index', 'time']] + dst._array['time'][src_li] = src_lt + dst._array['time'][src_2li] = src_2lt + + # last2 = dst.array[-2:] + # if ( + # last2[-1]['index'] != src_li + # or last2[-2]['index'] != src_2li + # ): + # dstl2 = list(last2) + # srcl2 = list(src_l2) + # print( + # # f'{dst.token}\n' + # f'src: {srcl2}\n' + # f'dst: {dstl2}\n' + # ) From 4cdd2271b0ed29148934848308f0c35a373c40f9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 14 Dec 2022 12:06:23 -0500 Subject: [PATCH 19/54] Drop `tractor` assert bug note --- piker/data/feed.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index 302b324c..c2c46bcc 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -1122,10 +1122,6 @@ async def open_feed_bus( servicename = tractor.current_actor().name assert 'brokerd' in servicename - # XXX: figure this not crashing into debug! - # await tractor.breakpoint() - # assert 0 - assert brokername in servicename bus = get_feed_bus(brokername) From 1c5141f4c62c3cd2e18553c3faae96a0d994b22a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 18 Dec 2022 14:38:28 -0500 Subject: [PATCH 20/54] Fix f-str in duplicate frame msg print --- piker/data/feed.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index c2c46bcc..aa2a6bad 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -405,7 +405,7 @@ async def start_backfill( if next_start_dt in starts: start_dt = min(starts) - print("SKIPPING DUPLICATE FRAME @ {next_start_dt}") + print(f"SKIPPING DUPLICATE FRAME @ {next_start_dt}") continue # only update new start point if not-yet-seen From b0a6dd46e44c57b16a73aedc76e249c12a226d82 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 20 Dec 2022 14:44:06 -0500 Subject: [PATCH 21/54] Use recon set on stack closing during reconnect Hopefully resolves https://github.com/pikers/piker/issues/434 --- piker/data/_web_bs.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/data/_web_bs.py b/piker/data/_web_bs.py index 1577a678..2dd7f4af 100644 --- a/piker/data/_web_bs.py +++ b/piker/data/_web_bs.py @@ -92,7 +92,7 @@ class NoBsWs: while True: try: await self._stack.aclose() - except (DisconnectionTimeout, RuntimeError): + except self.recon_errors: await trio.sleep(0.5) else: break From daf7b3f4a5b6a88e905ab59f483b7478e7511817 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Dec 2022 17:30:08 -0500 Subject: [PATCH 22/54] Only accept 6 tries for the same duplicate hist frame When we see multiple history frames that are duplicate to the request set, bail re-trying after a number of tries (6 just cuz) and return early from the tsdb backfill loop; presume that this many duplicates means we've hit the beginning of history. Use a `collections.Counter` for the duplicate counts. Make sure and warn log in such cases. --- piker/data/feed.py | 38 ++++++++++++++++++++++++++++---------- 1 file changed, 28 insertions(+), 10 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index aa2a6bad..744d301f 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -21,7 +21,10 @@ This module is enabled for ``brokerd`` daemons. """ from __future__ import annotations -from collections import defaultdict +from collections import ( + defaultdict, + Counter, +) from contextlib import asynccontextmanager as acm from datetime import datetime from functools import partial @@ -374,8 +377,9 @@ async def start_backfill( # erlangs = config.get('erlangs', 1) # avoid duplicate history frames with a set of datetime frame - # starts. - starts: set[datetime] = set() + # starts and associated counts of how many duplicates we see + # per time stamp. + starts: Counter[datetime] = Counter() # inline sequential loop where we simply pass the # last retrieved start dt to the next request as @@ -403,14 +407,24 @@ async def start_backfill( # request loop until the condition is resolved? return - if next_start_dt in starts: + if ( + next_start_dt in starts + and starts[next_start_dt] <= 6 + ): start_dt = min(starts) print(f"SKIPPING DUPLICATE FRAME @ {next_start_dt}") + starts[start_dt] += 1 continue + elif starts[next_start_dt] > 6: + log.warning( + f'NO-MORE-DATA: backend {mod.name} before {next_start_dt}?' + ) + return + # only update new start point if not-yet-seen start_dt = next_start_dt - starts.add(start_dt) + starts[start_dt] += 1 assert array['time'][0] == start_dt.timestamp() @@ -656,10 +670,10 @@ async def tsdb_backfill( # Load TSDB history into shm buffer (for display) if there is # remaining buffer space. + if ( len(tsdb_history) ): - # load the first (smaller) bit of history originally loaded # above from ``Storage.load()``. to_push = tsdb_history[-prepend_start:] @@ -682,14 +696,12 @@ async def tsdb_backfill( # load as much from storage into shm possible (depends on # user's shm size settings). - while ( - shm._first.value > 0 - ): + while shm._first.value > 0: tsdb_history = await storage.read_ohlcv( fqsn, - end=tsdb_last_frame_start, timeframe=timeframe, + end=tsdb_last_frame_start, ) # empty query @@ -930,6 +942,8 @@ async def allocate_persistent_feed( some_data_ready = trio.Event() feed_is_live = trio.Event() + symstr = symstr.lower() + # establish broker backend quote stream by calling # ``stream_quotes()``, which is a required broker backend endpoint. init_msg, first_quote = await bus.nursery.start( @@ -1130,6 +1144,10 @@ async def open_feed_bus( flumes: dict[str, Flume] = {} for symbol in symbols: + + # we always use lower case keys internally + symbol = symbol.lower() + # if no cached feed for this symbol has been created for this # brokerd yet, start persistent stream and shm writer task in # service nursery From 009bbe456e122338169b9b5d1c70cc50244a96b6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 21 Dec 2022 17:38:14 -0500 Subject: [PATCH 23/54] Always `.error()` log unknown queries for `marketstore` --- piker/data/marketstore.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/piker/data/marketstore.py b/piker/data/marketstore.py index d354f9b0..88553af7 100644 --- a/piker/data/marketstore.py +++ b/piker/data/marketstore.py @@ -454,8 +454,12 @@ class Storage: try: result = await client.query(params) - except purerpc.grpclib.exceptions.UnknownError: + except purerpc.grpclib.exceptions.UnknownError as err: # indicate there is no history for this timeframe + log.exception( + f'Unknown mkts QUERY error: {params}\n' + f'{err.args}' + ) return {} # TODO: it turns out column access on recarrays is actually slower: From 3efb0b58842291e18a10fdd6597b4702782a1d9d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 22 Dec 2022 13:20:55 -0500 Subject: [PATCH 24/54] Sync 1s (or less) sampler steps using rounded now-epoch --- piker/data/_sampling.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index c4b182ae..faa17da9 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -137,6 +137,14 @@ async def increment_ohlc_buffer( 'close', ]] + next_t = t + this_delay_s + i_epoch = round(time.time()) + + if this_delay_s <= 1: + next_t = i_epoch + + # print(f'epoch {shm.token["shm_name"]}: {next_t}') + # this copies non-std fields (eg. vwap) from the last datum last[[ 'time', @@ -149,7 +157,7 @@ async def increment_ohlc_buffer( 'volume', ]][0] = ( # epoch timestamp - t + this_delay_s, + next_t, # OHLC close, From b5f2ff854c1da47f1e5f54c5015bcbb6fad43658 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 2 Jan 2023 15:32:48 -0500 Subject: [PATCH 25/54] Drop meaning the clearing rate, use per step count --- piker/fsp/_volume.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/fsp/_volume.py b/piker/fsp/_volume.py index b998c67b..06d0be91 100644 --- a/piker/fsp/_volume.py +++ b/piker/fsp/_volume.py @@ -234,7 +234,7 @@ async def flow_rates( # FSPs, user input, and possibly any general event stream in # real-time. Hint: ideally implemented with caching until mutated # ;) - period: 'Param[int]' = 6, # noqa + period: 'Param[int]' = 1, # noqa # TODO: support other means by providing a map # to weights `partial()`-ed with `wma()`? From 2c76cee9282592fab1067ddc01b4a82d12d2f097 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 3 Jan 2023 11:54:18 -0500 Subject: [PATCH 26/54] Begin formalizing `Sampler` singleton API We're moving toward a single actor managing sampler work and distributed independently of `brokerd` services such that a user can run samplers on different hosts then real-time data feed infra. Most of the implementation details include aggregating `.data._sampling` routines into a new `Sampler` singleton type. Move the following methods to class methods: - `.increment_ohlc_buffer()` to allow a single task to increment all registered shm buffers. - `.broadcast()` for IPC relay to all registered clients/shms. Further add a new `maybe_open_global_sampler()` which allocates a service nursery and assigns it to the `Sampler.service_nursery`; this is prep for putting the step incrementer in a singleton service task higher up the data-layer actor tree. --- piker/data/_sampling.py | 296 ++++++++++++++++++++++------------------ piker/data/feed.py | 21 ++- 2 files changed, 174 insertions(+), 143 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index faa17da9..622a8ad1 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -33,7 +33,10 @@ import tractor import trio from trio_typing import TaskStatus -from ..log import get_logger +from ..log import ( + get_logger, + get_console_log, +) if TYPE_CHECKING: from ._sharedmem import ShmArray @@ -45,7 +48,7 @@ log = get_logger(__name__) _default_delay_s: float = 1.0 -class sampler: +class Sampler: ''' Global sampling engine registry. @@ -53,6 +56,8 @@ class sampler: sample period logic. ''' + service_nursery: None | trio.Nursery = None + # TODO: we could stick these in a composed type to avoid # angering the "i hate module scoped variables crowd" (yawn). ohlcv_shms: dict[int, list[ShmArray]] = {} @@ -67,165 +72,196 @@ class sampler: # notified on a step. subscribers: dict[int, tractor.Context] = {} + @classmethod + async def increment_ohlc_buffer( + self, + delay_s: int, + task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED, + ): + ''' + Task which inserts new bars into the provide shared memory array + every ``delay_s`` seconds. -async def increment_ohlc_buffer( - delay_s: int, - task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED, -): - ''' - Task which inserts new bars into the provide shared memory array - every ``delay_s`` seconds. + This task fulfills 2 purposes: + - it takes the subscribed set of shm arrays and increments them + on a common time period + - broadcast of this increment "signal" message to other actor + subscribers - This task fulfills 2 purposes: - - it takes the subscribed set of shm arrays and increments them - on a common time period - - broadcast of this increment "signal" message to other actor - subscribers + Note that if **no** actor has initiated this task then **none** of + the underlying buffers will actually be incremented. - Note that if **no** actor has initiated this task then **none** of - the underlying buffers will actually be incremented. + ''' + # # wait for brokerd to signal we should start sampling + # await shm_incrementing(shm_token['shm_name']).wait() - ''' - # # wait for brokerd to signal we should start sampling - # await shm_incrementing(shm_token['shm_name']).wait() + # TODO: right now we'll spin printing bars if the last time stamp is + # before a large period of no market activity. Likely the best way + # to solve this is to make this task aware of the instrument's + # tradable hours? - # TODO: right now we'll spin printing bars if the last time stamp is - # before a large period of no market activity. Likely the best way - # to solve this is to make this task aware of the instrument's - # tradable hours? + # adjust delay to compensate for trio processing time + ad = min(self.ohlcv_shms.keys()) - 0.001 - # adjust delay to compensate for trio processing time - ad = min(sampler.ohlcv_shms.keys()) - 0.001 + total_s = 0 # total seconds counted + lowest = min(self.ohlcv_shms.keys()) + lowest_shm = self.ohlcv_shms[lowest][0] + ad = lowest - 0.001 - total_s = 0 # total seconds counted - lowest = min(sampler.ohlcv_shms.keys()) - lowest_shm = sampler.ohlcv_shms[lowest][0] - ad = lowest - 0.001 + with trio.CancelScope() as cs: - with trio.CancelScope() as cs: + # register this time period step as active + self.incrementers[delay_s] = cs + task_status.started(cs) - # register this time period step as active - sampler.incrementers[delay_s] = cs - task_status.started(cs) + while True: + # TODO: do we want to support dynamically + # adding a "lower" lowest increment period? + await trio.sleep(ad) + total_s += delay_s - while True: - # TODO: do we want to support dynamically - # adding a "lower" lowest increment period? - await trio.sleep(ad) - total_s += delay_s + # increment all subscribed shm arrays + # TODO: + # - this in ``numba`` + # - just lookup shms for this step instead of iterating? + for this_delay_s, shms in self.ohlcv_shms.items(): - # increment all subscribed shm arrays - # TODO: - # - this in ``numba`` - # - just lookup shms for this step instead of iterating? - for this_delay_s, shms in sampler.ohlcv_shms.items(): + # short-circuit on any not-ready because slower sample + # rate consuming shm buffers. + if total_s % this_delay_s != 0: + # print(f'skipping `{this_delay_s}s` sample update') + continue - # short-circuit on any not-ready because slower sample - # rate consuming shm buffers. - if total_s % this_delay_s != 0: - # print(f'skipping `{this_delay_s}s` sample update') - continue + # TODO: ``numba`` this! + for shm in shms: + # append new entry to buffer thus "incrementing" + # the bar + array = shm.array + last = array[-1:][shm._write_fields].copy() - # TODO: ``numba`` this! - for shm in shms: - # append new entry to buffer thus "incrementing" the bar - array = shm.array - last = array[-1:][shm._write_fields].copy() + (t, close) = last[0][[ + 'time', + 'close', + ]] - (t, close) = last[0][[ - 'time', - 'close', - ]] + next_t = t + this_delay_s + i_epoch = round(time.time()) - next_t = t + this_delay_s - i_epoch = round(time.time()) + if this_delay_s <= 1: + next_t = i_epoch - if this_delay_s <= 1: - next_t = i_epoch + # print(f'epoch {shm.token["shm_name"]}: {next_t}') - # print(f'epoch {shm.token["shm_name"]}: {next_t}') + # this copies non-std fields (eg. vwap) from the + # last datum + last[[ + 'time', - # this copies non-std fields (eg. vwap) from the last datum - last[[ - 'time', + 'open', + 'high', + 'low', + 'close', - 'open', - 'high', - 'low', - 'close', + 'volume', + ]][0] = ( + # epoch timestamp + next_t, - 'volume', - ]][0] = ( - # epoch timestamp - next_t, + # OHLC + close, + close, + close, + close, - # OHLC - close, - close, - close, - close, + 0, # vlm + ) - 0, # vlm + # TODO: in theory we could make this faster by + # copying the "last" readable value into the + # underlying larger buffer's next value and then + # incrementing the counter instead of using + # ``.push()``? + + # write to the buffer + shm.push(last) + + await self.broadcast(delay_s, shm=lowest_shm) + + @classmethod + async def broadcast( + self, + delay_s: int, + shm: ShmArray | None = None, + + ) -> None: + ''' + Broadcast the given ``shm: ShmArray``'s buffer index step to any + subscribers for a given sample period. + + The sent msg will include the first and last index which slice into + the buffer's non-empty data. + + ''' + subs = self.subscribers.get(delay_s, ()) + first = last = -1 + + if shm is None: + periods = self.ohlcv_shms.keys() + # if this is an update triggered by a history update there + # might not actually be any sampling bus setup since there's + # no "live feed" active yet. + if periods: + lowest = min(periods) + shm = self.ohlcv_shms[lowest][0] + first = shm._first.value + last = shm._last.value + + for stream in subs: + try: + await stream.send({ + 'first': first, + 'last': last, + 'index': last, + }) + except ( + trio.BrokenResourceError, + trio.ClosedResourceError + ): + log.error( + f'{stream._ctx.chan.uid} dropped connection' + ) + try: + subs.remove(stream) + except ValueError: + log.warning( + f'{stream._ctx.chan.uid} sub already removed!?' ) - # TODO: in theory we could make this faster by copying the - # "last" readable value into the underlying larger buffer's - # next value and then incrementing the counter instead of - # using ``.push()``? - - # write to the buffer - shm.push(last) - - await broadcast(delay_s, shm=lowest_shm) + @classmethod + async def broadcast_all(self) -> None: + for delay_s in self.subscribers: + await self.broadcast(delay_s) -async def broadcast( - delay_s: int, - shm: ShmArray | None = None, +@tractor.context +async def maybe_open_global_sampler( + ctx: tractor.Context, + brokername: str, ) -> None: - ''' - Broadcast the given ``shm: ShmArray``'s buffer index step to any - subscribers for a given sample period. + get_console_log(tractor.current_actor().loglevel) - The sent msg will include the first and last index which slice into - the buffer's non-empty data. + global Sampler - ''' - subs = sampler.subscribers.get(delay_s, ()) - first = last = -1 + async with trio.open_nursery() as service_nursery: + Sampler.service_nursery = service_nursery - if shm is None: - periods = sampler.ohlcv_shms.keys() - # if this is an update triggered by a history update there - # might not actually be any sampling bus setup since there's - # no "live feed" active yet. - if periods: - lowest = min(periods) - shm = sampler.ohlcv_shms[lowest][0] - first = shm._first.value - last = shm._last.value + # unblock caller + await ctx.started() - for stream in subs: - try: - await stream.send({ - 'first': first, - 'last': last, - 'index': last, - }) - except ( - trio.BrokenResourceError, - trio.ClosedResourceError - ): - log.error( - f'{stream._ctx.chan.uid} dropped connection' - ) - try: - subs.remove(stream) - except ValueError: - log.warning( - f'{stream._ctx.chan.uid} sub already removed!?' - ) + # we pin this task to keep the feeds manager active until the + # parent actor decides to tear it down + await trio.sleep_forever() @tractor.context @@ -241,7 +277,7 @@ async def iter_ohlc_periods( ''' # add our subscription - subs = sampler.subscribers.setdefault(delay_s, []) + subs = Sampler.subscribers.setdefault(delay_s, []) await ctx.started() async with ctx.open_stream() as stream: subs.append(stream) diff --git a/piker/data/feed.py b/piker/data/feed.py index 744d301f..89330475 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -74,9 +74,7 @@ from ._source import ( ) from ..ui import _search from ._sampling import ( - sampler, - broadcast, - increment_ohlc_buffer, + Sampler, sample_and_broadcast, uniform_rate_send, _default_delay_s, @@ -327,8 +325,7 @@ async def start_backfill( # TODO: *** THIS IS A BUG *** # we need to only broadcast to subscribers for this fqsn.. # otherwise all fsps get reset on every chart.. - for delay_s in sampler.subscribers: - await broadcast(delay_s) + await Sampler.broadcast_all() # signal that backfilling to tsdb's end datum is complete bf_done = trio.Event() @@ -496,8 +493,7 @@ async def start_backfill( # in the block above to avoid entering new ``frames`` # values while we're pipelining the current ones to # memory... - for delay_s in sampler.subscribers: - await broadcast(delay_s) + await Sampler.broadcast_all() # short-circuit (for now) bf_done.set() @@ -738,8 +734,7 @@ async def tsdb_backfill( # (usually a chart showing graphics for said fsp) # which tells the chart to conduct a manual full # graphics loop cycle. - for delay_s in sampler.subscribers: - await broadcast(delay_s) + await Sampler.broadcast_all() # TODO: write new data to tsdb to be ready to for next read. @@ -1037,7 +1032,7 @@ async def allocate_persistent_feed( # insert 1s ohlc into the increment buffer set # to update and shift every second - sampler.ohlcv_shms.setdefault( + Sampler.ohlcv_shms.setdefault( 1, [] ).append(rt_shm) @@ -1053,13 +1048,13 @@ async def allocate_persistent_feed( # insert 1m ohlc into the increment buffer set # to shift every 60s. - sampler.ohlcv_shms.setdefault(60, []).append(hist_shm) + Sampler.ohlcv_shms.setdefault(60, []).append(hist_shm) # create buffer a single incrementer task broker backend # (aka `brokerd`) using the lowest sampler period. - if sampler.incrementers.get(_default_delay_s) is None: + if Sampler.incrementers.get(_default_delay_s) is None: await bus.start_task( - increment_ohlc_buffer, + Sampler.increment_ohlc_buffer, _default_delay_s, ) From a342f7d2d4b4f30d7a186514b284a5f819b2384f Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 4 Jan 2023 22:01:28 -0500 Subject: [PATCH 27/54] Make `._daemon.Services` for use as singleton Drop the `_services` module level ref and adjust all client code to match. Drop struct inheritance and convert all methods to class level. Move `Brokerd.locks` -> `Services.locks` and add sampling mod to pikerd enabled set. --- piker/_daemon.py | 97 ++++++++++++++++++++++-------------------------- 1 file changed, 45 insertions(+), 52 deletions(-) diff --git a/piker/_daemon.py b/piker/_daemon.py index d4ca7f21..4e13e1ec 100644 --- a/piker/_daemon.py +++ b/piker/_daemon.py @@ -22,7 +22,6 @@ from typing import Optional, Union, Callable, Any from contextlib import asynccontextmanager as acm from collections import defaultdict -from msgspec import Struct import tractor import trio from trio_typing import TaskStatus @@ -54,16 +53,19 @@ _root_modules = [ __name__, 'piker.clearing._ems', 'piker.clearing._client', + 'piker.data._sampling', ] -class Services(Struct): +class Services: actor_n: tractor._supervise.ActorNursery service_n: trio.Nursery debug_mode: bool # tractor sub-actor debug mode flag service_tasks: dict[str, tuple[trio.CancelScope, tractor.Portal]] = {} + locks = defaultdict(trio.Lock) + @classmethod async def start_service_task( self, name: str, @@ -119,11 +121,11 @@ class Services(Struct): return cs, first - # TODO: per service cancellation by scope, we aren't using this - # anywhere right? + @classmethod async def cancel_service( self, name: str, + ) -> Any: log.info(f'Cancelling `pikerd` service {name}') cs, portal = self.service_tasks[name] @@ -134,29 +136,25 @@ class Services(Struct): return await portal.cancel_actor() -_services: Optional[Services] = None - - @acm async def open_pikerd( start_method: str = 'trio', - loglevel: Optional[str] = None, + loglevel: str | None = None, # XXX: you should pretty much never want debug mode # for data daemons when running in production. debug_mode: bool = False, registry_addr: None | tuple[str, int] = None, -) -> Optional[tractor._portal.Portal]: +) -> None: ''' - Start a root piker daemon who's lifetime extends indefinitely - until cancelled. + Start a root piker daemon who's lifetime extends indefinitely until + cancelled. A root actor nursery is created which can be used to create and keep alive underling services (see below). ''' - global _services global _registry_addr if ( @@ -186,17 +184,11 @@ async def open_pikerd( ): async with trio.open_nursery() as service_nursery: - # # setup service mngr singleton instance - # async with AsyncExitStack() as stack: - # assign globally for future daemon/task creation - _services = Services( - actor_n=actor_nursery, - service_n=service_nursery, - debug_mode=debug_mode, - ) - - yield _services + Services.actor_n = actor_nursery + Services.service_n = service_nursery + Services.debug_mode = debug_mode + yield @acm @@ -217,7 +209,6 @@ async def open_piker_runtime( existing piker actors on the local link based on configuration. ''' - global _services global _registry_addr if ( @@ -276,11 +267,12 @@ async def maybe_open_pikerd( **kwargs, ) -> Union[tractor._portal.Portal, Services]: - """If no ``pikerd`` daemon-root-actor can be found start it and + ''' + If no ``pikerd`` daemon-root-actor can be found start it and yield up (we should probably figure out returning a portal to self though). - """ + ''' if loglevel: get_console_log(loglevel) @@ -316,7 +308,9 @@ async def maybe_open_pikerd( yield None -# brokerd enabled modules +# `brokerd` enabled modules +# NOTE: keeping this list as small as possible is part of our caps-sec +# model and should be treated with utmost care! _data_mods = [ 'piker.brokers.core', 'piker.brokers.data', @@ -326,10 +320,6 @@ _data_mods = [ ] -class Brokerd: - locks = defaultdict(trio.Lock) - - @acm async def find_service( service_name: str, @@ -366,6 +356,8 @@ async def maybe_spawn_daemon( service_task_target: Callable, spawn_args: dict[str, Any], loglevel: Optional[str] = None, + + singleton: bool = False, **kwargs, ) -> tractor.Portal: @@ -386,7 +378,7 @@ async def maybe_spawn_daemon( # serialize access to this section to avoid # 2 or more tasks racing to create a daemon - lock = Brokerd.locks[service_name] + lock = Services.locks[service_name] await lock.acquire() async with find_service(service_name) as portal: @@ -397,6 +389,9 @@ async def maybe_spawn_daemon( log.warning(f"Couldn't find any existing {service_name}") + # TODO: really shouldn't the actor spawning be part of the service + # starting method `Services.start_service()` ? + # ask root ``pikerd`` daemon to spawn the daemon we need if # pikerd is not live we now become the root of the # process tree @@ -407,15 +402,16 @@ async def maybe_spawn_daemon( ) as pikerd_portal: + # we are the root and thus are `pikerd` + # so spawn the target service directly by calling + # the provided target routine. + # XXX: this assumes that the target is well formed and will + # do the right things to setup both a sub-actor **and** call + # the ``_Services`` api from above to start the top level + # service task for that actor. + started: bool if pikerd_portal is None: - # we are the root and thus are `pikerd` - # so spawn the target service directly by calling - # the provided target routine. - # XXX: this assumes that the target is well formed and will - # do the right things to setup both a sub-actor **and** call - # the ``_Services`` api from above to start the top level - # service task for that actor. - await service_task_target(**spawn_args) + started = await service_task_target(**spawn_args) else: # tell the remote `pikerd` to start the target, @@ -424,11 +420,14 @@ async def maybe_spawn_daemon( # non-blocking and the target task will persist running # on `pikerd` after the client requesting it's start # disconnects. - await pikerd_portal.run( + started = await pikerd_portal.run( service_task_target, **spawn_args, ) + if started: + log.info(f'Service {service_name} started!') + async with tractor.wait_for_actor(service_name) as portal: lock.release() yield portal @@ -451,9 +450,6 @@ async def spawn_brokerd( extra_tractor_kwargs = getattr(brokermod, '_spawn_kwargs', {}) tractor_kwargs.update(extra_tractor_kwargs) - global _services - assert _services - # ask `pikerd` to spawn a new sub-actor and manage it under its # actor nursery modpath = brokermod.__name__ @@ -466,18 +462,18 @@ async def spawn_brokerd( subpath = f'{modpath}.{submodname}' broker_enable.append(subpath) - portal = await _services.actor_n.start_actor( + portal = await Services.actor_n.start_actor( dname, enable_modules=_data_mods + broker_enable, loglevel=loglevel, - debug_mode=_services.debug_mode, + debug_mode=Services.debug_mode, **tractor_kwargs ) # non-blocking setup of brokerd service nursery from .data import _setup_persistent_brokerd - await _services.start_service_task( + await Services.start_service_task( dname, portal, _setup_persistent_brokerd, @@ -523,24 +519,21 @@ async def spawn_emsd( """ log.info('Spawning emsd') - global _services - assert _services - - portal = await _services.actor_n.start_actor( + portal = await Services.actor_n.start_actor( 'emsd', enable_modules=[ 'piker.clearing._ems', 'piker.clearing._client', ], loglevel=loglevel, - debug_mode=_services.debug_mode, # set by pikerd flag + debug_mode=Services.debug_mode, # set by pikerd flag **extra_tractor_kwargs ) # non-blocking setup of clearing service from .clearing._ems import _setup_persistent_emsd - await _services.start_service_task( + await Services.start_service_task( 'emsd', portal, _setup_persistent_emsd, From 5ec1a72a3da11af80f39ce9441d7de8704a00d23 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 4 Jan 2023 22:04:51 -0500 Subject: [PATCH 28/54] Implement a `samplerd` singleton actor service Now spawned under the `pikerd` tree as a singleton-daemon-actor we offer a slew of new routines in support of this micro-service: - `maybe_open_samplerd()` and `spawn_samplerd()` which provide the `._daemon.Services` integration to conduct service spawning. - `open_sample_stream()` which is a client-side endpoint which does all the work of (lazily) starting the `samplerd` service (if dne) and registers shm buffers for update as well as connect a sample-index stream for iterator by the caller. - `register_with_sampler()` which is the `samplerd`-side service task endpoint implementing all the shm buffer and index-stream registry details as well as logic to ensure a lone service task runs `Services.increment_ohlc_buffer()`; it increments at the shortest period registered which, for now, is the default 1s duration. Further impl notes: - fixes to `Services.broadcast()` to ensure broken streams get discarded gracefully. - we use a `pikerd` side singleton mutex `trio.Lock()` to ensure one-and-only-one `samplerd` is ever spawned per `pikerd` actor tree. --- piker/data/_sampling.py | 424 +++++++++++++++++++++++++++++----------- 1 file changed, 315 insertions(+), 109 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index 622a8ad1..f70e4113 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -24,12 +24,17 @@ from collections import ( Counter, defaultdict, ) +from contextlib import asynccontextmanager as acm import time from typing import ( + AsyncIterator, TYPE_CHECKING, ) import tractor +from tractor.trionics import ( + maybe_open_nursery, +) import trio from trio_typing import TaskStatus @@ -37,14 +42,20 @@ from ..log import ( get_logger, get_console_log, ) +from .._daemon import maybe_spawn_daemon if TYPE_CHECKING: - from ._sharedmem import ShmArray + from ._sharedmem import ( + ShmArray, + ) from .feed import _FeedsBus log = get_logger(__name__) +# highest frequency sample step is 1 second by default, though in +# the future we may want to support shorter periods or a dynamic style +# tick-event stream. _default_delay_s: float = 1.0 @@ -55,32 +66,50 @@ class Sampler: Manages state for sampling events, shm incrementing and sample period logic. + This non-instantiated type is meant to be a singleton within + a `samplerd` actor-service spawned once by the user wishing to + time-step sample real-time quote feeds, see + ``._daemon.maybe_open_samplerd()`` and the below + ``register_with_sampler()``. + ''' service_nursery: None | trio.Nursery = None # TODO: we could stick these in a composed type to avoid # angering the "i hate module scoped variables crowd" (yawn). - ohlcv_shms: dict[int, list[ShmArray]] = {} + ohlcv_shms: dict[float, list[ShmArray]] = {} # holds one-task-per-sample-period tasks which are spawned as-needed by # data feed requests with a given detected time step usually from # history loading. - incrementers: dict[int, trio.CancelScope] = {} + incr_task_cs: trio.CancelScope | None = None # holds all the ``tractor.Context`` remote subscriptions for # a particular sample period increment event: all subscribers are # notified on a step. - subscribers: dict[int, tractor.Context] = {} + # subscribers: dict[int, list[tractor.MsgStream]] = {} + subscribers: defaultdict[ + float, + list[ + float, + set[tractor.MsgStream] + ], + ] = defaultdict( + lambda: [ + round(time.time()), + set(), + ] + ) @classmethod async def increment_ohlc_buffer( self, - delay_s: int, + period_s: float, task_status: TaskStatus[trio.CancelScope] = trio.TASK_STATUS_IGNORED, ): ''' Task which inserts new bars into the provide shared memory array - every ``delay_s`` seconds. + every ``period_s`` seconds. This task fulfills 2 purposes: - it takes the subscribed set of shm arrays and increments them @@ -92,66 +121,74 @@ class Sampler: the underlying buffers will actually be incremented. ''' - # # wait for brokerd to signal we should start sampling - # await shm_incrementing(shm_token['shm_name']).wait() - # TODO: right now we'll spin printing bars if the last time stamp is # before a large period of no market activity. Likely the best way # to solve this is to make this task aware of the instrument's # tradable hours? - # adjust delay to compensate for trio processing time - ad = min(self.ohlcv_shms.keys()) - 0.001 - - total_s = 0 # total seconds counted - lowest = min(self.ohlcv_shms.keys()) - lowest_shm = self.ohlcv_shms[lowest][0] - ad = lowest - 0.001 + total_s: float = 0 # total seconds counted + ad = period_s - 0.001 # compensate for trio processing time with trio.CancelScope() as cs: - # register this time period step as active - self.incrementers[delay_s] = cs task_status.started(cs) + # sample step loop: + # includes broadcasting to all connected consumers on every + # new sample step as well incrementing any registered + # buffers by registered sample period. while True: - # TODO: do we want to support dynamically - # adding a "lower" lowest increment period? await trio.sleep(ad) - total_s += delay_s + total_s += period_s # increment all subscribed shm arrays # TODO: # - this in ``numba`` # - just lookup shms for this step instead of iterating? - for this_delay_s, shms in self.ohlcv_shms.items(): + + i_epoch = round(time.time()) + broadcasted: set[float] = set() + + # print(f'epoch: {i_epoch} -> REGISTRY {self.ohlcv_shms}') + for shm_period_s, shms in self.ohlcv_shms.items(): # short-circuit on any not-ready because slower sample # rate consuming shm buffers. - if total_s % this_delay_s != 0: - # print(f'skipping `{this_delay_s}s` sample update') + if total_s % shm_period_s != 0: + # print(f'skipping `{shm_period_s}s` sample update') continue + # update last epoch stamp for this period group + if shm_period_s not in broadcasted: + sub_pair = self.subscribers[shm_period_s] + sub_pair[0] = i_epoch + print(f'skipping `{shm_period_s}s` sample update') + broadcasted.add(shm_period_s) + # TODO: ``numba`` this! for shm in shms: + # print(f'UPDATE {shm_period_s}s STEP for {shm.token}') + # append new entry to buffer thus "incrementing" # the bar array = shm.array last = array[-1:][shm._write_fields].copy() + # guard against startup backfilling race with + # empty buffers. + if not last.size: + continue + (t, close) = last[0][[ 'time', 'close', ]] - next_t = t + this_delay_s - i_epoch = round(time.time()) + next_t = t + shm_period_s - if this_delay_s <= 1: + if shm_period_s <= 1: next_t = i_epoch - # print(f'epoch {shm.token["shm_name"]}: {next_t}') - # this copies non-std fields (eg. vwap) from the # last datum last[[ @@ -185,43 +222,43 @@ class Sampler: # write to the buffer shm.push(last) - await self.broadcast(delay_s, shm=lowest_shm) + # broadcast increment msg to all updated subs per period + for shm_period_s in broadcasted: + await self.broadcast( + period_s=shm_period_s, + time_stamp=i_epoch, + ) @classmethod async def broadcast( self, - delay_s: int, - shm: ShmArray | None = None, + period_s: float, + time_stamp: float | None = None, ) -> None: ''' - Broadcast the given ``shm: ShmArray``'s buffer index step to any + Broadcast the period size and last index step value to all subscribers for a given sample period. - The sent msg will include the first and last index which slice into - the buffer's non-empty data. - ''' - subs = self.subscribers.get(delay_s, ()) - first = last = -1 + pair = self.subscribers[period_s] - if shm is None: - periods = self.ohlcv_shms.keys() - # if this is an update triggered by a history update there - # might not actually be any sampling bus setup since there's - # no "live feed" active yet. - if periods: - lowest = min(periods) - shm = self.ohlcv_shms[lowest][0] - first = shm._first.value - last = shm._last.value + last_ts, subs = pair + task = trio.lowlevel.current_task() + log.debug( + f'SUBS {self.subscribers}\n' + f'PAIR {pair}\n' + f'TASK: {task}: {id(task)}\n' + f'broadcasting {period_s} -> {last_ts}\n' + # f'consumers: {subs}' + ) + borked: set[tractor.MsgStream] = set() for stream in subs: try: await stream.send({ - 'first': first, - 'last': last, - 'index': last, + 'index': time_stamp or last_ts, + 'period': period_s, }) except ( trio.BrokenResourceError, @@ -230,69 +267,232 @@ class Sampler: log.error( f'{stream._ctx.chan.uid} dropped connection' ) - try: - subs.remove(stream) - except ValueError: - log.warning( - f'{stream._ctx.chan.uid} sub already removed!?' - ) + borked.add(stream) - @classmethod - async def broadcast_all(self) -> None: - for delay_s in self.subscribers: - await self.broadcast(delay_s) - - -@tractor.context -async def maybe_open_global_sampler( - ctx: tractor.Context, - brokername: str, - -) -> None: - get_console_log(tractor.current_actor().loglevel) - - global Sampler - - async with trio.open_nursery() as service_nursery: - Sampler.service_nursery = service_nursery - - # unblock caller - await ctx.started() - - # we pin this task to keep the feeds manager active until the - # parent actor decides to tear it down - await trio.sleep_forever() - - -@tractor.context -async def iter_ohlc_periods( - ctx: tractor.Context, - delay_s: int, - -) -> None: - ''' - Subscribe to OHLC sampling "step" events: when the time - aggregation period increments, this event stream emits an index - event. - - ''' - # add our subscription - subs = Sampler.subscribers.setdefault(delay_s, []) - await ctx.started() - async with ctx.open_stream() as stream: - subs.append(stream) - - try: - # stream and block until cancelled - await trio.sleep_forever() - finally: + for stream in borked: try: subs.remove(stream) except ValueError: - log.error( - f'iOHLC step stream was already dropped {ctx.chan.uid}?' + log.warning( + f'{stream._ctx.chan.uid} sub already removed!?' ) + @classmethod + async def broadcast_all(self) -> None: + for period_s in self.subscribers: + await self.broadcast(period_s) + + +@tractor.context +async def register_with_sampler( + ctx: tractor.Context, + period_s: float, + shms_by_period: dict[float, dict] | None = None, + open_index_stream: bool = True, + +) -> None: + + get_console_log(tractor.current_actor().loglevel) + incr_was_started: bool = False + + try: + async with maybe_open_nursery( + Sampler.service_nursery + ) as service_nursery: + + # init startup, create (actor-)local service nursery and start + # increment task + Sampler.service_nursery = service_nursery + + # always ensure a period subs entry exists + last_ts, subs = Sampler.subscribers[float(period_s)] + + async with trio.Lock(): + if Sampler.incr_task_cs is None: + Sampler.incr_task_cs = await service_nursery.start( + Sampler.increment_ohlc_buffer, + 1., + ) + incr_was_started = True + + # insert the base 1s period (for OHLC style sampling) into + # the increment buffer set to update and shift every second. + if shms_by_period is not None: + from ._sharedmem import ( + attach_shm_array, + _Token, + ) + for period in shms_by_period: + + # load and register shm handles + shm_token_msg = shms_by_period[period] + shm = attach_shm_array( + _Token.from_msg(shm_token_msg), + readonly=False, + ) + shms_by_period[period] = shm + Sampler.ohlcv_shms.setdefault(period, []).append(shm) + + assert Sampler.ohlcv_shms + + # unblock caller + await ctx.started(set(Sampler.ohlcv_shms.keys())) + + if open_index_stream: + try: + async with ctx.open_stream() as stream: + subs.add(stream) + + # except broadcast requests from the subscriber + async for msg in stream: + if msg == 'broadcast_all': + await Sampler.broadcast_all() + + finally: + subs.remove(stream) + else: + # if no shms are passed in we just wait until cancelled + # by caller. + await trio.sleep_forever() + + finally: + # TODO: why tf isn't this working? + if shms_by_period is not None: + for period, shm in shms_by_period.items(): + Sampler.ohlcv_shms[period].remove(shm) + + if incr_was_started: + Sampler.incr_task_cs.cancel() + Sampler.incr_task_cs = None + + +async def spawn_samplerd( + + loglevel: str | None = None, + **extra_tractor_kwargs + +) -> bool: + ''' + Daemon-side service task: start a sampling daemon for common step + update and increment count write and stream broadcasting. + + ''' + from piker._daemon import Services + + dname = 'samplerd' + log.info(f'Spawning `{dname}`') + + # singleton lock creation of ``samplerd`` since we only ever want + # one daemon per ``pikerd`` proc tree. + # TODO: make this built-into the service api? + async with Services.locks[dname + '_singleton']: + + if dname not in Services.service_tasks: + + portal = await Services.actor_n.start_actor( + dname, + enable_modules=[ + 'piker.data._sampling', + ], + loglevel=loglevel, + debug_mode=Services.debug_mode, # set by pikerd flag + **extra_tractor_kwargs + ) + + await Services.start_service_task( + dname, + portal, + register_with_sampler, + period_s=1, + ) + return True + + return False + + +@acm +async def maybe_open_samplerd( + + loglevel: str | None = None, + **kwargs, + +) -> tractor._portal.Portal: # noqa + ''' + Client-side helper to maybe startup the ``samplerd`` service + under the ``pikerd`` tree. + + ''' + dname = 'samplerd' + + async with maybe_spawn_daemon( + dname, + service_task_target=spawn_samplerd, + spawn_args={'loglevel': loglevel}, + loglevel=loglevel, + **kwargs, + + ) as portal: + yield portal + + +@acm +async def open_sample_stream( + period_s: int, + shms_by_period: dict[float, dict] | None = None, + open_index_stream: bool = True, + + cache_key: str | None = None, + allow_new_sampler: bool = True, + +) -> AsyncIterator[dict[str, float]]: + ''' + Subscribe to OHLC sampling "step" events: when the time aggregation + period increments, this event stream emits an index event. + + This is a client-side endpoint that does all the work of ensuring + the `samplerd` actor is up and that mult-consumer-tasks are given + a broadcast stream when possible. + + ''' + # TODO: wrap this manager with the following to make it cached + # per client-multitasks entry. + # maybe_open_context( + # acm_func=partial( + # portal.open_context, + # register_with_sampler, + # ), + # key=cache_key or period_s, + # ) + # if cache_hit: + # # add a new broadcast subscription for the quote stream + # # if this feed is likely already in use + # async with istream.subscribe() as bistream: + # yield bistream + # else: + + async with ( + # XXX: this should be singleton on a host, + # a lone broker-daemon per provider should be + # created for all practical purposes + maybe_open_samplerd() as portal, + + portal.open_context( + register_with_sampler, + **{ + 'period_s': period_s, + 'shms_by_period': shms_by_period, + 'open_index_stream': open_index_stream, + }, + ) as (ctx, first) + ): + async with ( + ctx.open_stream() as istream, + + # TODO: we don't need this task-bcasting right? + # istream.subscribe() as istream, + ): + yield istream + async def sample_and_broadcast( @@ -304,7 +504,14 @@ async def sample_and_broadcast( sum_tick_vlm: bool = True, ) -> None: + ''' + `brokerd`-side task which writes latest datum sampled data. + This task is meant to run in the same actor (mem space) as the + `brokerd` real-time quote feed which is being sampled to + a ``ShmArray`` buffer. + + ''' log.info("Started shared mem bar writer") overruns = Counter() @@ -341,7 +548,6 @@ async def sample_and_broadcast( for shm in [rt_shm, hist_shm]: # update last entry # benchmarked in the 4-5 us range - # for shm in [rt_shm, hist_shm]: o, high, low, v = shm.array[-1][ ['open', 'high', 'low', 'volume'] ] From b3d1b1aa639f4aca69ba2d53e84cf493dff96864 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 4 Jan 2023 23:03:43 -0500 Subject: [PATCH 29/54] Port feed layer to use new `samplerd` APIs Always use `open_sample_stream()` to register fast and slow quote feed buffers and get a sampler stream which we use to trigger `Sampler.broadcast_all()` calls on the service side after backfill events. --- piker/data/feed.py | 161 +++++++++++++++++++++++---------------------- 1 file changed, 82 insertions(+), 79 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index 89330475..aa2a9a5a 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -74,10 +74,9 @@ from ._source import ( ) from ..ui import _search from ._sampling import ( - Sampler, + open_sample_stream, sample_and_broadcast, uniform_rate_send, - _default_delay_s, ) from ..brokers._util import ( DataUnavailable, @@ -277,6 +276,7 @@ async def start_backfill( bfqsn: str, shm: ShmArray, timeframe: float, + sampler_stream: tractor.MsgStream, last_tsdb_dt: Optional[datetime] = None, storage: Optional[Storage] = None, @@ -325,7 +325,7 @@ async def start_backfill( # TODO: *** THIS IS A BUG *** # we need to only broadcast to subscribers for this fqsn.. # otherwise all fsps get reset on every chart.. - await Sampler.broadcast_all() + await sampler_stream.send('broadcast_all') # signal that backfilling to tsdb's end datum is complete bf_done = trio.Event() @@ -493,7 +493,7 @@ async def start_backfill( # in the block above to avoid entering new ``frames`` # values while we're pipelining the current ones to # memory... - await Sampler.broadcast_all() + await sampler_stream.send('broadcast_all') # short-circuit (for now) bf_done.set() @@ -504,6 +504,7 @@ async def basic_backfill( mod: ModuleType, bfqsn: str, shms: dict[int, ShmArray], + sampler_stream: tractor.MsgStream, ) -> None: @@ -521,7 +522,8 @@ async def basic_backfill( mod, bfqsn, shm, - timeframe=timeframe, + timeframe, + sampler_stream, ) ) except DataUnavailable: @@ -537,6 +539,7 @@ async def tsdb_backfill( fqsn: str, bfqsn: str, shms: dict[int, ShmArray], + sampler_stream: tractor.MsgStream, task_status: TaskStatus[ tuple[ShmArray, ShmArray] @@ -569,7 +572,8 @@ async def tsdb_backfill( mod, bfqsn, shm, - timeframe=timeframe, + timeframe, + sampler_stream, last_tsdb_dt=last_tsdb_dt, tsdb_is_up=True, storage=storage, @@ -734,7 +738,7 @@ async def tsdb_backfill( # (usually a chart showing graphics for said fsp) # which tells the chart to conduct a manual full # graphics loop cycle. - await Sampler.broadcast_all() + await sampler_stream.send('broadcast_all') # TODO: write new data to tsdb to be ready to for next read. @@ -823,79 +827,96 @@ async def manage_history( "Persistent shm for sym was already open?!" ) - log.info('Scanning for existing `marketstored`') - tsdb_is_up = await check_for_service('marketstored') + # register 1s and 1m buffers with the global incrementer task + async with open_sample_stream( + period_s=1, + cache_key=fqsn, + shms_by_period={ + 1.: rt_shm.token, + 60.: hist_shm.token, + }, + open_index_stream=True, + ) as sample_stream: - bfqsn = fqsn.replace('.' + mod.name, '') - open_history_client = getattr(mod, 'open_history_client', None) - assert open_history_client + log.info('Scanning for existing `marketstored`') + tsdb_is_up = await check_for_service('marketstored') - if ( - tsdb_is_up - and opened - and open_history_client - ): - log.info('Found existing `marketstored`') + bfqsn = fqsn.replace('.' + mod.name, '') + open_history_client = getattr(mod, 'open_history_client', None) + assert open_history_client - from . import marketstore - async with ( - marketstore.open_storage_client(fqsn)as storage, + if ( + tsdb_is_up + and opened + and open_history_client ): - hist_shm, rt_shm = await bus.nursery.start( - tsdb_backfill, - mod, - marketstore, + log.info('Found existing `marketstored`') + + from . import marketstore + async with ( + marketstore.open_storage_client(fqsn)as storage, + ): + # TODO: drop returning the output that we pass in? + ( + hist_shm, + rt_shm, + ) = await bus.nursery.start( + tsdb_backfill, + mod, + marketstore, + bus, + storage, + fqsn, + bfqsn, + { + 1: rt_shm, + 60: hist_shm, + }, + sample_stream, + ) + + # yield back after client connect with filled shm + task_status.started(( + hist_zero_index, + hist_shm, + rt_zero_index, + rt_shm, + )) + + # indicate to caller that feed can be delivered to + # remote requesting client since we've loaded history + # data that can be used. + some_data_ready.set() + + # history retreival loop depending on user interaction + # and thus a small RPC-prot for remotely controllinlg + # what data is loaded for viewing. + await trio.sleep_forever() + + # load less history if no tsdb can be found + elif ( + not tsdb_is_up + and opened + ): + await basic_backfill( bus, - storage, - fqsn, + mod, bfqsn, { 1: rt_shm, 60: hist_shm, }, + sample_stream, ) - - # yield back after client connect with filled shm task_status.started(( hist_zero_index, hist_shm, rt_zero_index, rt_shm, )) - - # indicate to caller that feed can be delivered to - # remote requesting client since we've loaded history - # data that can be used. some_data_ready.set() - - # history retreival loop depending on user interaction and thus - # a small RPC-prot for remotely controllinlg what data is loaded - # for viewing. await trio.sleep_forever() - # load less history if no tsdb can be found - elif ( - not tsdb_is_up - and opened - ): - await basic_backfill( - bus, - mod, - bfqsn, - shms={ - 1: rt_shm, - 60: hist_shm, - }, - ) - task_status.started(( - hist_zero_index, - hist_shm, - rt_zero_index, - rt_shm, - )) - some_data_ready.set() - await trio.sleep_forever() - async def allocate_persistent_feed( bus: _FeedsBus, @@ -997,6 +1018,7 @@ async def allocate_persistent_feed( # https://github.com/python-trio/trio/issues/2258 # bus.nursery.start_soon( # await bus.start_task( + ( izero_hist, hist_shm, @@ -1030,13 +1052,6 @@ async def allocate_persistent_feed( # feed to that name (for now). bus.feeds[symstr] = bus.feeds[bfqsn] = flume - # insert 1s ohlc into the increment buffer set - # to update and shift every second - Sampler.ohlcv_shms.setdefault( - 1, - [] - ).append(rt_shm) - task_status.started() if not start_stream: @@ -1046,18 +1061,6 @@ async def allocate_persistent_feed( # the backend will indicate when real-time quotes have begun. await feed_is_live.wait() - # insert 1m ohlc into the increment buffer set - # to shift every 60s. - Sampler.ohlcv_shms.setdefault(60, []).append(hist_shm) - - # create buffer a single incrementer task broker backend - # (aka `brokerd`) using the lowest sampler period. - if Sampler.incrementers.get(_default_delay_s) is None: - await bus.start_task( - Sampler.increment_ohlc_buffer, - _default_delay_s, - ) - sum_tick_vlm: bool = init_msg.get( 'shm_write_opts', {} ).get('sum_tick_vlm', True) From e0ca5d5200293324e94efac3b221efdb2b398cfe Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 4 Jan 2023 23:12:42 -0500 Subject: [PATCH 30/54] Use `open_sample_stream()` to increment fsp buffers --- piker/fsp/_engine.py | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index 93bf0388..78448d01 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -39,7 +39,10 @@ from ..data.feed import ( Feed, ) from ..data._sharedmem import ShmArray -from ..data._sampling import _default_delay_s +from ..data._sampling import ( + _default_delay_s, + open_sample_stream, +) from ..data._source import Symbol from ._api import ( Fsp, @@ -336,7 +339,6 @@ async def cascade( symbol = flume.symbol assert src.token == flume.rt_shm.token profiler(f'{func}: feed up') - # last_len = new_len = len(src.array) func_name = func.__name__ async with ( @@ -442,22 +444,21 @@ async def cascade( # signal times = src.array['time'] if len(times) > 1: - delay_s = times[-1] - times[times != times[-1]][-1] + delay_s = float(times[-1] - times[times != times[-1]][-1]) else: # our default "HFT" sample rate. delay_s = _default_delay_s - # Increment the underlying shared memory buffer on every - # "increment" msg received from the underlying data feed. - async with flume.index_stream( - int(delay_s) - ) as istream: + # sub and increment the underlying shared memory buffer + # on every step msg received from the global `samplerd` + # service. + async with open_sample_stream(float(delay_s)) as istream: profiler(f'{func_name}: sample stream up') profiler.finish() async for i in istream: - # print(f'FSP incrementing {i}') + print(f'FSP incrementing {i}') # respawn the compute task if the source # array has been updated such that we compute From 2778ee14010075ced4c8f443c0e32fdec59da665 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Jan 2023 10:12:42 -0500 Subject: [PATCH 31/54] Support not registering for sample-index msgs via `sub_for_broadcasts: bool` flag --- piker/data/_sampling.py | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index f70e4113..f1bbc500 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -162,7 +162,6 @@ class Sampler: if shm_period_s not in broadcasted: sub_pair = self.subscribers[shm_period_s] sub_pair[0] = i_epoch - print(f'skipping `{shm_period_s}s` sample update') broadcasted.add(shm_period_s) # TODO: ``numba`` this! @@ -174,8 +173,8 @@ class Sampler: array = shm.array last = array[-1:][shm._write_fields].copy() - # guard against startup backfilling race with - # empty buffers. + # guard against startup backfilling races where + # the buffer has not yet been filled. if not last.size: continue @@ -288,7 +287,9 @@ async def register_with_sampler( ctx: tractor.Context, period_s: float, shms_by_period: dict[float, dict] | None = None, - open_index_stream: bool = True, + + open_index_stream: bool = True, # open a 2way stream for sample step msgs? + sub_for_broadcasts: bool = True, # sampler side to send step updates? ) -> None: @@ -341,15 +342,16 @@ async def register_with_sampler( if open_index_stream: try: async with ctx.open_stream() as stream: - subs.add(stream) + if sub_for_broadcasts: + subs.add(stream) # except broadcast requests from the subscriber async for msg in stream: if msg == 'broadcast_all': await Sampler.broadcast_all() - finally: - subs.remove(stream) + if sub_for_broadcasts: + subs.remove(stream) else: # if no shms are passed in we just wait until cancelled # by caller. @@ -404,6 +406,7 @@ async def spawn_samplerd( portal, register_with_sampler, period_s=1, + sub_for_broadcasts=False, ) return True @@ -437,9 +440,10 @@ async def maybe_open_samplerd( @acm async def open_sample_stream( - period_s: int, + period_s: float, shms_by_period: dict[float, dict] | None = None, open_index_stream: bool = True, + sub_for_broadcasts: bool = True, cache_key: str | None = None, allow_new_sampler: bool = True, @@ -482,6 +486,7 @@ async def open_sample_stream( 'period_s': period_s, 'shms_by_period': shms_by_period, 'open_index_stream': open_index_stream, + 'sub_for_broadcasts': sub_for_broadcasts, }, ) as (ctx, first) ): From 5adb234a2471b850c2d44708dd7c0491f4f83f97 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Jan 2023 10:13:17 -0500 Subject: [PATCH 32/54] Don't receive sample-index msgs in feed layer --- piker/data/feed.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index aa2a9a5a..c517777a 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -829,13 +829,19 @@ async def manage_history( # register 1s and 1m buffers with the global incrementer task async with open_sample_stream( - period_s=1, - cache_key=fqsn, + period_s=1., shms_by_period={ 1.: rt_shm.token, 60.: hist_shm.token, }, + + # NOTE: we want to only open a stream for doing broadcasts on + # backfill operations, not receive the sample index-stream + # (since there's no code in this data feed layer that needs to + # consume it). open_index_stream=True, + sub_for_broadcasts=False, + ) as sample_stream: log.info('Scanning for existing `marketstored`') From a746258f997a3a36bc0bb4e72dc6cdab1f32c569 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Jan 2023 12:44:58 -0500 Subject: [PATCH 33/54] `binance`: always request an extra 1min OHLC bar Seems that by default their history indexing rounds down/back to the previous time step, so make sure we add a minute inside `Client.bars()` when the `end_dt=None`, indicating "get the latest bar". Add a breakpoint block that should trigger whenever the latest bar vs. the latest epoch time is mismatched; we'll remove this after some testing verifying the history bars issue is resolved. Further this drops the legacy `backfill_bars()` endpoint which has been deprecated and unused for a while. --- piker/fsp/_engine.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index 78448d01..f9a4f797 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -458,7 +458,7 @@ async def cascade( profiler.finish() async for i in istream: - print(f'FSP incrementing {i}') + # print(f'FSP incrementing {i}') # respawn the compute task if the source # array has been updated such that we compute From 78c7c8524c1f3065bb463bccf8b79ab6f4c48210 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Jan 2023 15:09:31 -0500 Subject: [PATCH 34/54] Breakpoint when bad 1m history offsets are detected --- piker/data/feed.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index c517777a..b1678d26 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -126,7 +126,7 @@ class _FeedsBus(Struct): target: Awaitable, *args, - ) -> None: + ) -> trio.CancelScope: async def start_with_cs( task_status: TaskStatus[ @@ -308,6 +308,11 @@ async def start_backfill( - pendulum.from_timestamp(times[-2]) ).seconds + if step_size_s == 60: + inow = round(time.time()) + if (inow - times[-1]) > 60: + await tractor.breakpoint() + # frame's worth of sample-period-steps, in seconds frame_size_s = len(array) * step_size_s From d66fb49077b363c03c67a0700c2e23299cdeddf8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Jan 2023 17:51:11 -0500 Subject: [PATCH 35/54] Don't deliver shms from `start_backfill()`, they're not used --- piker/data/feed.py | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index b1678d26..cc287d40 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -414,7 +414,9 @@ async def start_backfill( and starts[next_start_dt] <= 6 ): start_dt = min(starts) - print(f"SKIPPING DUPLICATE FRAME @ {next_start_dt}") + log.warning( + f"{bfqsn}: skipping duplicate frame @ {next_start_dt}" + ) starts[start_dt] += 1 continue @@ -616,10 +618,7 @@ async def tsdb_backfill( # unblock the feed bus management task # assert len(shms[1].array) - task_status.started(( - shms[60], - shms[1], - )) + task_status.started() async def back_load_from_tsdb( timeframe: int, @@ -868,10 +867,7 @@ async def manage_history( marketstore.open_storage_client(fqsn)as storage, ): # TODO: drop returning the output that we pass in? - ( - hist_shm, - rt_shm, - ) = await bus.nursery.start( + await bus.nursery.start( tsdb_backfill, mod, marketstore, From d792fed09911c8be0ac6a5105f394273638ffe49 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 5 Jan 2023 19:32:34 -0500 Subject: [PATCH 36/54] Move sync log msg back to info --- piker/fsp/_engine.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/piker/fsp/_engine.py b/piker/fsp/_engine.py index f9a4f797..a78308a4 100644 --- a/piker/fsp/_engine.py +++ b/piker/fsp/_engine.py @@ -21,7 +21,9 @@ core task logic for processing chains from dataclasses import dataclass from functools import partial from typing import ( - AsyncIterator, Callable, Optional, + AsyncIterator, + Callable, + Optional, Union, ) @@ -386,7 +388,7 @@ async def cascade( ) -> tuple[TaskTracker, int]: # TODO: adopt an incremental update engine/approach # where possible here eventually! - log.debug(f're-syncing fsp {func_name} to source') + log.info(f're-syncing fsp {func_name} to source') tracker.cs.cancel() await tracker.complete.wait() tracker, index = await n.start(fsp_target) @@ -429,6 +431,7 @@ async def cascade( tracker: TaskTracker, src: ShmArray, dst: ShmArray, + ) -> tuple[TaskTracker, int]: synced, step_diff, _ = is_synced(src, dst) @@ -444,7 +447,8 @@ async def cascade( # signal times = src.array['time'] if len(times) > 1: - delay_s = float(times[-1] - times[times != times[-1]][-1]) + last_ts = times[-1] + delay_s = float(last_ts - times[times != last_ts][-1]) else: # our default "HFT" sample rate. delay_s = _default_delay_s From 75591dd7e919036e67be5fb01ac00dd031a723a2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 9 Jan 2023 14:24:39 -0500 Subject: [PATCH 37/54] Don't raise on quote feed lags to dark clearing loop --- piker/clearing/_ems.py | 1 + 1 file changed, 1 insertion(+) diff --git a/piker/clearing/_ems.py b/piker/clearing/_ems.py index 06ae3b70..8d6ba868 100644 --- a/piker/clearing/_ems.py +++ b/piker/clearing/_ems.py @@ -172,6 +172,7 @@ async def clear_dark_triggers( # TODO: # - numba all this! # - this stream may eventually contain multiple symbols + quote_stream._raise_on_lag = False async for quotes in quote_stream: # start = time.time() for sym, quote in quotes.items(): From 6a1bb13feb5515905ea3e62ffd4a25fde02e5163 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Jan 2023 13:16:45 -0500 Subject: [PATCH 38/54] Add base `pikerd` service tree custom check test --- tests/test_services.py | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 tests/test_services.py diff --git a/tests/test_services.py b/tests/test_services.py new file mode 100644 index 00000000..19d38b54 --- /dev/null +++ b/tests/test_services.py @@ -0,0 +1,37 @@ +''' +Actor tree daemon sub-service verifications + +''' +from typing import AsyncContextManager + +import trio +import tractor + + +def test_runtime_boot( + open_test_pikerd: AsyncContextManager +): + ''' + Verify we can boot the `pikerd` service stack using the + `open_test_pikerd` fixture helper and that registry address details + match up. + + ''' + async def main(): + port = 6666 + daemon_addr = ('127.0.0.1', port) + + async with ( + open_test_pikerd( + reg_addr=daemon_addr, + ) as (_, _, pikerd_portal), + + tractor.wait_for_actor( + 'pikerd', + arbiter_sockaddr=daemon_addr, + ) as portal, + ): + assert pikerd_portal.channel.raddr == daemon_addr + assert pikerd_portal.channel.raddr == portal.channel.raddr + + trio.run(main) From c8c641a03821854b95d2bbb0b4ef35508aa4b218 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Jan 2023 15:25:25 -0500 Subject: [PATCH 39/54] Ensure all sub-services cancel on `pikerd` exit Previously we were relying on implicit actor termination in `maybe_spawn_daemon()` but really on `pikerd` teardown we should be sure to tear down not only all service tasks in each actor but also the actor runtimes. This adjusts `Services.cancel_service()` to only cancel the service task scope and wait on the `complete` event and reworks the `open_context_in_task()` inner closure body to, - always cancel the service actor at exit. - not call `.cancel_service()` (potentially causing recursion issues on cancellation). - allocate a `complete: trio.Event` to signal full task + actor termination. - pop the service task from the `.service_tasks` registry. Further, add a `maybe_set_global_registry_sockaddr()` helper-cm to do the work of checking whether a registry socket needs-to/has-been set and use it for discovery calls to the `pikerd` service tree. --- piker/_daemon.py | 190 +++++++++++++++++++++++++++++------------------ 1 file changed, 118 insertions(+), 72 deletions(-) diff --git a/piker/_daemon.py b/piker/_daemon.py index 4e13e1ec..9bfefca8 100644 --- a/piker/_daemon.py +++ b/piker/_daemon.py @@ -19,7 +19,10 @@ Structured, daemon tree service management. """ from typing import Optional, Union, Callable, Any -from contextlib import asynccontextmanager as acm +from contextlib import ( + asynccontextmanager as acm, + contextmanager as cm, +) from collections import defaultdict import tractor @@ -57,12 +60,21 @@ _root_modules = [ ] +# TODO: factor this into a ``tractor.highlevel`` extension +# pack for the library. class Services: actor_n: tractor._supervise.ActorNursery service_n: trio.Nursery debug_mode: bool # tractor sub-actor debug mode flag - service_tasks: dict[str, tuple[trio.CancelScope, tractor.Portal]] = {} + service_tasks: dict[ + str, + tuple[ + trio.CancelScope, + tractor.Portal, + trio.Event, + ] + ] = {} locks = defaultdict(trio.Lock) @classmethod @@ -84,7 +96,12 @@ class Services: ''' async def open_context_in_task( task_status: TaskStatus[ - trio.CancelScope] = trio.TASK_STATUS_IGNORED, + tuple[ + trio.CancelScope, + trio.Event, + Any, + ] + ] = trio.TASK_STATUS_IGNORED, ) -> Any: @@ -96,28 +113,33 @@ class Services: ) as (ctx, first): # unblock once the remote context has started - task_status.started((cs, first)) + complete = trio.Event() + task_status.started((cs, complete, first)) log.info( f'`pikerd` service {name} started with value {first}' ) try: # wait on any context's return value + # and any final portal result from the + # sub-actor. ctx_res = await ctx.result() - except tractor.ContextCancelled: - return await self.cancel_service(name) - else: - # wait on any error from the sub-actor - # NOTE: this will block indefinitely until - # cancelled either by error from the target - # context function or by being cancelled here by - # the surrounding cancel scope + + # NOTE: blocks indefinitely until cancelled + # either by error from the target context + # function or by being cancelled here by the + # surrounding cancel scope. return (await portal.result(), ctx_res) - cs, first = await self.service_n.start(open_context_in_task) + finally: + await portal.cancel_actor() + complete.set() + self.service_tasks.pop(name) + + cs, complete, first = await self.service_n.start(open_context_in_task) # store the cancel scope and portal for later cancellation or # retstart if needed. - self.service_tasks[name] = (cs, portal) + self.service_tasks[name] = (cs, portal, complete) return cs, first @@ -127,13 +149,38 @@ class Services: name: str, ) -> Any: + ''' + Cancel the service task and actor for the given ``name``. + + ''' log.info(f'Cancelling `pikerd` service {name}') - cs, portal = self.service_tasks[name] - # XXX: not entirely sure why this is required, - # and should probably be better fine tuned in - # ``tractor``? + cs, portal, complete = self.service_tasks[name] cs.cancel() - return await portal.cancel_actor() + await complete.wait() + assert name not in self.service_tasks, \ + f'Serice task for {name} not terminated?' + + +@cm +def maybe_set_global_registry_sockaddr( + registry_addr: None | tuple[str, int] = None, +) -> None: + + global _registry_addr + was_set: bool = False + if ( + _registry_addr is None + or registry_addr + ): + _registry_addr = registry_addr or _default_reg_addr + try: + yield _registry_addr + finally: + # XXX: always clear the global addr if we set it so that the + # next (set of) calls will apply whatever new one is passed + # in. + if was_set: + _registry_addr = None @acm @@ -155,40 +202,38 @@ async def open_pikerd( alive underling services (see below). ''' - global _registry_addr - if ( - _registry_addr is None - or registry_addr - ): - _registry_addr = registry_addr or _default_reg_addr + with maybe_set_global_registry_sockaddr(registry_addr) as reg_addr: + async with ( + tractor.open_root_actor( - # XXX: this may open a root actor as well - async with ( - tractor.open_root_actor( + # passed through to ``open_root_actor`` + arbiter_addr=reg_addr, + name=_root_dname, + loglevel=loglevel, + debug_mode=debug_mode, + start_method=start_method, - # passed through to ``open_root_actor`` - arbiter_addr=_registry_addr, - name=_root_dname, - loglevel=loglevel, - debug_mode=debug_mode, - start_method=start_method, + # TODO: eventually we should be able to avoid + # having the root have more then permissions to + # spawn other specialized daemons I think? + enable_modules=_root_modules, + ) as _, - # TODO: eventually we should be able to avoid - # having the root have more then permissions to - # spawn other specialized daemons I think? - enable_modules=_root_modules, - ) as _, + tractor.open_nursery() as actor_nursery, + ): + async with trio.open_nursery() as service_nursery: - tractor.open_nursery() as actor_nursery, - ): - async with trio.open_nursery() as service_nursery: - - # assign globally for future daemon/task creation - Services.actor_n = actor_nursery - Services.service_n = service_nursery - Services.debug_mode = debug_mode - yield + # assign globally for future daemon/task creation + Services.actor_n = actor_nursery + Services.service_n = service_nursery + Services.debug_mode = debug_mode + try: + yield + finally: + # if 'samplerd' in Services.service_tasks: + # await Services.cancel_service('samplerd') + service_nursery.cancel_scope.cancel() @acm @@ -209,32 +254,24 @@ async def open_piker_runtime( existing piker actors on the local link based on configuration. ''' - global _registry_addr + with maybe_set_global_registry_sockaddr(registry_addr) as reg_addr: + async with ( + tractor.open_root_actor( - if ( - _registry_addr is None - or registry_addr - ): - _registry_addr = registry_addr or _default_reg_addr + # passed through to ``open_root_actor`` + arbiter_addr=reg_addr, + name=name, + loglevel=loglevel, + debug_mode=debug_mode, + start_method=start_method, - # XXX: this may open a root actor as well - async with ( - tractor.open_root_actor( - - # passed through to ``open_root_actor`` - arbiter_addr=_registry_addr, - name=name, - loglevel=loglevel, - debug_mode=debug_mode, - start_method=start_method, - - # TODO: eventually we should be able to avoid - # having the root have more then permissions to - # spawn other specialized daemons I think? - enable_modules=_root_modules + enable_modules, - ) as _, - ): - yield tractor.current_actor() + # TODO: eventually we should be able to avoid + # having the root have more then permissions to + # spawn other specialized daemons I think? + enable_modules=_root_modules + enable_modules, + ) as _, + ): + yield tractor.current_actor() @acm @@ -325,6 +362,11 @@ async def find_service( service_name: str, ) -> Optional[tractor.Portal]: + global _registry_addr + if not _registry_addr: + yield None + return + log.info(f'Scanning for service `{service_name}`') # attach to existing daemon by name if possible async with tractor.find_actor( @@ -342,6 +384,10 @@ async def check_for_service( Service daemon "liveness" predicate. ''' + global _registry_addr + if not _registry_addr: + return None + async with tractor.query_actor( service_name, arbiter_sockaddr=_registry_addr, From 045b76bab5661e6163f5ceeb7633acd1336ba7b3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Jan 2023 15:36:39 -0500 Subject: [PATCH 40/54] Make `Flume.index_stream()` defer to new sampling api --- piker/data/flows.py | 22 +++++----------------- 1 file changed, 5 insertions(+), 17 deletions(-) diff --git a/piker/data/flows.py b/piker/data/flows.py index 23fa4207..9bb27230 100644 --- a/piker/data/flows.py +++ b/piker/data/flows.py @@ -21,6 +21,7 @@ real-time data processing data-structures. "Streams, flumes, cascades and flows.." """ +from __future__ import annotations from contextlib import asynccontextmanager as acm from functools import partial from typing import ( @@ -45,7 +46,7 @@ from ._sharedmem import ( _Token, ) from ._sampling import ( - iter_ohlc_periods, + open_sample_stream, ) if TYPE_CHECKING: @@ -149,7 +150,7 @@ class Flume(Struct): @acm async def index_stream( self, - delay_s: int = 1, + delay_s: float = 1, ) -> AsyncIterator[int]: @@ -163,21 +164,8 @@ class Flume(Struct): # XXX: this should be singleton on a host, # a lone broker-daemon per provider should be # created for all practical purposes - async with maybe_open_context( - acm_func=partial( - portal.open_context, - iter_ohlc_periods, - ), - kwargs={'delay_s': delay_s}, - ) as (cache_hit, (ctx, first)): - async with ctx.open_stream() as istream: - if cache_hit: - # add a new broadcast subscription for the quote stream - # if this feed is likely already in use - async with istream.subscribe() as bistream: - yield bistream - else: - yield istream + async with open_sample_stream(float(delay_s)) as stream: + yield stream def get_ds_info( self, From 37e0ec7b7d90f7e528841484c5418ab85c144dbe Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Jan 2023 15:40:45 -0500 Subject: [PATCH 41/54] Assert fixture caller is `pikerd` --- tests/conftest.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/conftest.py b/tests/conftest.py index 1bd1d86e..d781cf61 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -137,12 +137,16 @@ async def _open_test_pikerd( port = random.randint(6e3, 7e3) reg_addr = ('127.0.0.1', port) + # try: async with ( maybe_open_pikerd( registry_addr=reg_addr, **kwargs, - ), + ) as pikerd_portal, ): + # this proc/actor is the pikerd + assert pikerd_portal is None + async with tractor.wait_for_actor( 'pikerd', arbiter_sockaddr=reg_addr, From da659cf607d48cbb802051264812300fad40395d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Jan 2023 16:40:47 -0500 Subject: [PATCH 42/54] Facepalm: definitely do not short circuit discovery helpers.. --- piker/_daemon.py | 20 ++++++-------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/piker/_daemon.py b/piker/_daemon.py index 9bfefca8..9e68ebdd 100644 --- a/piker/_daemon.py +++ b/piker/_daemon.py @@ -193,7 +193,7 @@ async def open_pikerd( debug_mode: bool = False, registry_addr: None | tuple[str, int] = None, -) -> None: +) -> Services: ''' Start a root piker daemon who's lifetime extends indefinitely until cancelled. @@ -229,7 +229,7 @@ async def open_pikerd( Services.service_n = service_nursery Services.debug_mode = debug_mode try: - yield + yield Services finally: # if 'samplerd' in Services.service_tasks: # await Services.cancel_service('samplerd') @@ -338,10 +338,11 @@ async def maybe_open_pikerd( debug_mode=kwargs.get('debug_mode', False), registry_addr=registry_addr, - ) as _: + ) as service_manager: # in the case where we're starting up the # tractor-piker runtime stack in **this** process # we return no portal to self. + assert service_manager yield None @@ -362,16 +363,11 @@ async def find_service( service_name: str, ) -> Optional[tractor.Portal]: - global _registry_addr - if not _registry_addr: - yield None - return - log.info(f'Scanning for service `{service_name}`') # attach to existing daemon by name if possible async with tractor.find_actor( service_name, - arbiter_sockaddr=_registry_addr, + arbiter_sockaddr=_registry_addr or _default_reg_addr, ) as maybe_portal: yield maybe_portal @@ -384,13 +380,9 @@ async def check_for_service( Service daemon "liveness" predicate. ''' - global _registry_addr - if not _registry_addr: - return None - async with tractor.query_actor( service_name, - arbiter_sockaddr=_registry_addr, + arbiter_sockaddr=_registry_addr or _default_reg_addr, ) as sockaddr: return sockaddr From 9811dcf5f3ddb7a288d7652bc459c26e5a92bb81 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 10 Jan 2023 16:41:20 -0500 Subject: [PATCH 43/54] Match `services` subcmd to new reg addr module variables --- piker/cli/__init__.py | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/piker/cli/__init__.py b/piker/cli/__init__.py index 67647a83..07484634 100644 --- a/piker/cli/__init__.py +++ b/piker/cli/__init__.py @@ -28,7 +28,6 @@ import tractor from ..log import get_console_log, get_logger, colorize_json from ..brokers import get_brokermod from .._daemon import ( - _tractor_kwargs, _default_registry_host, _default_registry_port, ) @@ -176,20 +175,30 @@ def cli( @cli.command() @click.option('--tl', is_flag=True, help='Enable tractor logging') -@click.argument('names', nargs=-1, required=False) +@click.argument('ports', nargs=-1, required=False) @click.pass_obj -def services(config, tl, names): +def services(config, tl, ports): - from .._daemon import open_piker_runtime + from .._daemon import ( + open_piker_runtime, + _default_registry_port, + _default_registry_host, + ) + + host = _default_registry_host + if not ports: + ports = [_default_registry_port] async def list_services(): + nonlocal host async with ( open_piker_runtime( name='service_query', loglevel=config['loglevel'] if tl else None, ), tractor.get_arbiter( - *_tractor_kwargs['arbiter_addr'] + host=host, + port=ports[0] ) as portal ): registry = await portal.run_from_ns('self', 'get_registry') From 71ca4c8e1f5425c66064669a1f6bd3d5976c31f2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Jan 2023 11:05:24 -0500 Subject: [PATCH 44/54] Use actor uid in shm keys for rt quote buffers Allows running simultaneous data feed services on the same (linux) host by avoiding file-name collisions instead keying shm buffer sets by the given `brokerd` instance. This allows, for example, either multiple dev versions of the data layer to run side-by-side or for the test suite to be seamlessly run alongside a production instance. --- piker/data/feed.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/piker/data/feed.py b/piker/data/feed.py index cc287d40..88e9ceed 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -786,11 +786,14 @@ async def manage_history( # from tractor._state import _runtime_vars # port = _runtime_vars['_root_mailbox'][1] + uid = tractor.current_actor().uid + suffix = '.'.join(uid) + # (maybe) allocate shm array for this broker/symbol which will # be used for fast near-term history capture and processing. hist_shm, opened = maybe_open_shm_array( # key=f'{fqsn}_hist_p{port}', - key=f'{fqsn}_hist', + key=f'{fqsn}_hist.{suffix}', # use any broker defined ohlc dtype: dtype=getattr(mod, '_ohlc_dtype', base_iohlc_dtype), @@ -808,7 +811,7 @@ async def manage_history( rt_shm, opened = maybe_open_shm_array( # key=f'{fqsn}_rt_p{port}', - key=f'{fqsn}_rt', + key=f'{fqsn}_rt.{suffix}', # use any broker defined ohlc dtype: dtype=getattr(mod, '_ohlc_dtype', base_iohlc_dtype), From 09711750bf6e7f2f9ce8b2c998b6f6f9fe97084e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Jan 2023 20:51:21 -0500 Subject: [PATCH 45/54] Registry subsys rework More or less a revamp (and possibly first draft for something similar in `tractor` core) which ensures all actor trees attempt to discover the `pikerd` registry actor. Implementation improvements include: - new `Registry` singleton which houses the `pikerd` discovery socket-address `Registry.addr` + a `open_registry()` manager which provides bootstrapped actor-local access. - refine `open_piker_runtime()` to do the work of opening a root actor and call the new `open_registry()` depending on whether a runtime has yet been bootstrapped. - rejig `[maybe_]open_pikerd()` in terms of the above. --- piker/_daemon.py | 351 +++++++++++++++++++++++++++++++---------------- 1 file changed, 230 insertions(+), 121 deletions(-) diff --git a/piker/_daemon.py b/piker/_daemon.py index 9e68ebdd..6f77cc48 100644 --- a/piker/_daemon.py +++ b/piker/_daemon.py @@ -18,10 +18,16 @@ Structured, daemon tree service management. """ -from typing import Optional, Union, Callable, Any +from __future__ import annotations +import os +from typing import ( + Optional, + Union, + Callable, + Any, +) from contextlib import ( asynccontextmanager as acm, - contextmanager as cm, ) from collections import defaultdict @@ -29,7 +35,10 @@ import tractor import trio from trio_typing import TaskStatus -from .log import get_logger, get_console_log +from .log import ( + get_logger, + get_console_log, +) from .brokers import get_brokermod @@ -44,14 +53,86 @@ _default_reg_addr: tuple[str, int] = ( _default_registry_port, ) + # NOTE: this value is set as an actor-global once the first endpoint # who is capable, spawns a `pikerd` service tree. -_registry_addr: tuple[str, int] | None = None +_registry: Registry | None = None + + +class Registry: + addr: None | tuple[str, int] = None + + # TODO: table of uids to sockaddrs + peers: dict[ + tuple[str, str], + tuple[str, int], + ] = {} + + +# _registry_addr: None | tuple[str, int] = None +_tractor_kwargs: dict[str, Any] = {} + + +@acm +async def open_registry( + addr: None | tuple[str, int] = None, + ensure_exists: bool = True, + +) -> tuple[str, int]: + + global _tractor_kwargs + actor = tractor.current_actor() + uid = actor.uid + if ( + Registry.addr is not None + and addr + ): + raise RuntimeError( + f'`{uid}` registry addr already bound @ {_registry.sockaddr}' + ) + + was_set: bool = False + + if ( + not tractor.is_root_process() + and Registry.addr is None + ): + Registry.addr = actor._arb_addr + + if ( + ensure_exists + and Registry.addr is None + ): + raise RuntimeError( + f"`{uid}` registry should already exist bug doesn't?" + ) + + if ( + Registry.addr is None + ): + was_set = True + Registry.addr = addr or _default_reg_addr + + _tractor_kwargs['arbiter_addr'] = Registry.addr + + try: + yield Registry.addr + finally: + # XXX: always clear the global addr if we set it so that the + # next (set of) calls will apply whatever new one is passed + # in. + if was_set: + Registry.addr = None + + +def get_tractor_runtime_kwargs() -> dict[str, Any]: + ''' + Deliver ``tractor`` related runtime variables in a `dict`. + + ''' + return _tractor_kwargs + -_tractor_kwargs: dict[str, Any] = { - # use a different registry addr then tractor's default - 'arbiter_addr': _registry_addr -} _root_modules = [ __name__, 'piker.clearing._ems', @@ -161,31 +242,78 @@ class Services: f'Serice task for {name} not terminated?' -@cm -def maybe_set_global_registry_sockaddr( - registry_addr: None | tuple[str, int] = None, -) -> None: +@acm +async def open_piker_runtime( + name: str, + enable_modules: list[str] = [], + loglevel: Optional[str] = None, - global _registry_addr - was_set: bool = False - if ( - _registry_addr is None - or registry_addr - ): - _registry_addr = registry_addr or _default_reg_addr - try: - yield _registry_addr - finally: - # XXX: always clear the global addr if we set it so that the - # next (set of) calls will apply whatever new one is passed - # in. - if was_set: - _registry_addr = None + # XXX NOTE XXX: you should pretty much never want debug mode + # for data daemons when running in production. + debug_mode: bool = False, + + registry_addr: None | tuple[str, int] = None, + + # TODO: once we have `rsyscall` support we will read a config + # and spawn the service tree distributed per that. + start_method: str = 'trio', + + tractor_kwargs: dict = {}, + +) -> tuple[ + tractor.Actor, + tuple[str, int], +]: + ''' + Start a piker actor who's runtime will automatically sync with + existing piker actors on the local link based on configuration. + + Can be called from a subactor or any program that needs to start + a root actor. + + ''' + try: + # check for existing runtime + actor = tractor.current_actor().uid + + except tractor._exceptions.NoRuntime: + + registry_addr = registry_addr or _default_reg_addr + + async with ( + tractor.open_root_actor( + + # passed through to ``open_root_actor`` + arbiter_addr=registry_addr, + name=name, + loglevel=loglevel, + debug_mode=debug_mode, + start_method=start_method, + + # TODO: eventually we should be able to avoid + # having the root have more then permissions to + # spawn other specialized daemons I think? + enable_modules=enable_modules, + + **tractor_kwargs, + ) as _, + + open_registry(registry_addr, ensure_exists=False) as addr, + ): + yield ( + tractor.current_actor(), + addr, + ) + else: + async with open_registry(registry_addr) as addr: + yield ( + actor, + addr, + ) @acm async def open_pikerd( - start_method: str = 'trio', loglevel: str | None = None, # XXX: you should pretty much never want debug mode @@ -203,75 +331,35 @@ async def open_pikerd( ''' - with maybe_set_global_registry_sockaddr(registry_addr) as reg_addr: - async with ( - tractor.open_root_actor( + async with ( + open_piker_runtime( - # passed through to ``open_root_actor`` - arbiter_addr=reg_addr, - name=_root_dname, - loglevel=loglevel, - debug_mode=debug_mode, - start_method=start_method, + name=_root_dname, + # TODO: eventually we should be able to avoid + # having the root have more then permissions to + # spawn other specialized daemons I think? + enable_modules=_root_modules, - # TODO: eventually we should be able to avoid - # having the root have more then permissions to - # spawn other specialized daemons I think? - enable_modules=_root_modules, - ) as _, + loglevel=loglevel, + debug_mode=debug_mode, + registry_addr=registry_addr, - tractor.open_nursery() as actor_nursery, - ): - async with trio.open_nursery() as service_nursery: + ) as (root_actor, reg_addr), + tractor.open_nursery() as actor_nursery, + trio.open_nursery() as service_nursery, + ): + assert root_actor.accept_addr == reg_addr - # assign globally for future daemon/task creation - Services.actor_n = actor_nursery - Services.service_n = service_nursery - Services.debug_mode = debug_mode - try: - yield Services - finally: - # if 'samplerd' in Services.service_tasks: - # await Services.cancel_service('samplerd') - service_nursery.cancel_scope.cancel() - - -@acm -async def open_piker_runtime( - name: str, - enable_modules: list[str] = [], - start_method: str = 'trio', - loglevel: Optional[str] = None, - - # XXX: you should pretty much never want debug mode - # for data daemons when running in production. - debug_mode: bool = False, - registry_addr: None | tuple[str, int] = None, - -) -> tractor.Actor: - ''' - Start a piker actor who's runtime will automatically sync with - existing piker actors on the local link based on configuration. - - ''' - with maybe_set_global_registry_sockaddr(registry_addr) as reg_addr: - async with ( - tractor.open_root_actor( - - # passed through to ``open_root_actor`` - arbiter_addr=reg_addr, - name=name, - loglevel=loglevel, - debug_mode=debug_mode, - start_method=start_method, - - # TODO: eventually we should be able to avoid - # having the root have more then permissions to - # spawn other specialized daemons I think? - enable_modules=_root_modules + enable_modules, - ) as _, - ): - yield tractor.current_actor() + # assign globally for future daemon/task creation + Services.actor_n = actor_nursery + Services.service_n = service_nursery + Services.debug_mode = debug_mode + try: + yield Services + finally: + # if 'samplerd' in Services.service_tasks: + # await Services.cancel_service('samplerd') + service_nursery.cancel_scope.cancel() @acm @@ -284,23 +372,30 @@ async def maybe_open_runtime( Start the ``tractor`` runtime (a root actor) if none exists. """ - settings = _tractor_kwargs - settings.update(kwargs) + # settings = get_tractor_runtime_kwargs() + # settings.update(kwargs) + + name = kwargs.pop('name') if not tractor.current_actor(err_on_no_runtime=False): - async with tractor.open_root_actor( + async with open_piker_runtime( + name, loglevel=loglevel, - **settings, - ): - yield + **kwargs, + # registry + # tractor_kwargs=kwargs, + ) as (_, addr): + yield addr, else: - yield + async with open_registry() as addr: + yield addr @acm async def maybe_open_pikerd( loglevel: Optional[str] = None, registry_addr: None | tuple = None, + **kwargs, ) -> Union[tractor._portal.Portal, Services]: @@ -314,18 +409,31 @@ async def maybe_open_pikerd( get_console_log(loglevel) # subtle, we must have the runtime up here or portal lookup will fail + query_name = kwargs.pop('name', f'piker_query_{os.getpid()}') + + # TODO: if we need to make the query part faster we could not init + # an actor runtime and instead just hit the socket? + # from tractor._ipc import _connect_chan, Channel + # async with _connect_chan(host, port) as chan: + # async with open_portal(chan) as arb_portal: + # yield arb_portal + async with ( - maybe_open_runtime(loglevel, **kwargs), - tractor.find_actor(_root_dname) as portal + open_piker_runtime( + name=query_name, + registry_addr=registry_addr, + loglevel=loglevel, + **kwargs, + ) as _, + tractor.find_actor( + _root_dname, + arbiter_sockaddr=registry_addr, + ) as portal ): # connect to any existing daemon presuming # its registry socket was selected. if ( portal is not None - and ( - registry_addr is None - or portal.channel.raddr == registry_addr - ) ): yield portal return @@ -333,7 +441,6 @@ async def maybe_open_pikerd( # presume pikerd role since no daemon could be found at # configured address async with open_pikerd( - loglevel=loglevel, debug_mode=kwargs.get('debug_mode', False), registry_addr=registry_addr, @@ -361,30 +468,32 @@ _data_mods = [ @acm async def find_service( service_name: str, -) -> Optional[tractor.Portal]: +) -> tractor.Portal | None: - log.info(f'Scanning for service `{service_name}`') - # attach to existing daemon by name if possible - async with tractor.find_actor( - service_name, - arbiter_sockaddr=_registry_addr or _default_reg_addr, - ) as maybe_portal: - yield maybe_portal + async with open_registry() as reg_addr: + log.info(f'Scanning for service `{service_name}`') + # attach to existing daemon by name if possible + async with tractor.find_actor( + service_name, + arbiter_sockaddr=reg_addr, + ) as maybe_portal: + yield maybe_portal async def check_for_service( service_name: str, -) -> bool: +) -> None | tuple[str, int]: ''' Service daemon "liveness" predicate. ''' - async with tractor.query_actor( - service_name, - arbiter_sockaddr=_registry_addr or _default_reg_addr, - ) as sockaddr: - return sockaddr + async with open_registry(ensure_exists=False) as reg_addr: + async with tractor.query_actor( + service_name, + arbiter_sockaddr=reg_addr, + ) as sockaddr: + return sockaddr @acm From 0cd87d9e5458c3f75ea49a1739f40a2d7f1927c9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Jan 2023 21:46:12 -0500 Subject: [PATCH 46/54] Drop commented markestored spawner code --- piker/_daemon.py | 33 +++------------------------------ 1 file changed, 3 insertions(+), 30 deletions(-) diff --git a/piker/_daemon.py b/piker/_daemon.py index 6f77cc48..0369f3f9 100644 --- a/piker/_daemon.py +++ b/piker/_daemon.py @@ -69,7 +69,6 @@ class Registry: ] = {} -# _registry_addr: None | tuple[str, int] = None _tractor_kwargs: dict[str, Any] = {} @@ -357,6 +356,7 @@ async def open_pikerd( try: yield Services finally: + # TODO: is this more clever/efficient? # if 'samplerd' in Services.service_tasks: # await Services.cancel_service('samplerd') service_nursery.cancel_scope.cancel() @@ -368,13 +368,10 @@ async def maybe_open_runtime( **kwargs, ) -> None: - """ + ''' Start the ``tractor`` runtime (a root actor) if none exists. - """ - # settings = get_tractor_runtime_kwargs() - # settings.update(kwargs) - + ''' name = kwargs.pop('name') if not tractor.current_actor(err_on_no_runtime=False): @@ -382,8 +379,6 @@ async def maybe_open_runtime( name, loglevel=loglevel, **kwargs, - # registry - # tractor_kwargs=kwargs, ) as (_, addr): yield addr, else: @@ -707,25 +702,3 @@ async def maybe_open_emsd( ) as portal: yield portal - - -# TODO: ideally we can start the tsdb "on demand" but it's -# probably going to require "rootless" docker, at least if we don't -# want to expect the user to start ``pikerd`` with root perms all the -# time. -# async def maybe_open_marketstored( -# loglevel: Optional[str] = None, -# **kwargs, - -# ) -> tractor._portal.Portal: # noqa - -# async with maybe_spawn_daemon( - -# 'marketstored', -# service_task_target=spawn_emsd, -# spawn_args={'loglevel': loglevel}, -# loglevel=loglevel, -# **kwargs, - -# ) as portal: -# yield portal From 470079665f6a4e8cffc07224a94a539ca5484ca2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 11 Jan 2023 22:38:21 -0500 Subject: [PATCH 47/54] Use new tractor kwargs getter func --- piker/ui/_exec.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/piker/ui/_exec.py b/piker/ui/_exec.py index b0fa6446..d8eabb70 100644 --- a/piker/ui/_exec.py +++ b/piker/ui/_exec.py @@ -49,7 +49,10 @@ from qdarkstyle import DarkPalette import trio from outcome import Error -from .._daemon import maybe_open_pikerd, _tractor_kwargs +from .._daemon import ( + maybe_open_pikerd, + get_tractor_runtime_kwargs, +) from ..log import get_logger from ._pg_overrides import _do_overrides from . import _style @@ -170,7 +173,7 @@ def run_qtractor( instance.window = window # override tractor's defaults - tractor_kwargs.update(_tractor_kwargs) + tractor_kwargs.update(get_tractor_runtime_kwargs()) # define tractor entrypoint async def main(): From 844626f6dc9e1f94e616bd76cb66e78f4f30fd9a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 12 Jan 2023 10:16:16 -0500 Subject: [PATCH 48/54] Move `brokerd` service task to root `.data` mod --- piker/data/__init__.py | 44 ++++++++++++++++++++++++++++++++++++++++-- piker/data/feed.py | 35 ++++----------------------------- 2 files changed, 46 insertions(+), 33 deletions(-) diff --git a/piker/data/__init__.py b/piker/data/__init__.py index e98195b4..5c83150e 100644 --- a/piker/data/__init__.py +++ b/piker/data/__init__.py @@ -22,6 +22,12 @@ and storing data from your brokers as well as sharing live streams over a network. """ +import tractor +import trio + +from ..log import ( + get_console_log, +) from ._normalize import iterticks from ._sharedmem import ( maybe_open_shm_array, @@ -32,7 +38,6 @@ from ._sharedmem import ( ) from .feed import ( open_feed, - _setup_persistent_brokerd, ) @@ -44,5 +49,40 @@ __all__ = [ 'attach_shm_array', 'open_shm_array', 'get_shm_token', - '_setup_persistent_brokerd', ] + + +@tractor.context +async def _setup_persistent_brokerd( + ctx: tractor.Context, + brokername: str, + +) -> None: + ''' + Allocate a actor-wide service nursery in ``brokerd`` + such that feeds can be run in the background persistently by + the broker backend as needed. + + ''' + get_console_log(tractor.current_actor().loglevel) + + from .feed import ( + _bus, + get_feed_bus, + ) + global _bus + assert not _bus + + async with trio.open_nursery() as service_nursery: + # assign a nursery to the feeds bus for spawning + # background tasks from clients + get_feed_bus(brokername, service_nursery) + + # unblock caller + await ctx.started() + + # we pin this task to keep the feeds manager active until the + # parent actor decides to tear it down + await trio.sleep_forever() + + diff --git a/piker/data/feed.py b/piker/data/feed.py index 88e9ceed..b714c77e 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -54,7 +54,10 @@ import numpy as np from ..brokers import get_brokermod from ..calc import humanize -from ..log import get_logger, get_console_log +from ..log import ( + get_logger, + get_console_log, +) from .._daemon import ( maybe_spawn_brokerd, check_for_service, @@ -224,36 +227,6 @@ def get_feed_bus( return _bus -@tractor.context -async def _setup_persistent_brokerd( - ctx: tractor.Context, - brokername: str, - -) -> None: - ''' - Allocate a actor-wide service nursery in ``brokerd`` - such that feeds can be run in the background persistently by - the broker backend as needed. - - ''' - get_console_log(tractor.current_actor().loglevel) - - global _bus - assert not _bus - - async with trio.open_nursery() as service_nursery: - # assign a nursery to the feeds bus for spawning - # background tasks from clients - get_feed_bus(brokername, service_nursery) - - # unblock caller - await ctx.started() - - # we pin this task to keep the feeds manager active until the - # parent actor decides to tear it down - await trio.sleep_forever() - - def diff_history( array: np.ndarray, timeframe: int, From d9b73e1d0809c6b82d4e05dc8471325a129fdd1b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 24 Jan 2023 15:11:52 -0500 Subject: [PATCH 49/54] Yield services (manager) from `maybe_open_pikerd()` --- piker/_daemon.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/piker/_daemon.py b/piker/_daemon.py index 0369f3f9..b9c74853 100644 --- a/piker/_daemon.py +++ b/piker/_daemon.py @@ -22,9 +22,9 @@ from __future__ import annotations import os from typing import ( Optional, - Union, Callable, Any, + ClassVar, ) from contextlib import ( asynccontextmanager as acm, @@ -393,7 +393,7 @@ async def maybe_open_pikerd( **kwargs, -) -> Union[tractor._portal.Portal, Services]: +) -> tractor._portal.Portal | ClassVar[Services]: ''' If no ``pikerd`` daemon-root-actor can be found start it and yield up (we should probably figure out returning a portal to self @@ -445,7 +445,7 @@ async def maybe_open_pikerd( # tractor-piker runtime stack in **this** process # we return no portal to self. assert service_manager - yield None + yield service_manager # `brokerd` enabled modules From 61e20a86cc2ca1dd896c9c2312f3fd038aee607e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 24 Jan 2023 15:12:32 -0500 Subject: [PATCH 50/54] Fix clearing endpoint type annots, export `open_ems()` --- piker/clearing/__init__.py | 6 ++++++ piker/clearing/_client.py | 30 +++++++++++++++++++++++------- piker/clearing/_ems.py | 12 ++++++++++-- 3 files changed, 39 insertions(+), 9 deletions(-) diff --git a/piker/clearing/__init__.py b/piker/clearing/__init__.py index c4fc2647..06a9212e 100644 --- a/piker/clearing/__init__.py +++ b/piker/clearing/__init__.py @@ -18,3 +18,9 @@ Market machinery for order executions, book, management. """ +from ._client import open_ems + + +__all__ = [ + 'open_ems', +] diff --git a/piker/clearing/_client.py b/piker/clearing/_client.py index 03fb62d3..f3b26cbe 100644 --- a/piker/clearing/_client.py +++ b/piker/clearing/_client.py @@ -18,8 +18,10 @@ Orders and execution client API. """ +from __future__ import annotations from contextlib import asynccontextmanager as acm from pprint import pformat +from typing import TYPE_CHECKING import trio import tractor @@ -27,11 +29,16 @@ from tractor.trionics import broadcast_receiver from ..log import get_logger from ..data.types import Struct -from ._ems import _emsd_main from .._daemon import maybe_open_emsd from ._messages import Order, Cancel from ..brokers import get_brokermod +if TYPE_CHECKING: + from ._messages import ( + BrokerdPosition, + Status, + ) + log = get_logger(__name__) @@ -167,12 +174,19 @@ async def relay_order_cmds_from_sync_code( @acm async def open_ems( fqsn: str, + mode: str = 'live', -) -> ( +) -> tuple[ OrderBook, tractor.MsgStream, - dict, -): + dict[ + # brokername, acctid + tuple[str, str], + list[BrokerdPosition], + ], + list[str], + dict[str, Status], +]: ''' Spawn an EMS daemon and begin sending orders and receiving alerts. @@ -213,14 +227,16 @@ async def open_ems( from ..data._source import unpack_fqsn broker, symbol, suffix = unpack_fqsn(fqsn) - mode: str = 'live' - async with maybe_open_emsd(broker) as portal: mod = get_brokermod(broker) - if not getattr(mod, 'trades_dialogue', None): + if ( + not getattr(mod, 'trades_dialogue', None) + or mode == 'paper' + ): mode = 'paper' + from ._ems import _emsd_main async with ( # connect to emsd portal.open_context( diff --git a/piker/clearing/_ems.py b/piker/clearing/_ems.py index 8d6ba868..ba33e584 100644 --- a/piker/clearing/_ems.py +++ b/piker/clearing/_ems.py @@ -418,7 +418,7 @@ class Router(Struct): # load the paper trading engine exec_mode = 'paper' - log.warning(f'Entering paper trading mode for {broker}') + log.info(f'{broker}: Entering `paper` trading mode') # load the paper trading engine as a subactor of this emsd # actor to simulate the real IPC load it'll have when also @@ -1367,7 +1367,15 @@ async def _emsd_main( exec_mode: str, # ('paper', 'live') loglevel: str = 'info', -) -> None: +) -> tuple[ + dict[ + # brokername, acctid + tuple[str, str], + list[BrokerdPosition], + ], + list[str], + dict[str, Status], +]: ''' EMS (sub)actor entrypoint providing the execution management (micro)service which conducts broker order clearing control on From 7cfd431a2bd38708d37739d20af94a5739b9aca4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 24 Jan 2023 15:13:24 -0500 Subject: [PATCH 51/54] Yield `Services` in `open_test_pikerd()` fixture --- tests/conftest.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/conftest.py b/tests/conftest.py index d781cf61..2cfaad7a 100644 --- a/tests/conftest.py +++ b/tests/conftest.py @@ -7,6 +7,9 @@ from piker import ( # log, config, ) +from piker._daemon import ( + Services, +) def pytest_addoption(parser): @@ -142,10 +145,10 @@ async def _open_test_pikerd( maybe_open_pikerd( registry_addr=reg_addr, **kwargs, - ) as pikerd_portal, + ) as service_manager, ): # this proc/actor is the pikerd - assert pikerd_portal is None + assert service_manager is Services async with tractor.wait_for_actor( 'pikerd', @@ -157,6 +160,7 @@ async def _open_test_pikerd( raddr[0], raddr[1], portal, + service_manager, ) From fe4fb37b58bea28b8591746de079d7ba81878351 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 24 Jan 2023 15:14:09 -0500 Subject: [PATCH 52/54] Add service tree tests for data-feeds and the EMS --- tests/test_services.py | 141 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 140 insertions(+), 1 deletion(-) diff --git a/tests/test_services.py b/tests/test_services.py index 19d38b54..dd913c29 100644 --- a/tests/test_services.py +++ b/tests/test_services.py @@ -3,10 +3,31 @@ Actor tree daemon sub-service verifications ''' from typing import AsyncContextManager +from contextlib import asynccontextmanager as acm +import pytest import trio import tractor +from piker._daemon import ( + find_service, + check_for_service, + Services, +) +from piker.data import ( + open_feed, +) +from piker.clearing import ( + open_ems, +) +from piker.clearing._messages import ( + BrokerdPosition, + Status, +) +from piker.clearing._client import ( + OrderBook, +) + def test_runtime_boot( open_test_pikerd: AsyncContextManager @@ -20,11 +41,12 @@ def test_runtime_boot( async def main(): port = 6666 daemon_addr = ('127.0.0.1', port) + services: Services async with ( open_test_pikerd( reg_addr=daemon_addr, - ) as (_, _, pikerd_portal), + ) as (_, _, pikerd_portal, services), tractor.wait_for_actor( 'pikerd', @@ -35,3 +57,120 @@ def test_runtime_boot( assert pikerd_portal.channel.raddr == portal.channel.raddr trio.run(main) + + +@acm +async def ensure_service( + name: str, + sockaddr: tuple[str, int] | None = None, +) -> None: + async with find_service(name) as portal: + remote_sockaddr = portal.channel.raddr + print(f'FOUND `{name}` @ {remote_sockaddr}') + + if sockaddr: + assert remote_sockaddr == sockaddr + + yield portal + + +def test_ensure_datafeed_actors( + open_test_pikerd: AsyncContextManager + +) -> None: + ''' + Verify that booting a data feed starts a `brokerd` + actor and a singleton global `samplerd` and opening + an order mode in paper opens the `paperboi` service. + + ''' + actor_name: str = 'brokerd' + backend: str = 'kraken' + brokerd_name: str = f'{actor_name}.{backend}' + + async def main(): + async with ( + open_test_pikerd(), + open_feed( + ['xbtusdt.kraken'], + loglevel='info', + ) as feed + ): + # halt rt quote streams since we aren't testing them + await feed.pause() + + async with ( + ensure_service(brokerd_name), + ensure_service('samplerd'), + ): + pass + + trio.run(main) + + +def test_ensure_ems_in_paper_actors( + open_test_pikerd: AsyncContextManager + +) -> None: + + actor_name: str = 'brokerd' + backend: str = 'kraken' + brokerd_name: str = f'{actor_name}.{backend}' + + async def main(): + + # type declares + book: OrderBook + trades_stream: tractor.MsgStream + pps: dict[str, list[BrokerdPosition]] + accounts: list[str] + dialogs: dict[str, Status] + + # ensure we timeout after is startup is too slow. + # TODO: something like this should be our start point for + # benchmarking end-to-end startup B) + with trio.fail_after(7): + async with ( + open_test_pikerd() as (_, _, _, services), + + open_ems( + 'xbtusdt.kraken', + mode='paper', + ) as ( + book, + trades_stream, + pps, + accounts, + dialogs, + ), + ): + # there should be no on-going positions, + # TODO: though eventually we'll want to validate against + # local ledger and `pps.toml` state ;) + assert not pps + assert not dialogs + + pikerd_subservices = ['emsd', 'samplerd'] + + async with ( + ensure_service('emsd'), + ensure_service(brokerd_name), + ensure_service(f'paperboi.{backend}'), + ): + for name in pikerd_subservices: + assert name in services.service_tasks + + # brokerd.kraken actor should have been started + # implicitly by the ems. + assert brokerd_name in services.service_tasks + + print('ALL SERVICES STARTED, terminating..') + await services.cancel_service('emsd') + + with pytest.raises( + tractor._exceptions.ContextCancelled, + ) as exc_info: + trio.run(main) + + cancel_msg: str = '`_emsd_main()` was remotely cancelled by its caller' + assert cancel_msg in exc_info.value.args[0] From 69074f4fa5453eef061bd05e8b121ae30e7a619d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 26 Jan 2023 17:59:11 -0500 Subject: [PATCH 53/54] Bump up service tree spawn timeout a couple secs --- tests/test_services.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/test_services.py b/tests/test_services.py index dd913c29..936a426e 100644 --- a/tests/test_services.py +++ b/tests/test_services.py @@ -129,7 +129,7 @@ def test_ensure_ems_in_paper_actors( # ensure we timeout after is startup is too slow. # TODO: something like this should be our start point for # benchmarking end-to-end startup B) - with trio.fail_after(7): + with trio.fail_after(9): async with ( open_test_pikerd() as (_, _, _, services), From fcfc0f31f08ff5243e051e68609a4e367495c80c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 30 Jan 2023 11:45:29 -0500 Subject: [PATCH 54/54] Enable backpressure in an effort to prevent bootup overruns --- piker/data/_sampling.py | 3 +++ piker/data/feed.py | 23 +++++++++++++++++++++-- 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/piker/data/_sampling.py b/piker/data/_sampling.py index f1bbc500..a5df96cc 100644 --- a/piker/data/_sampling.py +++ b/piker/data/_sampling.py @@ -751,6 +751,9 @@ async def uniform_rate_send( ) -> None: + # try not to error-out on overruns of the subscribed (chart) client + stream._ctx._backpressure = True + # TODO: compute the approx overhead latency per cycle left_to_sleep = throttle_period = 1/rate - 0.000616 diff --git a/piker/data/feed.py b/piker/data/feed.py index b714c77e..534aebc9 100644 --- a/piker/data/feed.py +++ b/piker/data/feed.py @@ -283,8 +283,22 @@ async def start_backfill( if step_size_s == 60: inow = round(time.time()) - if (inow - times[-1]) > 60: - await tractor.breakpoint() + diff = inow - times[-1] + if abs(diff) > 60: + surr = array[-6:] + diff_in_mins = round(diff/60., ndigits=2) + log.warning( + f'STEP ERROR `{bfqsn}` for period {step_size_s}s:\n' + f'Off by `{diff}` seconds (or `{diff_in_mins}` mins)\n' + 'Surrounding 6 time stamps:\n' + f'{list(surr["time"])}\n' + 'Here is surrounding 6 samples:\n' + f'{surr}\nn' + ) + + # uncomment this for a hacker who wants to investigate + # this case manually.. + # await tractor.breakpoint() # frame's worth of sample-period-steps, in seconds frame_size_s = len(array) * step_size_s @@ -1104,6 +1118,10 @@ async def open_feed_bus( symbol. ''' + # ensure that a quote feed stream which is pushing too fast doesn't + # cause and overrun in the client. + ctx._backpressure = True + if loglevel is None: loglevel = tractor.current_actor().loglevel @@ -1215,6 +1233,7 @@ async def open_feed_bus( # a max ``tick_throttle`` instantaneous rate. send, recv = trio.open_memory_channel(2**10) + ctx._backpressure = False cs = await bus.start_task( uniform_rate_send, tick_throttle,