Make the data feed layer "fqsn" aware
In order to support instruments with lifetimes (aka derivatives) we need generally need special symbol annotations which detail such meta data (such as `MNQ.GLOBEX.20220717` for daq futes). Further there is really no reason for the public api for this feed layer to care about getting a special "brokername" field since generally the data is coming directly from UIs (eg. search selection) so we might as well accept a fqsn (fully qualified symbol name) which includes the broker name; for now a suffix like `'.ib'`. We may change this schema (soon) but this at least gets us to a point where we expect the full name including broker/provider. An additional detail: for certain "generic" symbol names (like for futes) we will pull a so called "front contract" and map this to a specific fqsn underneath, so there is a double (cached) entry for that entry such that other consumers can use it the same way if desired. Some other machinery changes: - expect the `stream_quotes()` endpoint to deliver it's `.started()` msg almost immediately since we now need it deliver any fqsn asap (yes this means the ep should no longer wait on a "live" first quote and instead deliver what quote data it can right away. - expect the quotes ohlc sampler task to add in the broker name before broadcast to remote (actor) consumers since the backend isn't (yet) expected to do that add in itself. - obviously we start using all the new fqsn related `Symbol` apisfqsns
parent
e9d64ffee8
commit
8462ea8a28
|
@ -50,9 +50,8 @@ from ._sharedmem import (
|
||||||
from .ingest import get_ingestormod
|
from .ingest import get_ingestormod
|
||||||
from ._source import (
|
from ._source import (
|
||||||
base_iohlc_dtype,
|
base_iohlc_dtype,
|
||||||
mk_symbol,
|
|
||||||
Symbol,
|
Symbol,
|
||||||
mk_fqsn,
|
uncons_fqsn,
|
||||||
)
|
)
|
||||||
from ..ui import _search
|
from ..ui import _search
|
||||||
from ._sampling import (
|
from ._sampling import (
|
||||||
|
@ -192,7 +191,7 @@ async def _setup_persistent_brokerd(
|
||||||
async def manage_history(
|
async def manage_history(
|
||||||
mod: ModuleType,
|
mod: ModuleType,
|
||||||
bus: _FeedsBus,
|
bus: _FeedsBus,
|
||||||
symbol: str,
|
fqsn: str,
|
||||||
some_data_ready: trio.Event,
|
some_data_ready: trio.Event,
|
||||||
feed_is_live: trio.Event,
|
feed_is_live: trio.Event,
|
||||||
|
|
||||||
|
@ -206,8 +205,6 @@ async def manage_history(
|
||||||
buffer.
|
buffer.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
fqsn = mk_fqsn(mod.name, symbol)
|
|
||||||
|
|
||||||
# (maybe) allocate shm array for this broker/symbol which will
|
# (maybe) allocate shm array for this broker/symbol which will
|
||||||
# be used for fast near-term history capture and processing.
|
# be used for fast near-term history capture and processing.
|
||||||
shm, opened = maybe_open_shm_array(
|
shm, opened = maybe_open_shm_array(
|
||||||
|
@ -226,7 +223,7 @@ async def manage_history(
|
||||||
# start history backfill task ``backfill_bars()`` is
|
# start history backfill task ``backfill_bars()`` is
|
||||||
# a required backend func this must block until shm is
|
# a required backend func this must block until shm is
|
||||||
# filled with first set of ohlc bars
|
# filled with first set of ohlc bars
|
||||||
_ = await bus.nursery.start(mod.backfill_bars, symbol, shm)
|
_ = await bus.nursery.start(mod.backfill_bars, fqsn, shm)
|
||||||
|
|
||||||
# yield back after client connect with filled shm
|
# yield back after client connect with filled shm
|
||||||
task_status.started(shm)
|
task_status.started(shm)
|
||||||
|
@ -285,8 +282,6 @@ async def allocate_persistent_feed(
|
||||||
except ImportError:
|
except ImportError:
|
||||||
mod = get_ingestormod(brokername)
|
mod = get_ingestormod(brokername)
|
||||||
|
|
||||||
fqsn = mk_fqsn(brokername, symbol)
|
|
||||||
|
|
||||||
# mem chan handed to broker backend so it can push real-time
|
# mem chan handed to broker backend so it can push real-time
|
||||||
# quotes to this task for sampling and history storage (see below).
|
# quotes to this task for sampling and history storage (see below).
|
||||||
send, quote_stream = trio.open_memory_channel(10)
|
send, quote_stream = trio.open_memory_channel(10)
|
||||||
|
@ -295,28 +290,9 @@ async def allocate_persistent_feed(
|
||||||
some_data_ready = trio.Event()
|
some_data_ready = trio.Event()
|
||||||
feed_is_live = trio.Event()
|
feed_is_live = trio.Event()
|
||||||
|
|
||||||
# run 2 tasks:
|
|
||||||
# - a history loader / maintainer
|
|
||||||
# - a real-time streamer which consumers and sends new data to any
|
|
||||||
# consumers as well as writes to storage backends (as configured).
|
|
||||||
|
|
||||||
# XXX: neither of these will raise but will cause an inf hang due to:
|
|
||||||
# https://github.com/python-trio/trio/issues/2258
|
|
||||||
# bus.nursery.start_soon(
|
|
||||||
# await bus.start_task(
|
|
||||||
|
|
||||||
shm = await bus.nursery.start(
|
|
||||||
manage_history,
|
|
||||||
mod,
|
|
||||||
bus,
|
|
||||||
symbol,
|
|
||||||
some_data_ready,
|
|
||||||
feed_is_live,
|
|
||||||
)
|
|
||||||
|
|
||||||
# establish broker backend quote stream by calling
|
# establish broker backend quote stream by calling
|
||||||
# ``stream_quotes()``, which is a required broker backend endpoint.
|
# ``stream_quotes()``, which is a required broker backend endpoint.
|
||||||
init_msg, first_quotes = await bus.nursery.start(
|
init_msg, first_quote = await bus.nursery.start(
|
||||||
partial(
|
partial(
|
||||||
mod.stream_quotes,
|
mod.stream_quotes,
|
||||||
send_chan=send,
|
send_chan=send,
|
||||||
|
@ -325,11 +301,38 @@ async def allocate_persistent_feed(
|
||||||
loglevel=loglevel,
|
loglevel=loglevel,
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
# the broker-specific fully qualified symbol name
|
||||||
|
bfqsn = init_msg[symbol]['fqsn']
|
||||||
|
|
||||||
|
# HISTORY, run 2 tasks:
|
||||||
|
# - a history loader / maintainer
|
||||||
|
# - a real-time streamer which consumers and sends new data to any
|
||||||
|
# consumers as well as writes to storage backends (as configured).
|
||||||
|
|
||||||
|
# XXX: neither of these will raise but will cause an inf hang due to:
|
||||||
|
# https://github.com/python-trio/trio/issues/2258
|
||||||
|
# bus.nursery.start_soon(
|
||||||
|
# await bus.start_task(
|
||||||
|
shm = await bus.nursery.start(
|
||||||
|
manage_history,
|
||||||
|
mod,
|
||||||
|
bus,
|
||||||
|
bfqsn,
|
||||||
|
some_data_ready,
|
||||||
|
feed_is_live,
|
||||||
|
)
|
||||||
|
|
||||||
# we hand an IPC-msg compatible shm token to the caller so it
|
# we hand an IPC-msg compatible shm token to the caller so it
|
||||||
# can read directly from the memory which will be written by
|
# can read directly from the memory which will be written by
|
||||||
# this task.
|
# this task.
|
||||||
init_msg[symbol]['shm_token'] = shm.token
|
msg = init_msg[symbol]
|
||||||
|
msg['shm_token'] = shm.token
|
||||||
|
|
||||||
|
# true fqsn
|
||||||
|
fqsn = '.'.join((bfqsn, brokername))
|
||||||
|
|
||||||
|
# add a fqsn entry that includes the ``.<broker>`` suffix
|
||||||
|
init_msg[fqsn] = msg
|
||||||
|
|
||||||
# TODO: pretty sure we don't need this? why not just leave 1s as
|
# TODO: pretty sure we don't need this? why not just leave 1s as
|
||||||
# the fastest "sample period" since we'll probably always want that
|
# the fastest "sample period" since we'll probably always want that
|
||||||
|
@ -342,8 +345,22 @@ async def allocate_persistent_feed(
|
||||||
log.info(f'waiting on history to load: {fqsn}')
|
log.info(f'waiting on history to load: {fqsn}')
|
||||||
await some_data_ready.wait()
|
await some_data_ready.wait()
|
||||||
|
|
||||||
bus.feeds[symbol.lower()] = (init_msg, first_quotes)
|
# append ``.<broker>`` suffix to each quote symbol
|
||||||
task_status.started((init_msg, first_quotes))
|
bsym = symbol + f'.{brokername}'
|
||||||
|
generic_first_quotes = {
|
||||||
|
bsym: first_quote,
|
||||||
|
fqsn: first_quote,
|
||||||
|
}
|
||||||
|
|
||||||
|
bus.feeds[symbol] = bus.feeds[fqsn] = (
|
||||||
|
init_msg,
|
||||||
|
generic_first_quotes,
|
||||||
|
)
|
||||||
|
# for ambiguous names we simply apply the retreived
|
||||||
|
# feed to that name (for now).
|
||||||
|
|
||||||
|
# task_status.started((init_msg, generic_first_quotes))
|
||||||
|
task_status.started()
|
||||||
|
|
||||||
# backend will indicate when real-time quotes have begun.
|
# backend will indicate when real-time quotes have begun.
|
||||||
await feed_is_live.wait()
|
await feed_is_live.wait()
|
||||||
|
@ -358,10 +375,11 @@ async def allocate_persistent_feed(
|
||||||
bus,
|
bus,
|
||||||
shm,
|
shm,
|
||||||
quote_stream,
|
quote_stream,
|
||||||
|
brokername,
|
||||||
sum_tick_vlm
|
sum_tick_vlm
|
||||||
)
|
)
|
||||||
finally:
|
finally:
|
||||||
log.warning(f'{symbol}@{brokername} feed task terminated')
|
log.warning(f'{fqsn} feed task terminated')
|
||||||
|
|
||||||
|
|
||||||
@tractor.context
|
@tractor.context
|
||||||
|
@ -394,25 +412,16 @@ async def open_feed_bus(
|
||||||
assert 'brokerd' in tractor.current_actor().name
|
assert 'brokerd' in tractor.current_actor().name
|
||||||
|
|
||||||
bus = get_feed_bus(brokername)
|
bus = get_feed_bus(brokername)
|
||||||
bus._subscribers.setdefault(symbol, [])
|
|
||||||
fqsn = mk_fqsn(brokername, symbol)
|
|
||||||
|
|
||||||
entry = bus.feeds.get(symbol)
|
|
||||||
|
|
||||||
# if no cached feed for this symbol has been created for this
|
# if no cached feed for this symbol has been created for this
|
||||||
# brokerd yet, start persistent stream and shm writer task in
|
# brokerd yet, start persistent stream and shm writer task in
|
||||||
# service nursery
|
# service nursery
|
||||||
|
entry = bus.feeds.get(symbol)
|
||||||
if entry is None:
|
if entry is None:
|
||||||
if not start_stream:
|
# allocate a new actor-local stream bus which
|
||||||
raise RuntimeError(
|
# will persist for this `brokerd`.
|
||||||
f'No stream feed exists for {fqsn}?\n'
|
|
||||||
f'You may need a `brokerd` started first.'
|
|
||||||
)
|
|
||||||
|
|
||||||
# allocate a new actor-local stream bus which will persist for
|
|
||||||
# this `brokerd`.
|
|
||||||
async with bus.task_lock:
|
async with bus.task_lock:
|
||||||
init_msg, first_quotes = await bus.nursery.start(
|
await bus.nursery.start(
|
||||||
partial(
|
partial(
|
||||||
allocate_persistent_feed,
|
allocate_persistent_feed,
|
||||||
|
|
||||||
|
@ -434,9 +443,30 @@ async def open_feed_bus(
|
||||||
# subscriber
|
# subscriber
|
||||||
init_msg, first_quotes = bus.feeds[symbol]
|
init_msg, first_quotes = bus.feeds[symbol]
|
||||||
|
|
||||||
|
msg = init_msg[symbol]
|
||||||
|
bfqsn = msg['fqsn']
|
||||||
|
|
||||||
|
# true fqsn
|
||||||
|
fqsn = '.'.join([bfqsn, brokername])
|
||||||
|
assert fqsn in first_quotes
|
||||||
|
assert bus.feeds[fqsn]
|
||||||
|
|
||||||
|
# broker-ambiguous symbol (provided on cli - eg. mnq.globex.ib)
|
||||||
|
bsym = symbol + f'.{brokername}'
|
||||||
|
assert bsym in first_quotes
|
||||||
|
|
||||||
|
# we use the broker-specific fqsn (bfqsn) for
|
||||||
|
# the sampler subscription since the backend isn't (yet)
|
||||||
|
# expected to append it's own name to the fqsn, so we filter
|
||||||
|
# on keys which *do not* include that name (e.g .ib) .
|
||||||
|
bus._subscribers.setdefault(bfqsn, [])
|
||||||
|
|
||||||
# send this even to subscribers to existing feed?
|
# send this even to subscribers to existing feed?
|
||||||
# deliver initial info message a first quote asap
|
# deliver initial info message a first quote asap
|
||||||
await ctx.started((init_msg, first_quotes))
|
await ctx.started((
|
||||||
|
init_msg,
|
||||||
|
first_quotes,
|
||||||
|
))
|
||||||
|
|
||||||
if not start_stream:
|
if not start_stream:
|
||||||
log.warning(f'Not opening real-time stream for {fqsn}')
|
log.warning(f'Not opening real-time stream for {fqsn}')
|
||||||
|
@ -449,14 +479,12 @@ async def open_feed_bus(
|
||||||
# re-send to trigger display loop cycle (necessary especially
|
# re-send to trigger display loop cycle (necessary especially
|
||||||
# when the mkt is closed and no real-time messages are
|
# when the mkt is closed and no real-time messages are
|
||||||
# expected).
|
# expected).
|
||||||
await stream.send(first_quotes)
|
await stream.send({fqsn: first_quotes})
|
||||||
|
|
||||||
if tick_throttle:
|
|
||||||
|
|
||||||
# open a bg task which receives quotes over a mem chan
|
# open a bg task which receives quotes over a mem chan
|
||||||
# and only pushes them to the target actor-consumer at
|
# and only pushes them to the target actor-consumer at
|
||||||
# a max ``tick_throttle`` instantaneous rate.
|
# a max ``tick_throttle`` instantaneous rate.
|
||||||
|
if tick_throttle:
|
||||||
send, recv = trio.open_memory_channel(2**10)
|
send, recv = trio.open_memory_channel(2**10)
|
||||||
cs = await bus.start_task(
|
cs = await bus.start_task(
|
||||||
uniform_rate_send,
|
uniform_rate_send,
|
||||||
|
@ -469,12 +497,15 @@ async def open_feed_bus(
|
||||||
else:
|
else:
|
||||||
sub = (stream, tick_throttle)
|
sub = (stream, tick_throttle)
|
||||||
|
|
||||||
subs = bus._subscribers[symbol]
|
subs = bus._subscribers[bfqsn]
|
||||||
subs.append(sub)
|
subs.append(sub)
|
||||||
|
|
||||||
try:
|
try:
|
||||||
uid = ctx.chan.uid
|
uid = ctx.chan.uid
|
||||||
|
|
||||||
|
# ctrl protocol for start/stop of quote streams based on UI
|
||||||
|
# state (eg. don't need a stream when a symbol isn't being
|
||||||
|
# displayed).
|
||||||
async for msg in stream:
|
async for msg in stream:
|
||||||
|
|
||||||
if msg == 'pause':
|
if msg == 'pause':
|
||||||
|
@ -499,7 +530,7 @@ async def open_feed_bus(
|
||||||
# n.cancel_scope.cancel()
|
# n.cancel_scope.cancel()
|
||||||
cs.cancel()
|
cs.cancel()
|
||||||
try:
|
try:
|
||||||
bus._subscribers[symbol].remove(sub)
|
bus._subscribers[bfqsn].remove(sub)
|
||||||
except ValueError:
|
except ValueError:
|
||||||
log.warning(f'{sub} for {symbol} was already removed?')
|
log.warning(f'{sub} for {symbol} was already removed?')
|
||||||
|
|
||||||
|
@ -625,10 +656,10 @@ async def install_brokerd_search(
|
||||||
|
|
||||||
@asynccontextmanager
|
@asynccontextmanager
|
||||||
async def open_feed(
|
async def open_feed(
|
||||||
brokername: str,
|
|
||||||
symbols: list[str],
|
|
||||||
loglevel: Optional[str] = None,
|
|
||||||
|
|
||||||
|
fqsns: list[str],
|
||||||
|
|
||||||
|
loglevel: Optional[str] = None,
|
||||||
backpressure: bool = True,
|
backpressure: bool = True,
|
||||||
start_stream: bool = True,
|
start_stream: bool = True,
|
||||||
tick_throttle: Optional[float] = None, # Hz
|
tick_throttle: Optional[float] = None, # Hz
|
||||||
|
@ -638,7 +669,10 @@ async def open_feed(
|
||||||
Open a "data feed" which provides streamed real-time quotes.
|
Open a "data feed" which provides streamed real-time quotes.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
sym = symbols[0].lower()
|
fqsn = fqsns[0].lower()
|
||||||
|
|
||||||
|
brokername, key, suffix = uncons_fqsn(fqsn)
|
||||||
|
bfqsn = fqsn.replace('.' + brokername, '')
|
||||||
|
|
||||||
try:
|
try:
|
||||||
mod = get_brokermod(brokername)
|
mod = get_brokermod(brokername)
|
||||||
|
@ -659,7 +693,7 @@ async def open_feed(
|
||||||
portal.open_context(
|
portal.open_context(
|
||||||
open_feed_bus,
|
open_feed_bus,
|
||||||
brokername=brokername,
|
brokername=brokername,
|
||||||
symbol=sym,
|
symbol=bfqsn,
|
||||||
loglevel=loglevel,
|
loglevel=loglevel,
|
||||||
start_stream=start_stream,
|
start_stream=start_stream,
|
||||||
tick_throttle=tick_throttle,
|
tick_throttle=tick_throttle,
|
||||||
|
@ -676,9 +710,10 @@ async def open_feed(
|
||||||
):
|
):
|
||||||
# we can only read from shm
|
# we can only read from shm
|
||||||
shm = attach_shm_array(
|
shm = attach_shm_array(
|
||||||
token=init_msg[sym]['shm_token'],
|
token=init_msg[bfqsn]['shm_token'],
|
||||||
readonly=True,
|
readonly=True,
|
||||||
)
|
)
|
||||||
|
assert fqsn in first_quotes
|
||||||
|
|
||||||
feed = Feed(
|
feed = Feed(
|
||||||
name=brokername,
|
name=brokername,
|
||||||
|
@ -691,17 +726,15 @@ async def open_feed(
|
||||||
)
|
)
|
||||||
|
|
||||||
for sym, data in init_msg.items():
|
for sym, data in init_msg.items():
|
||||||
|
|
||||||
si = data['symbol_info']
|
si = data['symbol_info']
|
||||||
|
fqsn = data['fqsn'] + f'.{brokername}'
|
||||||
symbol = mk_symbol(
|
symbol = Symbol.from_fqsn(
|
||||||
key=sym,
|
fqsn,
|
||||||
type_key=si.get('asset_type', 'forex'),
|
info=si,
|
||||||
tick_size=si.get('price_tick_size', 0.01),
|
|
||||||
lot_tick_size=si.get('lot_tick_size', 0.0),
|
|
||||||
)
|
)
|
||||||
symbol.broker_info[brokername] = si
|
|
||||||
|
|
||||||
|
# symbol.broker_info[brokername] = si
|
||||||
|
feed.symbols[fqsn] = symbol
|
||||||
feed.symbols[sym] = symbol
|
feed.symbols[sym] = symbol
|
||||||
|
|
||||||
# cast shm dtype to list... can't member why we need this
|
# cast shm dtype to list... can't member why we need this
|
||||||
|
@ -725,8 +758,7 @@ async def open_feed(
|
||||||
@asynccontextmanager
|
@asynccontextmanager
|
||||||
async def maybe_open_feed(
|
async def maybe_open_feed(
|
||||||
|
|
||||||
brokername: str,
|
fqsns: list[str],
|
||||||
symbols: list[str],
|
|
||||||
loglevel: Optional[str] = None,
|
loglevel: Optional[str] = None,
|
||||||
|
|
||||||
**kwargs,
|
**kwargs,
|
||||||
|
@ -738,13 +770,12 @@ async def maybe_open_feed(
|
||||||
in a tractor broadcast receiver.
|
in a tractor broadcast receiver.
|
||||||
|
|
||||||
'''
|
'''
|
||||||
sym = symbols[0].lower()
|
fqsn = fqsns[0]
|
||||||
|
|
||||||
async with maybe_open_context(
|
async with maybe_open_context(
|
||||||
acm_func=open_feed,
|
acm_func=open_feed,
|
||||||
kwargs={
|
kwargs={
|
||||||
'brokername': brokername,
|
'fqsns': fqsns,
|
||||||
'symbols': [sym],
|
|
||||||
'loglevel': loglevel,
|
'loglevel': loglevel,
|
||||||
'tick_throttle': kwargs.get('tick_throttle'),
|
'tick_throttle': kwargs.get('tick_throttle'),
|
||||||
|
|
||||||
|
@ -752,11 +783,11 @@ async def maybe_open_feed(
|
||||||
'backpressure': kwargs.get('backpressure', True),
|
'backpressure': kwargs.get('backpressure', True),
|
||||||
'start_stream': kwargs.get('start_stream', True),
|
'start_stream': kwargs.get('start_stream', True),
|
||||||
},
|
},
|
||||||
key=sym,
|
key=fqsn,
|
||||||
) as (cache_hit, feed):
|
) as (cache_hit, feed):
|
||||||
|
|
||||||
if cache_hit:
|
if cache_hit:
|
||||||
log.info(f'Using cached feed for {brokername}.{sym}')
|
log.info(f'Using cached feed for {fqsn}')
|
||||||
# add a new broadcast subscription for the quote stream
|
# add a new broadcast subscription for the quote stream
|
||||||
# if this feed is likely already in use
|
# if this feed is likely already in use
|
||||||
async with feed.stream.subscribe() as bstream:
|
async with feed.stream.subscribe() as bstream:
|
||||||
|
|
Loading…
Reference in New Issue