Pass display state table to interaction handlers

This took a teensie bit of reworking in some `.ui` modules
more or less in the following order of functional dependence:

- add a `Ctl-R` kb-binding to trigger a `Viz.reset_graphics()` in
  the kb-handler task `handle_viewmode_kb_inputs()`.
  - call the new method on all `Viz`s (& for all sample-rates) and
    `DisplayState` refs provided in a (new input)
    `dss: dict[str, DisplayState]` table, which was originally inite-ed
    from the multi-feed display loop (so orig in `.graphics_update_loop()`
    but now provided as an input to that func, see below..)
- `._interaction`: allow binding in `async_handler()` kwargs (`via
  a `functools.partial`) passed to `ChartView.open_async_input_handler()`
  such that arbitrary inputs to our kb+mouse handler funcs can accept
  "wtv we desire".
  - use ^ to bind in the aforementioned `dss` display-state table to
    said handlers!
- define the `dss` table (as mentioned) inside `._display.display_symbol_data()`
  and pass it into the update loop funcs as well as the newly augmented
  `.open_async_input_handler()` calls,
  - drop calling `chart.view.open_async_input_handler()` from the
    `.order_mode.open_order_mode()`'s enter block and instead factor it
    into the caller to support passing the `dss` table to the kb
    handlers.
  - comment out the original history update loop handling of forced `Viz`
    redraws entirely since we now have a manual method via `Ctl-R`.
  - now, just update the `._remote_ctl.dss: dict` with this table since
    we want to also provide rc for **all** loaded feeds, not just the
    currently shown one/set.
- docs, naming and typing tweaks to `._event.open_handlers()`
distribute_dis
Tyler Goodlet 2023-12-28 20:41:21 -05:00
parent a7ad50cf8f
commit c82ca812a8
4 changed files with 184 additions and 102 deletions

View File

@ -210,9 +210,9 @@ async def increment_history_view(
): ):
hist_chart: ChartPlotWidget = ds.hist_chart hist_chart: ChartPlotWidget = ds.hist_chart
hist_viz: Viz = ds.hist_viz hist_viz: Viz = ds.hist_viz
viz: Viz = ds.viz # viz: Viz = ds.viz
assert 'hist' in hist_viz.shm.token['shm_name'] assert 'hist' in hist_viz.shm.token['shm_name']
name: str = hist_viz.name # name: str = hist_viz.name
# TODO: seems this is more reliable at keeping the slow # TODO: seems this is more reliable at keeping the slow
# chart incremented in view more correctly? # chart incremented in view more correctly?
@ -225,7 +225,8 @@ async def increment_history_view(
# draw everything from scratch on first entry! # draw everything from scratch on first entry!
for curve_name, hist_viz in hist_chart._vizs.items(): for curve_name, hist_viz in hist_chart._vizs.items():
log.info(f'Forcing hard redraw -> {curve_name}') log.info(f'Forcing hard redraw -> {curve_name}')
hist_viz.update_graphics(force_redraw=True) hist_viz.reset_graphics()
# hist_viz.update_graphics(force_redraw=True)
async with open_sample_stream(1.) as min_istream: async with open_sample_stream(1.) as min_istream:
async for msg in min_istream: async for msg in min_istream:
@ -248,27 +249,27 @@ async def increment_history_view(
# - samplerd could emit the actual update range via # - samplerd could emit the actual update range via
# tuple and then we only enter the below block if that # tuple and then we only enter the below block if that
# range is detected as in-view? # range is detected as in-view?
match msg: # match msg:
case { # case {
'backfilling': (viz_name, timeframe), # 'backfilling': (viz_name, timeframe),
} if ( # } if (
viz_name == name # viz_name == name
): # ):
log.warning( # log.warning(
f'Forcing HARD REDRAW:\n' # f'Forcing HARD REDRAW:\n'
f'name: {name}\n' # f'name: {name}\n'
f'timeframe: {timeframe}\n' # f'timeframe: {timeframe}\n'
) # )
# TODO: only allow this when the data is IN VIEW! # # TODO: only allow this when the data is IN VIEW!
# also, we probably can do this more efficiently # # also, we probably can do this more efficiently
# / smarter by only redrawing the portion of the # # / smarter by only redrawing the portion of the
# path necessary? # # path necessary?
{ # {
60: hist_viz, # 60: hist_viz,
1: viz, # 1: viz,
}[timeframe].update_graphics( # }[timeframe].update_graphics(
force_redraw=True # force_redraw=True
) # )
# check if slow chart needs an x-domain shift and/or # check if slow chart needs an x-domain shift and/or
# y-range resize. # y-range resize.
@ -309,6 +310,7 @@ async def increment_history_view(
async def graphics_update_loop( async def graphics_update_loop(
dss: dict[str, DisplayState],
nurse: trio.Nursery, nurse: trio.Nursery,
godwidget: GodWidget, godwidget: GodWidget,
feed: Feed, feed: Feed,
@ -350,8 +352,6 @@ async def graphics_update_loop(
'i_last_slow_t': 0, # multiview-global slow (1m) step index 'i_last_slow_t': 0, # multiview-global slow (1m) step index
} }
dss: dict[str, DisplayState] = {}
for fqme, flume in feed.flumes.items(): for fqme, flume in feed.flumes.items():
ohlcv = flume.rt_shm ohlcv = flume.rt_shm
hist_ohlcv = flume.hist_shm hist_ohlcv = flume.hist_shm
@ -470,67 +470,68 @@ async def graphics_update_loop(
if ds.hist_vars['i_last'] < ds.hist_vars['i_last_append']: if ds.hist_vars['i_last'] < ds.hist_vars['i_last_append']:
await tractor.pause() await tractor.pause()
try: # try:
# XXX TODO: we need to do _dss UPDATE here so that when
# a feed-view is switched you can still remote annotate the
# prior view..
from . import _remote_ctl
_remote_ctl._dss = dss
# main real-time quotes update loop # XXX TODO: we need to do _dss UPDATE here so that when
stream: tractor.MsgStream # a feed-view is switched you can still remote annotate the
async with feed.open_multi_stream() as stream: # prior view..
assert stream from . import _remote_ctl
async for quotes in stream: _remote_ctl._dss.update(dss)
quote_period = time.time() - last_quote_s
quote_rate = round( # main real-time quotes update loop
1/quote_period, 1) if quote_period > 0 else float('inf') stream: tractor.MsgStream
async with feed.open_multi_stream() as stream:
# assert stream
async for quotes in stream:
quote_period = time.time() - last_quote_s
quote_rate = round(
1/quote_period, 1) if quote_period > 0 else float('inf')
if (
quote_period <= 1/_quote_throttle_rate
# in the absolute worst case we shouldn't see more then
# twice the expected throttle rate right!?
# and quote_rate >= _quote_throttle_rate * 2
and quote_rate >= display_rate
):
pass
# log.warning(f'High quote rate {mkt.fqme}: {quote_rate}')
last_quote_s: float = time.time()
for fqme, quote in quotes.items():
ds = dss[fqme]
ds.quotes = quote
rt_pi, hist_pi = pis[fqme]
# chart isn't active/shown so skip render cycle and
# pause feed(s)
if ( if (
quote_period <= 1/_quote_throttle_rate fast_chart.linked.isHidden()
or not rt_pi.isVisible()
# in the absolute worst case we shouldn't see more then
# twice the expected throttle rate right!?
# and quote_rate >= _quote_throttle_rate * 2
and quote_rate >= display_rate
): ):
pass print(f'{fqme} skipping update for HIDDEN CHART')
# log.warning(f'High quote rate {mkt.fqme}: {quote_rate}') fast_chart.pause_all_feeds()
continue
last_quote_s = time.time() ic = fast_chart.view._in_interact
if ic:
fast_chart.pause_all_feeds()
print(f'{fqme} PAUSING DURING INTERACTION')
await ic.wait()
fast_chart.resume_all_feeds()
for fqme, quote in quotes.items(): # sync call to update all graphics/UX components.
ds = dss[fqme] graphics_update_cycle(
ds.quotes = quote ds,
rt_pi, hist_pi = pis[fqme] quote,
)
# chart isn't active/shown so skip render cycle and # finally:
# pause feed(s) # # XXX: cancel any remote annotation control ctxs
if ( # _remote_ctl._dss = None
fast_chart.linked.isHidden() # for cid, (ctx, aids) in _remote_ctl._ctxs.items():
or not rt_pi.isVisible() # await ctx.cancel()
):
print(f'{fqme} skipping update for HIDDEN CHART')
fast_chart.pause_all_feeds()
continue
ic = fast_chart.view._in_interact
if ic:
fast_chart.pause_all_feeds()
print(f'{fqme} PAUSING DURING INTERACTION')
await ic.wait()
fast_chart.resume_all_feeds()
# sync call to update all graphics/UX components.
graphics_update_cycle(
ds,
quote,
)
finally:
# XXX: cancel any remote annotation control ctxs
_remote_ctl._dss = None
for cid, (ctx, aids) in _remote_ctl._ctxs.items():
await ctx.cancel()
def graphics_update_cycle( def graphics_update_cycle(
@ -1554,8 +1555,10 @@ async def display_symbol_data(
) )
# start update loop task # start update loop task
dss: dict[str, DisplayState] = {}
ln.start_soon( ln.start_soon(
graphics_update_loop, graphics_update_loop,
dss,
ln, ln,
godwidget, godwidget,
feed, feed,
@ -1569,15 +1572,31 @@ async def display_symbol_data(
order_ctl_fqme: str = fqmes[0] order_ctl_fqme: str = fqmes[0]
mode: OrderMode mode: OrderMode
async with ( async with (
open_order_mode( open_order_mode(
feed, feed,
godwidget, godwidget,
order_ctl_fqme, order_ctl_fqme,
order_mode_started, order_mode_started,
loglevel=loglevel loglevel=loglevel
) as mode ) as mode,
):
# TODO: maybe have these startup sooner before
# order mode fully boots? but we gotta,
# -[ ] decouple the order mode bindings until
# the mode has fully booted..
# -[ ] maybe do an Event to sync?
# start input handling for ``ChartView`` input
# (i.e. kb + mouse handling loops)
rt_chart.view.open_async_input_handler(
dss=dss,
),
hist_chart.view.open_async_input_handler(
dss=dss,
),
):
rt_linked.mode = mode rt_linked.mode = mode
rt_viz = rt_chart.get_viz(order_ctl_fqme) rt_viz = rt_chart.get_viz(order_ctl_fqme)

View File

@ -201,8 +201,8 @@ async def open_signal_handler(
async for args in recv: async for args in recv:
await async_handler(*args) await async_handler(*args)
async with trio.open_nursery() as n: async with trio.open_nursery() as tn:
n.start_soon(proxy_to_handler) tn.start_soon(proxy_to_handler)
async with send: async with send:
yield yield
@ -212,18 +212,48 @@ async def open_handlers(
source_widgets: list[QWidget], source_widgets: list[QWidget],
event_types: set[QEvent], event_types: set[QEvent],
async_handler: Callable[[QWidget, trio.abc.ReceiveChannel], None],
**kwargs, # NOTE: if you want to bind in additional kwargs to the handler
# pass in a `partial()` instead!
async_handler: Callable[
[QWidget, trio.abc.ReceiveChannel], # required handler args
None
],
# XXX: these are ONLY inputs available to the
# `open_event_stream()` event-relay to mem-chan factor above!
**open_ev_stream_kwargs,
) -> None: ) -> None:
'''
Connect and schedule an async handler function to receive an
arbitrary `QWidget`'s events with kb/mouse msgs repacked into
structs (see above) and shuttled over a mem-chan to the input
`async_handler` to allow interaction-IO processing from
a `trio` func-as-task.
'''
widget: QWidget
streams: list[trio.abc.ReceiveChannel]
async with ( async with (
trio.open_nursery() as n, trio.open_nursery() as tn,
gather_contexts([ gather_contexts([
open_event_stream(widget, event_types, **kwargs) open_event_stream(
widget,
event_types,
**open_ev_stream_kwargs,
)
for widget in source_widgets for widget in source_widgets
]) as streams, ]) as streams,
): ):
for widget, event_recv_stream in zip(source_widgets, streams): for widget, event_recv_stream in zip(
n.start_soon(async_handler, widget, event_recv_stream) source_widgets,
streams,
):
tn.start_soon(
async_handler,
widget,
event_recv_stream,
)
yield yield

View File

@ -23,6 +23,7 @@ from contextlib import (
asynccontextmanager, asynccontextmanager,
ExitStack, ExitStack,
) )
from functools import partial
import time import time
from typing import ( from typing import (
Callable, Callable,
@ -74,6 +75,7 @@ if TYPE_CHECKING:
) )
from ._dataviz import Viz from ._dataviz import Viz
from .order_mode import OrderMode from .order_mode import OrderMode
from ._display import DisplayState
log = get_logger(__name__) log = get_logger(__name__)
@ -102,6 +104,7 @@ async def handle_viewmode_kb_inputs(
view: ChartView, view: ChartView,
recv_chan: trio.abc.ReceiveChannel, recv_chan: trio.abc.ReceiveChannel,
dss: dict[str, DisplayState],
) -> None: ) -> None:
@ -177,17 +180,42 @@ async def handle_viewmode_kb_inputs(
Qt.Key_P, Qt.Key_P,
} }
): ):
import tractor
feed = order_mode.feed # noqa feed = order_mode.feed # noqa
chart = order_mode.chart # noqa chart = order_mode.chart # noqa
viz = chart.main_viz # noqa viz = chart.main_viz # noqa
vlm_chart = chart.linked.subplots['volume'] # noqa vlm_chart = chart.linked.subplots['volume'] # noqa
vlm_viz = vlm_chart.main_viz # noqa vlm_viz = vlm_chart.main_viz # noqa
dvlm_pi = vlm_chart._vizs['dolla_vlm'].plot # noqa dvlm_pi = vlm_chart._vizs['dolla_vlm'].plot # noqa
import tractor
await tractor.pause() await tractor.pause()
view.interact_graphics_cycle() view.interact_graphics_cycle()
# SEARCH MODE # # FORCE graphics reset-and-render of all currently
# shown data `Viz`s for the current chart app.
if (
ctrl
and key in {
Qt.Key_R,
}
):
fqme: str
ds: DisplayState
for fqme, ds in dss.items():
viz: Viz
for tf, viz in {
60: ds.hist_viz,
1: ds.viz,
}.items():
# TODO: only allow this when the data is IN VIEW!
# also, we probably can do this more efficiently
# / smarter by only redrawing the portion of the
# path necessary?
viz.reset_graphics()
# ------ - ------
# SEARCH MODE
# ------ - ------
# ctlr-<space>/<l> for "lookup", "search" -> open search tree # ctlr-<space>/<l> for "lookup", "search" -> open search tree
if ( if (
ctrl ctrl
@ -247,8 +275,10 @@ async def handle_viewmode_kb_inputs(
delta=-view.def_delta, delta=-view.def_delta,
) )
elif key == Qt.Key_R: elif (
not ctrl
and key == Qt.Key_R
):
# NOTE: seems that if we don't yield a Qt render # NOTE: seems that if we don't yield a Qt render
# cycle then the m4 downsampled curves will show here # cycle then the m4 downsampled curves will show here
# without another reset.. # without another reset..
@ -431,6 +461,7 @@ async def handle_viewmode_mouse(
view: ChartView, view: ChartView,
recv_chan: trio.abc.ReceiveChannel, recv_chan: trio.abc.ReceiveChannel,
dss: dict[str, DisplayState],
) -> None: ) -> None:
@ -567,6 +598,7 @@ class ChartView(ViewBox):
@asynccontextmanager @asynccontextmanager
async def open_async_input_handler( async def open_async_input_handler(
self, self,
**handler_kwargs,
) -> ChartView: ) -> ChartView:
@ -577,14 +609,20 @@ class ChartView(ViewBox):
QEvent.KeyPress, QEvent.KeyPress,
QEvent.KeyRelease, QEvent.KeyRelease,
}, },
async_handler=handle_viewmode_kb_inputs, async_handler=partial(
handle_viewmode_kb_inputs,
**handler_kwargs,
),
), ),
_event.open_handlers( _event.open_handlers(
[self], [self],
event_types={ event_types={
gs_mouse.GraphicsSceneMousePress, gs_mouse.GraphicsSceneMousePress,
}, },
async_handler=handle_viewmode_mouse, async_handler=partial(
handle_viewmode_mouse,
**handler_kwargs,
),
), ),
): ):
yield self yield self

View File

@ -930,13 +930,8 @@ async def open_order_mode(
msg, msg,
) )
# start async input handling for chart's view
async with ( async with (
# ``ChartView`` input async handler startup
chart.view.open_async_input_handler(),
hist_chart.view.open_async_input_handler(),
# pp pane kb inputs # pp pane kb inputs
open_form_input_handling( open_form_input_handling(
form, form,