From 00378c330c3266ed8851ae928691adc8647cc741 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 4 Jul 2022 22:00:56 -0400 Subject: [PATCH 01/48] First draft, working WS based order management Move to using the websocket API for all order control ops and dropping the sync rest api approach which resulted in a bunch of buggy races. Further this gets us must faster (batch) order cancellation for free and a simpler ems request handler loop. We now heavily leverage the new py3.10 `match:` syntax for all kraken-side API msg parsing and processing and handle both the `openOrders` and `ownTrades` subscription streams. We also block "order editing" (by immediate cancellation) for now since the EMS isn't entirely yet equipped to handle brokerd side `.reqid` changes (which is how kraken implements so called order "updates" or "edits") for a given order-request dialog and we may want to even consider just implementing "updates" ourselves via independent cancel and submit requests? Definitely something to ponder. Alternatively we can "masquerade" such updates behind the count-style `.oid` remapping we had to implement anyway (kraken's limitation) and maybe everything will just work? Further details in this patch: - create 2 tables for tracking the EMS's `.oid` (uui4) value to `int`s that kraken expects (for `reqid`s): `ids` and `reqmsgs` which enable local lookup of ems uids to piker-backend-client-side request ids and received order messages. - add `openOrders` sub support which more or less directly relays to equivalent `BrokerdStatus` updates and calc the `.filled` and `.remaining` values based on cleared vlm updates. - add handler blocks for `[add/edit/cancel]OrderStatus` events including error msg cases. - don't do any order request response processing in `handle_order_requests()` since responses are always received via one (or both?) of the new ws subs: `ownTrades` and `openOrders` and thus such msgs are now handled in the response relay loop. Relates to #290 Resolves #310, #296 --- piker/brokers/kraken/api.py | 1 + piker/brokers/kraken/broker.py | 463 +++++++++++++++++++++++---------- 2 files changed, 322 insertions(+), 142 deletions(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index 3abf533e..17b79027 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -282,6 +282,7 @@ class Client: "volume": str(size), } return await self.endpoint('AddOrder', data) + else: # Edit order data for kraken api data["txid"] = reqid diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 588a0924..1b5eb736 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -20,7 +20,7 @@ Order api and machinery ''' from contextlib import asynccontextmanager as acm from functools import partial -from itertools import chain +from itertools import chain, count from pprint import pformat import time from typing import ( @@ -31,6 +31,7 @@ from typing import ( # Union, ) +from bidict import bidict import pendulum import trio import tractor @@ -46,7 +47,6 @@ from piker.clearing._messages import ( BrokerdPosition, BrokerdStatus, ) -from piker.data.types import Struct from . import log from .api import ( Client, @@ -62,27 +62,25 @@ from .feed import ( ) -class Trade(Struct): - ''' - Trade class that helps parse and validate ownTrades stream - - ''' - reqid: str # kraken order transaction id - action: str # buy or sell - price: float # price of asset - size: float # vol of asset - broker_time: str # e.g GTC, GTD - - async def handle_order_requests( + ws: NoBsWs, client: Client, ems_order_stream: tractor.MsgStream, + token: str, + requests: dict[str, BrokerdOrder], + ids: bidict[str, int], ) -> None: + ''' + Process new order submission requests from the EMS + and deliver acks or errors. + ''' + # XXX: UGH, let's unify this.. with ``msgspec``. request_msg: dict order: BrokerdOrder + counter = count() async for request_msg in ems_order_stream: log.info( @@ -90,150 +88,105 @@ async def handle_order_requests( f'{pformat(request_msg)}' ) - action = request_msg['action'] + account = request_msg['account'] - if action in {'buy', 'sell'}: - - account = request_msg['account'] - if account != 'kraken.spot': - log.error( - 'This is a kraken account, \ - only a `kraken.spot` selection is valid' - ) - await ems_order_stream.send(BrokerdError( + if account != 'kraken.spot': + log.error( + 'This is a kraken account, \ + only a `kraken.spot` selection is valid' + ) + await ems_order_stream.send( + BrokerdError( oid=request_msg['oid'], symbol=request_msg['symbol'], - - # reason=f'Kraken only, No account found: `{account}` ?', reason=( 'Kraken only, order mode disabled due to ' 'https://github.com/pikers/piker/issues/299' ), + ) + ) + continue - )) - continue + action = request_msg['action'] + if action in {'buy', 'sell'}: # validate - order = BrokerdOrder(**request_msg) - # call our client api to submit the order - resp = await client.submit_limit( - symbol=order.symbol, - price=order.price, - action=order.action, - size=order.size, - reqid=order.reqid, - ) + msg = BrokerdOrder(**request_msg) - err = resp['error'] - if err: - oid = order.oid - log.error(f'Failed to submit order: {oid}') + # logic from old `Client.submit_limit()` + if msg.oid in ids: + ep = 'editOrder' + reqid = ids[msg.oid] # integer not txid + order = requests[msg.oid] + assert order.oid == msg.oid + extra = { + 'orderid': msg.reqid, # txid + } + + # XXX: TODO: get this working, but currently the EMS + # doesn't support changing order `.reqid` (in this case + # kraken changes them via a cancel and a new + # submission). So for now cancel and report the error. + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [msg.reqid], # should be txid from submission + }) + continue - await ems_order_stream.send( - BrokerdError( - oid=order.oid, - reqid=order.reqid, - symbol=order.symbol, - reason="Failed order submission", - broker_details=resp - ) - ) else: - # TODO: handle multiple orders (cancels?) - # txid is an array of strings - if order.reqid is None: - reqid = resp['result']['txid'][0] - else: - # update the internal pairing of oid to krakens - # txid with the new txid that is returned on edit - reqid = resp['result']['txid'] - - # deliver ack that order has been submitted to broker routing - await ems_order_stream.send( - BrokerdOrderAck( - - # ems order request id - oid=order.oid, - - # broker specific request id - reqid=reqid, - - # account the made the order - account=order.account - - ) + ep = 'addOrder' + reqid = next(counter) + ids[msg.oid] = reqid + log.debug( + f"GENERATED ORDER {reqid}\n" + f'{ids}' ) + extra = { + 'ordertype': 'limit', + 'type': msg.action, + } + + psym = msg.symbol.upper() + pair = f'{psym[:3]}/{psym[3:]}' + + # call ws api to submit the order: + # https://docs.kraken.com/websockets/#message-addOrder + await ws.send_msg({ + 'event': ep, + 'token': token, + + 'reqid': reqid, # remapped-to-int uid from ems + 'pair': pair, + 'price': str(msg.price), + 'volume': str(msg.size), + + # only ensures request is valid, nothing more + # validate: 'true', + + } | extra) elif action == 'cancel': + msg = BrokerdCancel(**request_msg) + assert msg.oid in requests + reqid = ids[msg.oid] - # Send order cancellation to kraken - resp = await client.submit_cancel( - reqid=msg.reqid - ) + # call ws api to cancel: + # https://docs.kraken.com/websockets/#message-cancelOrder + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [msg.reqid], # should be txid from submission + }) - # Check to make sure there was no error returned by - # the kraken endpoint. Assert one order was cancelled. - try: - result = resp['result'] - count = result['count'] + else: + log.error(f'Unknown order command: {request_msg}') - # check for 'error' key if we received no 'result' - except KeyError: - error = resp.get('error') - - await ems_order_stream.send( - BrokerdError( - oid=msg.oid, - reqid=msg.reqid, - symbol=msg.symbol, - reason="Failed order cancel", - broker_details=resp - ) - ) - - if not error: - raise BrokerError(f'Unknown order cancel response: {resp}') - - else: - if not count: # no orders were cancelled? - - # XXX: what exactly is this from and why would we care? - # there doesn't seem to be any docs here? - # https://docs.kraken.com/rest/#operation/cancelOrder - - # Check to make sure the cancellation is NOT pending, - # then send the confirmation to the ems order stream - pending = result.get('pending') - if pending: - log.error(f'Order {oid} cancel was not yet successful') - - await ems_order_stream.send( - BrokerdError( - oid=msg.oid, - reqid=msg.reqid, - symbol=msg.symbol, - # TODO: maybe figure out if pending - # cancels will eventually get cancelled - reason="Order cancel is still pending?", - broker_details=resp - ) - ) - - else: # order cancel success case. - - await ems_order_stream.send( - BrokerdStatus( - reqid=msg.reqid, - account=msg.account, - time_ns=time.time_ns(), - status='cancelled', - reason='Order cancelled', - broker_details={'name': 'kraken'} - ) - ) - else: - log.error(f'Unknown order command: {request_msg}') + # placehold for sanity checking in relay loop + requests[msg.oid] = msg @acm @@ -358,8 +311,21 @@ async def trades_dialogue( ) as ws, trio.open_nursery() as n, ): + reqmsgs: dict[str, BrokerdOrder] = {} + + # 2way map for ems ids to kraken int reqids.. + ids: bidict[str, int] = bidict() + # task for processing inbound requests from ems - n.start_soon(handle_order_requests, client, ems_stream) + n.start_soon( + handle_order_requests, + ws, + client, + ems_stream, + token, + reqmsgs, + ids, + ) count: int = 0 @@ -472,7 +438,7 @@ async def trades_dialogue( await ems_stream.send(pp_msg) case [ - trades_msgs, + order_msgs, 'openOrders', {'sequence': seq}, ]: @@ -481,11 +447,224 @@ async def trades_dialogue( # above: # https://github.com/pikers/piker/issues/293 # https://github.com/pikers/piker/issues/310 - log.info(f'Order update {seq}:{trades_msgs}') + log.info(f'Orders update {seq}:{order_msgs}') + + for order_msg in order_msgs: + log.info( + 'Order msg update:\n' + f'{pformat(order_msg)}' + ) + txid, update_msg = list(order_msg.items())[0] + match update_msg: + case { + 'status': status, + 'userref': reqid, + **rest, + + # 'avg_price': _, + # 'cost': _, + # 'descr': { + # 'close': None, + # 'leverage': None, + # 'order': descr, + # 'ordertype': 'limit', + # 'pair': 'XMR/EUR', + # 'price': '74.94000000', + # 'price2': '0.00000000', + # 'type': 'buy' + # }, + # 'expiretm': None, + # 'fee': '0.00000000', + # 'limitprice': '0.00000000', + # 'misc': '', + # 'oflags': 'fciq', + # 'opentm': '1656966131.337344', + # 'refid': None, + # 'starttm': None, + # 'stopprice': '0.00000000', + # 'timeinforce': 'GTC', + # 'vol': submit_vlm, # '13.34400854', + # 'vol_exec': exec_vlm, # 0.0000 + }: + ems_status = { + 'open': 'submitted', + 'closed': 'cancelled', + 'canceled': 'cancelled', + 'pending': 'pending', + }[status] + + submit_vlm = rest.get('vol', 0) + exec_vlm = rest.get('vol_exec', 0) + + # send BrokerdStatus messages for all + # order state updates + msg = BrokerdStatus( + + reqid=txid, + time_ns=time.time_ns(), # cuz why not + account=f'kraken.{acctid}', + + # everyone doin camel case.. + status=ems_status, # force lower case + + filled=exec_vlm, + reason='', # why held? + remaining=( + float(submit_vlm) + - + float(exec_vlm) + ), + + broker_details=dict( + {'name': 'kraken'}, **update_msg + ), + ) + await ems_stream.send(msg.dict()) + + case _: + log.warning( + 'Unknown orders msg:\n' + f'{txid}:{order_msg}' + ) + + case { + 'event': etype, + 'status': status, + 'errorMessage': errmsg, + 'reqid': reqid, + } if ( + etype in {'addOrderStatus', 'editOrderStatus'} + and status == 'error' + ): + log.error( + f'Failed to submit order {reqid}:\n' + f'{errmsg}' + ) + oid = ids.inverse[reqid] + order = reqmsgs[oid] + await ems_stream.send( + BrokerdError( + oid=oid, + # use old reqid in case it changed? + reqid=order.reqid, + symbol=order.symbol, + reason=f'Failed submit:\n{errmsg}', + broker_details=resp + ).dict() + ) + + # if we rx any error cancel the order again + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [order.reqid], # txid from submission + }) + + case { + 'event': 'addOrderStatus', + 'status': status, + 'reqid': reqid, # oid from ems side + + # NOTE: in the case of an edit request this is + # a new value! + 'txid': txid, + + 'descr': descr, # only on success? + # 'originaltxid': txid, # only on edits + # **rest, + }: + oid = ids.inverse[reqid] + order = reqmsgs[oid] + log.info( + f'Submitting order {oid}[{reqid}]:\n' + f'txid: {txid}\n' + f'{descr}' + ) + + # deliver ack immediately + await ems_stream.send( + BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=order.account, # piker account + ).dict() + ) + + case { + 'event': 'editOrderStatus', + 'status': status, + 'errorMessage': errmsg, + 'reqid': reqid, # oid from ems side + 'descr': descr, + + # NOTE: for edit request this is a new value + 'txid': txid, + 'originaltxid': origtxid, + # **rest, + }: + log.info( + f'Editting order {oid}[{reqid}]:\n' + f'txid: {origtxid} -> {txid}\n' + f'{descr}' + ) + # deliver another ack to update the ems-side + # `.reqid`. + oid = ids.inverse[reqid] + await ems_stream.send( + BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=order.account, # piker account + ).dict() + ) + + # successful cancellation + case { + "event": "cancelOrderStatus", + "status": "ok", + 'txid': txids, + 'reqid': reqid, + }: + # TODO: should we support "batch" acking of + # multiple cancels thus avoiding the below loop? + oid = ids.inverse[reqid] + msg = reqmsgs[oid] + + for txid in txids: + await ems_stream.send( + BrokerdStatus( + reqid=txid, + account=msg.account, + time_ns=time.time_ns(), + status='cancelled', + reason='Cancel success: {oid}@{txid}', + broker_details=resp, + ).dict() + ) + + # failed cancel + case { + "event": "cancelOrderStatus", + "status": "error", + "errorMessage": errmsg, + 'reqid': reqid, + }: + oid = ids.inverse[reqid] + msg = reqmsgs[oid] + + await ems_stream.send( + BrokerdError( + oid=oid, + reqid=msg.reqid, + symbol=msg.symbol, + reason=f'Failed order cancel {errmsg}', + broker_details=resp + ).dict() + ) case _: log.warning(f'Unhandled trades msg: {msg}') - await tractor.breakpoint() def norm_trade_records( From f79d9865a0e97cdd23e42f13e077b3bd9ba65ba1 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 4 Jul 2022 22:52:25 -0400 Subject: [PATCH 02/48] Use `match:` syntax in data feed subs processing --- piker/brokers/kraken/feed.py | 95 ++++++++++++++++++++++++------------ 1 file changed, 63 insertions(+), 32 deletions(-) diff --git a/piker/brokers/kraken/feed.py b/piker/brokers/kraken/feed.py index e52f49aa..4966db8a 100644 --- a/piker/brokers/kraken/feed.py +++ b/piker/brokers/kraken/feed.py @@ -117,9 +117,8 @@ async def stream_messages( too_slow_count = 0 continue - if isinstance(msg, dict): - if msg.get('event') == 'heartbeat': - + match msg: + case {'event': 'heartbeat'}: now = time.time() delay = now - last_hb last_hb = now @@ -130,11 +129,20 @@ async def stream_messages( continue - err = msg.get('errorMessage') - if err: - raise BrokerError(err) - else: - yield msg + case { + 'connectionID': _, + 'event': 'systemStatus', + 'status': 'online', + 'version': _, + } as msg: + log.info( + 'WS connection is up:\n' + f'{msg}' + ) + continue + + case _: + yield msg async def process_data_feed_msgs( @@ -145,37 +153,60 @@ async def process_data_feed_msgs( ''' async for msg in stream_messages(ws): + match msg: + case { + 'errorMessage': errmsg + }: + raise BrokerError(errmsg) - chan_id, *payload_array, chan_name, pair = msg + case { + 'event': 'subscriptionStatus', + } as sub: + log.info( + 'WS subscription is active:\n' + f'{sub}' + ) + continue - if 'ohlc' in chan_name: + case [ + chan_id, + *payload_array, + chan_name, + pair + ]: + if 'ohlc' in chan_name: + yield 'ohlc', OHLC( + chan_id, + chan_name, + pair, + *payload_array[0] + ) - yield 'ohlc', OHLC(chan_id, chan_name, pair, *payload_array[0]) + elif 'spread' in chan_name: - elif 'spread' in chan_name: + bid, ask, ts, bsize, asize = map( + float, payload_array[0]) - bid, ask, ts, bsize, asize = map(float, payload_array[0]) + # TODO: really makes you think IB has a horrible API... + quote = { + 'symbol': pair.replace('/', ''), + 'ticks': [ + {'type': 'bid', 'price': bid, 'size': bsize}, + {'type': 'bsize', 'price': bid, 'size': bsize}, - # TODO: really makes you think IB has a horrible API... - quote = { - 'symbol': pair.replace('/', ''), - 'ticks': [ - {'type': 'bid', 'price': bid, 'size': bsize}, - {'type': 'bsize', 'price': bid, 'size': bsize}, + {'type': 'ask', 'price': ask, 'size': asize}, + {'type': 'asize', 'price': ask, 'size': asize}, + ], + } + yield 'l1', quote - {'type': 'ask', 'price': ask, 'size': asize}, - {'type': 'asize', 'price': ask, 'size': asize}, - ], - } - yield 'l1', quote + # elif 'book' in msg[-2]: + # chan_id, *payload_array, chan_name, pair = msg + # print(msg) - # elif 'book' in msg[-2]: - # chan_id, *payload_array, chan_name, pair = msg - # print(msg) - - else: - print(f'UNHANDLED MSG: {msg}') - yield msg + case _: + print(f'UNHANDLED MSG: {msg}') + # yield msg def normalize( @@ -385,7 +416,7 @@ async def stream_quotes( msg_gen = process_data_feed_msgs(ws) # TODO: use ``anext()`` when it lands in 3.10! - typ, ohlc_last = await msg_gen.__anext__() + typ, ohlc_last = await anext(msg_gen) topic, quote = normalize(ohlc_last) From ab08dc582db4ef833ac21d7a7de9a5916145b1b7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 09:06:53 -0400 Subject: [PATCH 03/48] Make ems relay loop report on brokerd `.reqid` changes --- piker/clearing/_ems.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/piker/clearing/_ems.py b/piker/clearing/_ems.py index 1a764812..57b044ad 100644 --- a/piker/clearing/_ems.py +++ b/piker/clearing/_ems.py @@ -88,7 +88,8 @@ def mk_check( @dataclass class _DarkBook: - '''EMS-trigger execution book. + ''' + EMS-trigger execution book. Contains conditions for executions (aka "orders" or "triggers") which are not exposed to brokers and thus the market; i.e. these are @@ -653,6 +654,13 @@ async def translate_and_relay_brokerd_events( else: # check for existing live flow entry entry = book._ems_entries.get(oid) + old_reqid = entry.reqid + + if old_reqid and old_reqid != reqid: + log.warning( + f'Brokerd order id change for {oid}:\n' + f'{old_reqid} -> {reqid}' + ) # initial response to brokerd order request if name == 'ack': @@ -663,6 +671,10 @@ async def translate_and_relay_brokerd_events( # a ``BrokerdOrderAck`` **must** be sent after an order # request in order to establish this id mapping. book._ems2brokerd_ids[oid] = reqid + log.info( + 'Rx ACK for order\n' + f'oid: {oid} -> reqid: {reqid}' + ) # new order which has not yet be registered into the # local ems book, insert it now and handle 2 cases: From df4cec930b182c5ddb58f4537ac8bca09345117a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 11:03:32 -0400 Subject: [PATCH 04/48] Get order "editing" working fully Turns out the EMS can support this as originally expected: you can update a `brokerd`-side `.reqid` through a `BrokerdAck` msg and the ems which update its cross-dialog (leg) tracking correctly! The issue was a bug in the `editOrderStatus` msg handling and appropriate tracking of the correct `.oid` (ems uid) on the kraken side. This unfortunately required adding a `emsflow: dict[str, list[BrokerdOrder]]` msg flow tracing table which means the broker daemon is tracking all the msg flow with the ems, though I'm wondering now if this is just good practise anyway and maybe we should offer a small primitive type from our msging utils to aid with this? I've used such constructs in event handling systems prior. There's a lot more factoring that can be done after these changes as well but the quick detailed summary is, - rework the `handle_order_requests()` loop to use `match:` syntax and update the new `emsflow` table on every new request from the ems. - fix the `editOrderStatus` case pattern to not include an error msg and thus actually be triggered to respond to the ems with a `BrokerdAck` containing the new `.reqid`, the new kraken side `txid`. - skip any `openOrders` msgs which are detected as being kraken's internal order "edits" by matching on the `cancel_reason` field. - update the `emsflow` table in all ws-stream msg handling blocks with responses sent to the ems. Relates to #290 --- piker/brokers/kraken/broker.py | 314 +++++++++++++++++---------------- 1 file changed, 166 insertions(+), 148 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 1b5eb736..c8f110b9 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -26,9 +26,8 @@ import time from typing import ( Any, AsyncIterator, - # Callable, # Optional, - # Union, + Union, ) from bidict import bidict @@ -61,6 +60,16 @@ from .feed import ( stream_messages, ) +MsgUnion = Union[ + BrokerdCancel, + BrokerdError, + BrokerdFill, + BrokerdOrder, + BrokerdOrderAck, + BrokerdPosition, + BrokerdStatus, +] + async def handle_order_requests( @@ -68,7 +77,7 @@ async def handle_order_requests( client: Client, ems_order_stream: tractor.MsgStream, token: str, - requests: dict[str, BrokerdOrder], + emsflow: dict[str, list[MsgUnion]], ids: bidict[str, int], ) -> None: @@ -78,115 +87,103 @@ async def handle_order_requests( ''' # XXX: UGH, let's unify this.. with ``msgspec``. - request_msg: dict + msg: dict[str, Any] order: BrokerdOrder counter = count() - async for request_msg in ems_order_stream: - log.info( - 'Received order request:\n' - f'{pformat(request_msg)}' - ) + async for msg in ems_order_stream: + log.info(f'Rx order msg:\n{pformat(msg)}') + match msg: + case { + 'account': 'kraken.spot', + 'action': action, + } if action in {'buy', 'sell'}: - account = request_msg['account'] + # validate + order = BrokerdOrder(**msg) - if account != 'kraken.spot': - log.error( - 'This is a kraken account, \ - only a `kraken.spot` selection is valid' - ) - await ems_order_stream.send( - BrokerdError( - oid=request_msg['oid'], - symbol=request_msg['symbol'], - reason=( - 'Kraken only, order mode disabled due to ' - 'https://github.com/pikers/piker/issues/299' - ), - ) - ) - continue + # logic from old `Client.submit_limit()` + if order.oid in ids: + ep = 'editOrder' + reqid = ids[order.oid] # integer not txid + last = emsflow[order.oid][-1] + assert last.reqid == order.reqid + extra = { + 'orderid': last.reqid, # txid + } - action = request_msg['action'] - if action in {'buy', 'sell'}: + else: + ep = 'addOrder' + reqid = next(counter) + ids[order.oid] = reqid + log.debug( + f"GENERATED ORDER {reqid}\n" + f'{ids}' + ) + extra = { + 'ordertype': 'limit', + 'type': order.action, + } - # validate - msg = BrokerdOrder(**request_msg) - - # logic from old `Client.submit_limit()` - if msg.oid in ids: - ep = 'editOrder' - reqid = ids[msg.oid] # integer not txid - order = requests[msg.oid] - assert order.oid == msg.oid - extra = { - 'orderid': msg.reqid, # txid - } - - # XXX: TODO: get this working, but currently the EMS - # doesn't support changing order `.reqid` (in this case - # kraken changes them via a cancel and a new - # submission). So for now cancel and report the error. - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid, - 'txid': [msg.reqid], # should be txid from submission - }) - continue - - else: - ep = 'addOrder' - reqid = next(counter) - ids[msg.oid] = reqid - log.debug( - f"GENERATED ORDER {reqid}\n" - f'{ids}' - ) - extra = { - 'ordertype': 'limit', - 'type': msg.action, - } - - psym = msg.symbol.upper() - pair = f'{psym[:3]}/{psym[3:]}' + psym = order.symbol.upper() + pair = f'{psym[:3]}/{psym[3:]}' # call ws api to submit the order: # https://docs.kraken.com/websockets/#message-addOrder - await ws.send_msg({ + req = { 'event': ep, 'token': token, 'reqid': reqid, # remapped-to-int uid from ems 'pair': pair, - 'price': str(msg.price), - 'volume': str(msg.size), + 'price': str(order.price), + 'volume': str(order.size), # only ensures request is valid, nothing more # validate: 'true', - } | extra) + } | extra + log.info(f'Submitting WS order request:\n{pformat(req)}') + await ws.send_msg(req) - elif action == 'cancel': + # placehold for sanity checking in relay loop + emsflow.setdefault(order.oid, []).append(order) - msg = BrokerdCancel(**request_msg) - assert msg.oid in requests - reqid = ids[msg.oid] + case { + 'account': 'kraken.spot', + 'action': 'cancel', + }: + cancel = BrokerdCancel(**msg) + assert cancel.oid in emsflow + reqid = ids[cancel.oid] - # call ws api to cancel: - # https://docs.kraken.com/websockets/#message-cancelOrder - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid, - 'txid': [msg.reqid], # should be txid from submission - }) + # call ws api to cancel: + # https://docs.kraken.com/websockets/#message-cancelOrder + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [cancel.reqid], # should be txid from submission + }) - else: - log.error(f'Unknown order command: {request_msg}') + case _: + account = msg.get('account') + if account != 'kraken.spot': + log.error( + 'This is a kraken account, \ + only a `kraken.spot` selection is valid' + ) - # placehold for sanity checking in relay loop - requests[msg.oid] = msg + await ems_order_stream.send( + BrokerdError( + oid=msg['oid'], + symbol=msg['symbol'], + reason=( + 'Invalid request msg:\n{msg}' + ), + + ).dict() + ) @acm @@ -311,7 +308,11 @@ async def trades_dialogue( ) as ws, trio.open_nursery() as n, ): - reqmsgs: dict[str, BrokerdOrder] = {} + # task local msg dialog tracking + emsflow: dict[ + str, + list[MsgUnion], + ] = {} # 2way map for ems ids to kraken int reqids.. ids: bidict[str, int] = bidict() @@ -323,7 +324,7 @@ async def trades_dialogue( client, ems_stream, token, - reqmsgs, + emsflow, ids, ) @@ -447,15 +448,24 @@ async def trades_dialogue( # above: # https://github.com/pikers/piker/issues/293 # https://github.com/pikers/piker/issues/310 - log.info(f'Orders update {seq}:{order_msgs}') - for order_msg in order_msgs: log.info( - 'Order msg update:\n' + 'Order msg update_{seq}:\n' f'{pformat(order_msg)}' ) txid, update_msg = list(order_msg.items())[0] match update_msg: + case { + 'cancel_reason': 'Order replaced', + 'status': status, + 'userref': reqid, + **rest, + }: + # we ignore internal order updates + # triggered by kraken's "edit" + # endpoint. + continue + case { 'status': status, 'userref': reqid, @@ -490,15 +500,20 @@ async def trades_dialogue( 'open': 'submitted', 'closed': 'cancelled', 'canceled': 'cancelled', + # do we even need to forward + # this state to the ems? 'pending': 'pending', }[status] submit_vlm = rest.get('vol', 0) exec_vlm = rest.get('vol_exec', 0) + oid = ids.inverse[reqid] + msgs = emsflow[oid] + # send BrokerdStatus messages for all # order state updates - msg = BrokerdStatus( + resp = BrokerdStatus( reqid=txid, time_ns=time.time_ns(), # cuz why not @@ -519,7 +534,8 @@ async def trades_dialogue( {'name': 'kraken'}, **update_msg ), ) - await ems_stream.send(msg.dict()) + msgs.append(resp) + await ems_stream.send(resp.dict()) case _: log.warning( @@ -537,28 +553,29 @@ async def trades_dialogue( and status == 'error' ): log.error( - f'Failed to submit order {reqid}:\n' + f'Failed to submit/edit order {reqid}:\n' f'{errmsg}' ) oid = ids.inverse[reqid] - order = reqmsgs[oid] - await ems_stream.send( - BrokerdError( - oid=oid, - # use old reqid in case it changed? - reqid=order.reqid, - symbol=order.symbol, - reason=f'Failed submit:\n{errmsg}', - broker_details=resp - ).dict() + msgs = emsflow[oid] + last = msgs[-1] + resp = BrokerdError( + oid=oid, + # use old reqid in case it changed? + reqid=last.reqid, + symbol=last.symbol, + reason=f'Failed submit:\n{errmsg}', + broker_details=resp ) + msgs.append(resp) + await ems_stream.send(resp.dict()) # if we rx any error cancel the order again await ws.send_msg({ 'event': 'cancelOrder', 'token': token, 'reqid': reqid, - 'txid': [order.reqid], # txid from submission + 'txid': [last.reqid], # txid from submission }) case { @@ -575,49 +592,48 @@ async def trades_dialogue( # **rest, }: oid = ids.inverse[reqid] - order = reqmsgs[oid] + msgs = emsflow[oid] + last = msgs[-1] log.info( - f'Submitting order {oid}[{reqid}]:\n' + f'Submitting order: {descr}\n' + f'ems oid: {oid}\n' + f're-mapped reqid: {reqid}\n' f'txid: {txid}\n' - f'{descr}' ) - - # deliver ack immediately - await ems_stream.send( - BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=order.account, # piker account - ).dict() + resp = BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=last.account, # piker account ) + msgs.append(resp) + await ems_stream.send(resp.dict()) case { 'event': 'editOrderStatus', 'status': status, - 'errorMessage': errmsg, 'reqid': reqid, # oid from ems side 'descr': descr, # NOTE: for edit request this is a new value 'txid': txid, 'originaltxid': origtxid, - # **rest, }: log.info( - f'Editting order {oid}[{reqid}]:\n' + f'Editting order {oid}[requid={reqid}]:\n' f'txid: {origtxid} -> {txid}\n' f'{descr}' ) - # deliver another ack to update the ems-side - # `.reqid`. + # deliver another ack to update the ems-side `.reqid`. oid = ids.inverse[reqid] - await ems_stream.send( - BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=order.account, # piker account - ).dict() + msgs = emsflow[oid] + last = msgs[-1] + resp = BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=last.account, # piker account ) + msgs.append(resp) + await ems_stream.send(resp.dict()) # successful cancellation case { @@ -629,19 +645,20 @@ async def trades_dialogue( # TODO: should we support "batch" acking of # multiple cancels thus avoiding the below loop? oid = ids.inverse[reqid] - msg = reqmsgs[oid] + msgs = emsflow[oid] + last = msgs[-1] for txid in txids: - await ems_stream.send( - BrokerdStatus( - reqid=txid, - account=msg.account, - time_ns=time.time_ns(), - status='cancelled', - reason='Cancel success: {oid}@{txid}', - broker_details=resp, - ).dict() + resp = BrokerdStatus( + reqid=txid, + account=last.account, + time_ns=time.time_ns(), + status='cancelled', + reason='Cancel success: {oid}@{txid}', + broker_details=resp, ) + msgs.append(resp) + await ems_stream.send(resp.dict()) # failed cancel case { @@ -651,17 +668,18 @@ async def trades_dialogue( 'reqid': reqid, }: oid = ids.inverse[reqid] - msg = reqmsgs[oid] + msgs = emsflow[oid] + last = msgs[-1] - await ems_stream.send( - BrokerdError( - oid=oid, - reqid=msg.reqid, - symbol=msg.symbol, - reason=f'Failed order cancel {errmsg}', - broker_details=resp - ).dict() + resp = BrokerdError( + oid=oid, + reqid=last.reqid, + symbol=last.symbol, + reason=f'Failed order cancel {errmsg}', + broker_details=resp ) + msgs.append(resp) + await ems_stream.send(resp.dict()) case _: log.warning(f'Unhandled trades msg: {msg}') From 84cab1327d85211c511411ad795129ead7b7fd42 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 11:25:40 -0400 Subject: [PATCH 05/48] Drop uneeded count-sequencec verification --- piker/brokers/kraken/broker.py | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index c8f110b9..5a248851 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -328,8 +328,6 @@ async def trades_dialogue( ids, ) - count: int = 0 - # process and relay trades events to ems # https://docs.kraken.com/websockets/#message-ownTrades async for msg in stream_messages(ws): @@ -339,11 +337,6 @@ async def trades_dialogue( 'ownTrades', {'sequence': seq}, ]: - # XXX: do we actually need this orrr? - # ensure that we are only processing new trades? - assert seq > count - count += 1 - # flatten msgs for processing trades = { tid: trade @@ -471,6 +464,7 @@ async def trades_dialogue( 'userref': reqid, **rest, + # XXX: eg. of remaining msg schema: # 'avg_price': _, # 'cost': _, # 'descr': { From d9b4c4a413c6326d4605990d6b340ce976cfaed4 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 11:48:10 -0400 Subject: [PATCH 06/48] Factor msg loop into new func: `handle_order_updates()` --- piker/brokers/kraken/broker.py | 674 +++++++++++++++++---------------- 1 file changed, 351 insertions(+), 323 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 5a248851..ece37c67 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -328,355 +328,383 @@ async def trades_dialogue( ids, ) - # process and relay trades events to ems + # enter relay loop + await handle_order_updates( + ws, + ems_stream, + emsflow, + ids, + trans, + acctid, + acc_name, + token, + ) + + +async def handle_order_updates( + ws: NoBsWs, + ems_stream: tractor.MsgStream, + emsflow: dict[str, list[MsgUnion]], + ids: bidict[str, int], + trans: list[pp.Transaction], + acctid: str, + acc_name: str, + token: str, + +) -> None: + ''' + Main msg handling loop for all things order management. + + This code is broken out to make the context explicit and state variables + defined in the signature clear to the reader. + + ''' + async for msg in stream_messages(ws): + match msg: + # process and relay clearing trade events to ems # https://docs.kraken.com/websockets/#message-ownTrades - async for msg in stream_messages(ws): - match msg: - case [ - trades_msgs, - 'ownTrades', - {'sequence': seq}, - ]: - # flatten msgs for processing - trades = { - tid: trade - for entry in trades_msgs - for (tid, trade) in entry.items() + case [ + trades_msgs, + 'ownTrades', + {'sequence': seq}, + ]: + # flatten msgs for processing + trades = { + tid: trade + for entry in trades_msgs + for (tid, trade) in entry.items() - # only emit entries which are already not-in-ledger - if tid not in {r.tid for r in trans} - } - for tid, trade in trades.items(): + # only emit entries which are already not-in-ledger + if tid not in {r.tid for r in trans} + } + for tid, trade in trades.items(): - # parse-cast - reqid = trade['ordertxid'] - action = trade['type'] - price = float(trade['price']) - size = float(trade['vol']) - broker_time = float(trade['time']) + # parse-cast + reqid = trade['ordertxid'] + action = trade['type'] + price = float(trade['price']) + size = float(trade['vol']) + broker_time = float(trade['time']) - # send a fill msg for gui update - fill_msg = BrokerdFill( - reqid=reqid, - time_ns=time.time_ns(), + # send a fill msg for gui update + fill_msg = BrokerdFill( + reqid=reqid, + time_ns=time.time_ns(), - action=action, - size=size, - price=price, - # TODO: maybe capture more msg data - # i.e fees? - broker_details={'name': 'kraken'}, - broker_time=broker_time - ) - await ems_stream.send(fill_msg) + action=action, + size=size, + price=price, + # TODO: maybe capture more msg data + # i.e fees? + broker_details={'name': 'kraken'}, + broker_time=broker_time + ) + await ems_stream.send(fill_msg) - filled_msg = BrokerdStatus( - reqid=reqid, - time_ns=time.time_ns(), + filled_msg = BrokerdStatus( + reqid=reqid, + time_ns=time.time_ns(), - account=acc_name, - status='filled', - filled=size, - reason='Order filled by kraken', - broker_details={ - 'name': 'kraken', - 'broker_time': broker_time - }, + account=acc_name, + status='filled', + filled=size, + reason='Order filled by kraken', + broker_details={ + 'name': 'kraken', + 'broker_time': broker_time + }, - # TODO: figure out if kraken gives a count - # of how many units of underlying were - # filled. Alternatively we can decrement - # this value ourselves by associating and - # calcing from the diff with the original - # client-side request, see: - # https://github.com/pikers/piker/issues/296 - remaining=0, - ) - await ems_stream.send(filled_msg) + # TODO: figure out if kraken gives a count + # of how many units of underlying were + # filled. Alternatively we can decrement + # this value ourselves by associating and + # calcing from the diff with the original + # client-side request, see: + # https://github.com/pikers/piker/issues/296 + remaining=0, + ) + await ems_stream.send(filled_msg) - # update ledger and position tracking - trans = await update_ledger(acctid, trades) - active, closed = pp.update_pps_conf( - 'kraken', - acctid, - trade_records=trans, - ledger_reload={}.fromkeys( - t.bsuid for t in trans), - ) + # update ledger and position tracking + trans = await update_ledger(acctid, trades) + active, closed = pp.update_pps_conf( + 'kraken', + acctid, + trade_records=trans, + ledger_reload={}.fromkeys( + t.bsuid for t in trans), + ) - # emit pp msgs - for pos in filter( - bool, - chain(active.values(), closed.values()), - ): - pp_msg = BrokerdPosition( - broker='kraken', + # emit any new pp msgs to ems + for pos in filter( + bool, + chain(active.values(), closed.values()), + ): + pp_msg = BrokerdPosition( + broker='kraken', - # XXX: ok so this is annoying, we're - # relaying an account name with the - # backend suffix prefixed but when - # reading accounts from ledgers we - # don't need it and/or it's prefixed - # in the section table.. we should - # just strip this from the message - # right since `.broker` is already - # included? - account=f'kraken.{acctid}', - symbol=pos.symbol.front_fqsn(), - size=pos.size, - avg_price=pos.be_price, + # XXX: ok so this is annoying, we're + # relaying an account name with the + # backend suffix prefixed but when + # reading accounts from ledgers we + # don't need it and/or it's prefixed + # in the section table.. we should + # just strip this from the message + # right since `.broker` is already + # included? + account=f'kraken.{acctid}', + symbol=pos.symbol.front_fqsn(), + size=pos.size, + avg_price=pos.be_price, - # TODO - # currency='' - ) - await ems_stream.send(pp_msg) + # TODO + # currency='' + ) + await ems_stream.send(pp_msg) - case [ - order_msgs, - 'openOrders', - {'sequence': seq}, - ]: - # TODO: async order update handling which we - # should remove from `handle_order_requests()` - # above: - # https://github.com/pikers/piker/issues/293 - # https://github.com/pikers/piker/issues/310 - for order_msg in order_msgs: - log.info( - 'Order msg update_{seq}:\n' - f'{pformat(order_msg)}' - ) - txid, update_msg = list(order_msg.items())[0] - match update_msg: - case { - 'cancel_reason': 'Order replaced', - 'status': status, - 'userref': reqid, - **rest, - }: - # we ignore internal order updates - # triggered by kraken's "edit" - # endpoint. - continue + # process and relay order state change events + # https://docs.kraken.com/websockets/#message-openOrders + case [ + order_msgs, + 'openOrders', + {'sequence': seq}, + ]: + for order_msg in order_msgs: + log.info( + f'Order msg update_{seq}:\n' + f'{pformat(order_msg)}' + ) + txid, update_msg = list(order_msg.items())[0] + match update_msg: + case { + 'cancel_reason': 'Order replaced', + 'status': status, + 'userref': reqid, + **rest, + }: + # we ignore internal order updates + # triggered by kraken's "edit" + # endpoint. + continue - case { - 'status': status, - 'userref': reqid, - **rest, + case { + 'status': status, + 'userref': reqid, + **rest, - # XXX: eg. of remaining msg schema: - # 'avg_price': _, - # 'cost': _, - # 'descr': { - # 'close': None, - # 'leverage': None, - # 'order': descr, - # 'ordertype': 'limit', - # 'pair': 'XMR/EUR', - # 'price': '74.94000000', - # 'price2': '0.00000000', - # 'type': 'buy' - # }, - # 'expiretm': None, - # 'fee': '0.00000000', - # 'limitprice': '0.00000000', - # 'misc': '', - # 'oflags': 'fciq', - # 'opentm': '1656966131.337344', - # 'refid': None, - # 'starttm': None, - # 'stopprice': '0.00000000', - # 'timeinforce': 'GTC', - # 'vol': submit_vlm, # '13.34400854', - # 'vol_exec': exec_vlm, # 0.0000 - }: - ems_status = { - 'open': 'submitted', - 'closed': 'cancelled', - 'canceled': 'cancelled', - # do we even need to forward - # this state to the ems? - 'pending': 'pending', - }[status] + # XXX: eg. of remaining msg schema: + # 'avg_price': _, + # 'cost': _, + # 'descr': { + # 'close': None, + # 'leverage': None, + # 'order': descr, + # 'ordertype': 'limit', + # 'pair': 'XMR/EUR', + # 'price': '74.94000000', + # 'price2': '0.00000000', + # 'type': 'buy' + # }, + # 'expiretm': None, + # 'fee': '0.00000000', + # 'limitprice': '0.00000000', + # 'misc': '', + # 'oflags': 'fciq', + # 'opentm': '1656966131.337344', + # 'refid': None, + # 'starttm': None, + # 'stopprice': '0.00000000', + # 'timeinforce': 'GTC', + # 'vol': submit_vlm, # '13.34400854', + # 'vol_exec': exec_vlm, # 0.0000 + }: + ems_status = { + 'open': 'submitted', + 'closed': 'cancelled', + 'canceled': 'cancelled', + # do we even need to forward + # this state to the ems? + 'pending': 'pending', + }[status] - submit_vlm = rest.get('vol', 0) - exec_vlm = rest.get('vol_exec', 0) + submit_vlm = rest.get('vol', 0) + exec_vlm = rest.get('vol_exec', 0) - oid = ids.inverse[reqid] - msgs = emsflow[oid] + oid = ids.inverse[reqid] + msgs = emsflow[oid] - # send BrokerdStatus messages for all - # order state updates - resp = BrokerdStatus( - - reqid=txid, - time_ns=time.time_ns(), # cuz why not - account=f'kraken.{acctid}', - - # everyone doin camel case.. - status=ems_status, # force lower case - - filled=exec_vlm, - reason='', # why held? - remaining=( - float(submit_vlm) - - - float(exec_vlm) - ), - - broker_details=dict( - {'name': 'kraken'}, **update_msg - ), - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - - case _: - log.warning( - 'Unknown orders msg:\n' - f'{txid}:{order_msg}' - ) - - case { - 'event': etype, - 'status': status, - 'errorMessage': errmsg, - 'reqid': reqid, - } if ( - etype in {'addOrderStatus', 'editOrderStatus'} - and status == 'error' - ): - log.error( - f'Failed to submit/edit order {reqid}:\n' - f'{errmsg}' - ) - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - resp = BrokerdError( - oid=oid, - # use old reqid in case it changed? - reqid=last.reqid, - symbol=last.symbol, - reason=f'Failed submit:\n{errmsg}', - broker_details=resp - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - - # if we rx any error cancel the order again - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid, - 'txid': [last.reqid], # txid from submission - }) - - case { - 'event': 'addOrderStatus', - 'status': status, - 'reqid': reqid, # oid from ems side - - # NOTE: in the case of an edit request this is - # a new value! - 'txid': txid, - - 'descr': descr, # only on success? - # 'originaltxid': txid, # only on edits - # **rest, - }: - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - log.info( - f'Submitting order: {descr}\n' - f'ems oid: {oid}\n' - f're-mapped reqid: {reqid}\n' - f'txid: {txid}\n' - ) - resp = BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=last.account, # piker account - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - - case { - 'event': 'editOrderStatus', - 'status': status, - 'reqid': reqid, # oid from ems side - 'descr': descr, - - # NOTE: for edit request this is a new value - 'txid': txid, - 'originaltxid': origtxid, - }: - log.info( - f'Editting order {oid}[requid={reqid}]:\n' - f'txid: {origtxid} -> {txid}\n' - f'{descr}' - ) - # deliver another ack to update the ems-side `.reqid`. - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - resp = BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=last.account, # piker account - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - - # successful cancellation - case { - "event": "cancelOrderStatus", - "status": "ok", - 'txid': txids, - 'reqid': reqid, - }: - # TODO: should we support "batch" acking of - # multiple cancels thus avoiding the below loop? - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - - for txid in txids: + # send BrokerdStatus messages for all + # order state updates resp = BrokerdStatus( + reqid=txid, - account=last.account, - time_ns=time.time_ns(), - status='cancelled', - reason='Cancel success: {oid}@{txid}', - broker_details=resp, + time_ns=time.time_ns(), # cuz why not + account=f'kraken.{acctid}', + + # everyone doin camel case.. + status=ems_status, # force lower case + + filled=exec_vlm, + reason='', # why held? + remaining=( + float(submit_vlm) + - + float(exec_vlm) + ), + + broker_details=dict( + {'name': 'kraken'}, **update_msg + ), ) msgs.append(resp) await ems_stream.send(resp.dict()) - # failed cancel - case { - "event": "cancelOrderStatus", - "status": "error", - "errorMessage": errmsg, - 'reqid': reqid, - }: - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] + case _: + log.warning( + 'Unknown orders msg:\n' + f'{txid}:{order_msg}' + ) - resp = BrokerdError( - oid=oid, - reqid=last.reqid, - symbol=last.symbol, - reason=f'Failed order cancel {errmsg}', - broker_details=resp - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) + case { + 'event': etype, + 'status': status, + 'errorMessage': errmsg, + 'reqid': reqid, + } if ( + etype in {'addOrderStatus', 'editOrderStatus'} + and status == 'error' + ): + log.error( + f'Failed to submit/edit order {reqid}:\n' + f'{errmsg}' + ) + oid = ids.inverse[reqid] + msgs = emsflow[oid] + last = msgs[-1] + resp = BrokerdError( + oid=oid, + # use old reqid in case it changed? + reqid=last.reqid, + symbol=last.symbol, + reason=f'Failed submit:\n{errmsg}', + broker_details=resp + ) + msgs.append(resp) + await ems_stream.send(resp.dict()) - case _: - log.warning(f'Unhandled trades msg: {msg}') + # if we rx any error cancel the order again + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [last.reqid], # txid from submission + }) + + case { + 'event': 'addOrderStatus', + 'status': status, + 'reqid': reqid, # oid from ems side + + # NOTE: in the case of an edit request this is + # a new value! + 'txid': txid, + + 'descr': descr, # only on success? + # 'originaltxid': txid, # only on edits + # **rest, + }: + oid = ids.inverse[reqid] + msgs = emsflow[oid] + last = msgs[-1] + log.info( + f'Submitting order: {descr}\n' + f'ems oid: {oid}\n' + f're-mapped reqid: {reqid}\n' + f'txid: {txid}\n' + ) + resp = BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=last.account, # piker account + ) + msgs.append(resp) + await ems_stream.send(resp.dict()) + + case { + 'event': 'editOrderStatus', + 'status': status, + 'reqid': reqid, # oid from ems side + 'descr': descr, + + # NOTE: for edit request this is a new value + 'txid': txid, + 'originaltxid': origtxid, + }: + log.info( + f'Editting order {oid}[requid={reqid}]:\n' + f'txid: {origtxid} -> {txid}\n' + f'{descr}' + ) + # deliver another ack to update the ems-side `.reqid`. + oid = ids.inverse[reqid] + msgs = emsflow[oid] + last = msgs[-1] + resp = BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=last.account, # piker account + ) + msgs.append(resp) + await ems_stream.send(resp.dict()) + + # successful cancellation + case { + "event": "cancelOrderStatus", + "status": "ok", + 'txid': txids, + 'reqid': reqid, + }: + # TODO: should we support "batch" acking of + # multiple cancels thus avoiding the below loop? + oid = ids.inverse[reqid] + msgs = emsflow[oid] + last = msgs[-1] + + for txid in txids: + resp = BrokerdStatus( + reqid=txid, + account=last.account, + time_ns=time.time_ns(), + status='cancelled', + reason='Cancel success: {oid}@{txid}', + broker_details=resp, + ) + msgs.append(resp) + await ems_stream.send(resp.dict()) + + # failed cancel + case { + "event": "cancelOrderStatus", + "status": "error", + "errorMessage": errmsg, + 'reqid': reqid, + }: + oid = ids.inverse[reqid] + msgs = emsflow[oid] + last = msgs[-1] + + resp = BrokerdError( + oid=oid, + reqid=last.reqid, + symbol=last.symbol, + reason=f'Failed order cancel {errmsg}', + broker_details=resp + ) + msgs.append(resp) + await ems_stream.send(resp.dict()) + + case _: + log.warning(f'Unhandled trades msg: {msg}') def norm_trade_records( From 9fa9c27e4d656cbf6f3dd3582900e771a55d99ae Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 12:58:08 -0400 Subject: [PATCH 07/48] Factor status handling into a new `process_status()` helper --- piker/brokers/kraken/broker.py | 269 +++++++++++++++++---------------- 1 file changed, 136 insertions(+), 133 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index ece37c67..128cd46a 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -26,7 +26,6 @@ import time from typing import ( Any, AsyncIterator, - # Optional, Union, ) @@ -476,15 +475,15 @@ async def handle_order_updates( ) txid, update_msg = list(order_msg.items())[0] match update_msg: + + # we ignore internal order updates triggered by + # kraken's "edit" endpoint. case { 'cancel_reason': 'Order replaced', 'status': status, 'userref': reqid, **rest, }: - # we ignore internal order updates - # triggered by kraken's "edit" - # endpoint. continue case { @@ -568,143 +567,146 @@ async def handle_order_updates( case { 'event': etype, 'status': status, - 'errorMessage': errmsg, 'reqid': reqid, - } if ( - etype in {'addOrderStatus', 'editOrderStatus'} - and status == 'error' + } as event if ( + etype in { + 'addOrderStatus', + 'editOrderStatus', + 'cancelOrderStatus', + } ): - log.error( - f'Failed to submit/edit order {reqid}:\n' - f'{errmsg}' - ) oid = ids.inverse[reqid] msgs = emsflow[oid] last = msgs[-1] - resp = BrokerdError( - oid=oid, - # use old reqid in case it changed? - reqid=last.reqid, - symbol=last.symbol, - reason=f'Failed submit:\n{errmsg}', - broker_details=resp + resps, errored = process_status( + event, + oid, + token, + msgs, + last, ) - msgs.append(resp) - await ems_stream.send(resp.dict()) + # if errored: + # if we rx any error cancel the order again + # await ws.send_msg({ + # 'event': 'cancelOrder', + # 'token': token, + # 'reqid': reqid, + # 'txid': [last.reqid], # txid from submission + # }) - # if we rx any error cancel the order again - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid, - 'txid': [last.reqid], # txid from submission - }) - - case { - 'event': 'addOrderStatus', - 'status': status, - 'reqid': reqid, # oid from ems side - - # NOTE: in the case of an edit request this is - # a new value! - 'txid': txid, - - 'descr': descr, # only on success? - # 'originaltxid': txid, # only on edits - # **rest, - }: - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - log.info( - f'Submitting order: {descr}\n' - f'ems oid: {oid}\n' - f're-mapped reqid: {reqid}\n' - f'txid: {txid}\n' - ) - resp = BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=last.account, # piker account - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - - case { - 'event': 'editOrderStatus', - 'status': status, - 'reqid': reqid, # oid from ems side - 'descr': descr, - - # NOTE: for edit request this is a new value - 'txid': txid, - 'originaltxid': origtxid, - }: - log.info( - f'Editting order {oid}[requid={reqid}]:\n' - f'txid: {origtxid} -> {txid}\n' - f'{descr}' - ) - # deliver another ack to update the ems-side `.reqid`. - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - resp = BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=last.account, # piker account - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - - # successful cancellation - case { - "event": "cancelOrderStatus", - "status": "ok", - 'txid': txids, - 'reqid': reqid, - }: - # TODO: should we support "batch" acking of - # multiple cancels thus avoiding the below loop? - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - - for txid in txids: - resp = BrokerdStatus( - reqid=txid, - account=last.account, - time_ns=time.time_ns(), - status='cancelled', - reason='Cancel success: {oid}@{txid}', - broker_details=resp, - ) - msgs.append(resp) + msgs.extend(resps) + for resp in resps: await ems_stream.send(resp.dict()) - # failed cancel - case { - "event": "cancelOrderStatus", - "status": "error", - "errorMessage": errmsg, - 'reqid': reqid, - }: - oid = ids.inverse[reqid] - msgs = emsflow[oid] - last = msgs[-1] - - resp = BrokerdError( - oid=oid, - reqid=last.reqid, - symbol=last.symbol, - reason=f'Failed order cancel {errmsg}', - broker_details=resp - ) - msgs.append(resp) - await ems_stream.send(resp.dict()) - case _: - log.warning(f'Unhandled trades msg: {msg}') + log.warning(f'Unhandled trades update msg: {msg}') + + +def process_status( + event: dict[str, str], + oid: str, + token: str, + msgs: list[MsgUnion], + last: MsgUnion, + +) -> tuple[list[MsgUnion], bool]: + ''' + Process `'[add/edit/cancel]OrderStatus'` events by translating to + and returning the equivalent EMS-msg responses. + + ''' + match event: + case { + 'event': etype, + 'status': 'error', + 'reqid': reqid, + 'errorMessage': errmsg, + }: + # any of ``{'add', 'edit', 'cancel'}`` + action = etype.rstrip('OrderStatus') + log.error( + f'Failed to {action} order {reqid}:\n' + f'{errmsg}' + ) + resp = BrokerdError( + oid=oid, + # XXX: use old reqid in case it changed? + reqid=last.reqid, + symbol=last.symbol, + + reason=f'Failed {action}:\n{errmsg}', + broker_details=event + ) + return [resp], True + + # successful request cases + case { + 'event': 'addOrderStatus', + 'status': "ok", + 'reqid': reqid, # oid from ems side + 'txid': txid, + 'descr': descr, # only on success? + }: + log.info( + f'Submitting order: {descr}\n' + f'ems oid: {oid}\n' + f're-mapped reqid: {reqid}\n' + f'txid: {txid}\n' + ) + resp = BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=last.account, # piker account + ) + return [resp], False + + case { + 'event': 'editOrderStatus', + 'status': "ok", + 'reqid': reqid, # oid from ems side + 'descr': descr, + + # NOTE: for edit request this is a new value + 'txid': txid, + 'originaltxid': origtxid, + }: + log.info( + f'Editting order {oid}[requid={reqid}]:\n' + f'txid: {origtxid} -> {txid}\n' + f'{descr}' + ) + # deliver another ack to update the ems-side `.reqid`. + resp = BrokerdOrderAck( + oid=oid, # ems order request id + reqid=txid, # kraken unique order id + account=last.account, # piker account + ) + return [resp], False + + case { + "event": "cancelOrderStatus", + "status": "ok", + 'reqid': reqid, + + # XXX: sometimes this isn't provided!? + # 'txid': txids, + **rest, + }: + # TODO: should we support "batch" acking of + # multiple cancels thus avoiding the below loop? + resps: list[MsgUnion] = [] + for txid in rest.get('txid', [last.reqid]): + resp = BrokerdStatus( + reqid=txid, + account=last.account, + time_ns=time.time_ns(), + status='cancelled', + reason='Cancel success: {oid}@{txid}', + broker_details=event, + ) + resps.append(resp) + + return resps, False def norm_trade_records( @@ -713,7 +715,6 @@ def norm_trade_records( ) -> list[pp.Transaction]: records: list[pp.Transaction] = [] - for tid, record in ledger.items(): size = record.get('vol') * { @@ -746,8 +747,10 @@ async def update_ledger( trade_entries: list[dict[str, Any]], ) -> list[pp.Transaction]: + ''' + Write recent session's trades to the user's (local) ledger file. - # write recent session's trades to the user's (local) ledger file. + ''' with pp.open_trade_ledger( 'kraken', acctid, From bbcc55b24c1befb1cae8cada834dee79d3c6b568 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 16:39:18 -0400 Subject: [PATCH 08/48] Update ledger *after* pps updates from new trades Addressing same issue as in #350 where we need to compute position updates using the *first read* from the ledger **before** we update it to make sure `Position.lifo_update()` gets called and **not skipped** because new trades were read as clears entries but haven't actually been included in update calcs yet.. aka we call `Position.lifo_update()`. Main change here is to convert `update_ledger()` into a context mngr so that the ledger write is committed after pps updates using `pp.update_pps_conf()`.. This is basically a hotfix to #346 as well. --- piker/brokers/kraken/broker.py | 48 +++++++++++++++++++--------------- 1 file changed, 27 insertions(+), 21 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 128cd46a..100561ae 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -18,7 +18,10 @@ Order api and machinery ''' -from contextlib import asynccontextmanager as acm +from contextlib import ( + asynccontextmanager as acm, + contextmanager as cm, +) from functools import partial from itertools import chain, count from pprint import pformat @@ -259,13 +262,13 @@ async def trades_dialogue( log.info( f'Loaded {len(trades)} trades from account `{acc_name}`' ) - trans = await update_ledger(acctid, trades) - active, closed = pp.update_pps_conf( - 'kraken', - acctid, - trade_records=trans, - ledger_reload={}.fromkeys(t.bsuid for t in trans), - ) + with open_ledger(acctid, trades) as trans: + active, closed = pp.update_pps_conf( + 'kraken', + acctid, + trade_records=trans, + ledger_reload={}.fromkeys(t.bsuid for t in trans), + ) position_msgs: list[dict] = [] pps: dict[int, pp.Position] @@ -425,14 +428,14 @@ async def handle_order_updates( await ems_stream.send(filled_msg) # update ledger and position tracking - trans = await update_ledger(acctid, trades) - active, closed = pp.update_pps_conf( - 'kraken', - acctid, - trade_records=trans, - ledger_reload={}.fromkeys( - t.bsuid for t in trans), - ) + with open_ledger(acctid, trades) as trans: + active, closed = pp.update_pps_conf( + 'kraken', + acctid, + trade_records=trans, + ledger_reload={}.fromkeys( + t.bsuid for t in trans), + ) # emit any new pp msgs to ems for pos in filter( @@ -742,7 +745,8 @@ def norm_trade_records( return records -async def update_ledger( +@cm +def open_ledger( acctid: str, trade_entries: list[dict[str, Any]], @@ -755,8 +759,10 @@ async def update_ledger( 'kraken', acctid, ) as ledger: - ledger.update(trade_entries) - # normalize to transaction form - records = norm_trade_records(trade_entries) - return records + # normalize to transaction form + records = norm_trade_records(trade_entries) + yield records + + # update on exit + ledger.update(trade_entries) From 8ab8268edc47b8c4f41fe5cf276057705cb9acd7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 17:35:17 -0400 Subject: [PATCH 09/48] Don't require an ems msg symbol on error statuses --- piker/brokers/kraken/broker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 100561ae..5ffe21b6 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -635,7 +635,7 @@ def process_status( oid=oid, # XXX: use old reqid in case it changed? reqid=last.reqid, - symbol=last.symbol, + symbol=getattr(last, 'symbol', 'N/A'), reason=f'Failed {action}:\n{errmsg}', broker_details=event From bb2f8e4304dec3ef8bcb5b778b52f81ae6768505 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 17:45:47 -0400 Subject: [PATCH 10/48] Try out a backend readme --- piker/brokers/kraken/README.rst | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 piker/brokers/kraken/README.rst diff --git a/piker/brokers/kraken/README.rst b/piker/brokers/kraken/README.rst new file mode 100644 index 00000000..7a801a94 --- /dev/null +++ b/piker/brokers/kraken/README.rst @@ -0,0 +1,31 @@ +``kraken`` backend +------------------ +though they don't have the most liquidity of all the cexes they sure are +accommodating to those of us who appreciate a little ``xmr``. + +status +****** +current support is *production grade* and both real-time data and order +management should be correct and fast. this backend is used by core devs +for live trading. + + +config +****** +In order to get order mode support your ``brokers.toml`` +needs to have something like the following: + +.. code:: toml + + [kraken] + accounts.spot = 'spot' + key_descr = "spot" + api_key = "69696969696969696696969696969696969696969696969696969696" + secret = "BOOBSBOOBSBOOBSBOOBSBOOBSSMBZ69696969696969669969696969696" + + +If everything works correctly you should see any current positions +loaded in the pps pane on chart load and you should also be able to +check your trade records in the file:: + + /ledgers/trades_kraken_spot.toml From 89bcaed15e160eac1d3523eb35f88a79b5aab38d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 5 Jul 2022 18:06:51 -0400 Subject: [PATCH 11/48] Add ledger and `pps.toml` snippets --- piker/brokers/kraken/README.rst | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/piker/brokers/kraken/README.rst b/piker/brokers/kraken/README.rst index 7a801a94..afeaeb2b 100644 --- a/piker/brokers/kraken/README.rst +++ b/piker/brokers/kraken/README.rst @@ -29,3 +29,36 @@ loaded in the pps pane on chart load and you should also be able to check your trade records in the file:: /ledgers/trades_kraken_spot.toml + + +An example ledger file will have entries written verbatim from the +trade events schema: + +.. code:: toml + + [TFJBKK-SMBZS-VJ4UWS] + ordertxid = "SMBZSA-7CNQU-3HWLNJ" + postxid = "SMBZSE-M7IF5-CFI7LT" + pair = "XXMRZEUR" + time = 1655691993.4133966 + type = "buy" + ordertype = "limit" + price = "103.97000000" + cost = "499.99999977" + fee = "0.80000000" + vol = "4.80907954" + margin = "0.00000000" + misc = "" + + +your ``pps.toml`` file will have position entries like, + +.. code:: toml + + [kraken.spot."xmreur.kraken"] + size = 4.80907954 + be_price = 103.97000000 + bsuid = "XXMRZEUR" + clears = [ + { tid = "TFJBKK-SMBZS-VJ4UWS", cost = 0.8, price = 103.97, size = 4.80907954, dt = "2022-05-20T02:26:33.413397+00:00" }, + ] From 804e9afdde1f75653378478db92473336618235c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Jul 2022 19:00:13 -0400 Subject: [PATCH 12/48] Pass our manually mapped `reqid: int` to EMS Since we seem to always be able to get back the `reqid`/`userref` value we send to kraken ws endpoints, we can use this as our brokerd side order id and avoid all race cases with getting the true `txid` value that `kraken` assigns (and which changes when you do "edits" :eyeroll:). This simplifies status updates by allowing our relay loop just to pass back our generated `.reqid` verbatim and allows responding with a `BrokerdOrderAck` immediately in the request handler task which should guarantee there are no further race conditions with the relay loop and mapping `txid`s from kraken.. and figuring out wtf to do when they change, etc. --- piker/brokers/kraken/broker.py | 51 +++++++++++++++++++--------------- 1 file changed, 28 insertions(+), 23 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 5ffe21b6..eceedf06 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -97,7 +97,7 @@ async def handle_order_requests( log.info(f'Rx order msg:\n{pformat(msg)}') match msg: case { - 'account': 'kraken.spot', + 'account': 'kraken.spot' as account, 'action': action, } if action in {'buy', 'sell'}: @@ -119,7 +119,7 @@ async def handle_order_requests( reqid = next(counter) ids[order.oid] = reqid log.debug( - f"GENERATED ORDER {reqid}\n" + f"Adding order {reqid}\n" f'{ids}' ) extra = { @@ -148,11 +148,18 @@ async def handle_order_requests( log.info(f'Submitting WS order request:\n{pformat(req)}') await ws.send_msg(req) + resp = BrokerdOrderAck( + oid=order.oid, # ems order request id + reqid=reqid, # our custom int mapping + account=account, # piker account + ) + await ems_order_stream.send(resp) + # placehold for sanity checking in relay loop emsflow.setdefault(order.oid, []).append(order) case { - 'account': 'kraken.spot', + 'account': 'kraken.spot' as account, 'action': 'cancel', }: cancel = BrokerdCancel(**msg) @@ -368,6 +375,8 @@ async def handle_order_updates( case [ trades_msgs, 'ownTrades', + # won't exist for historical values? + # 'userref': reqid, {'sequence': seq}, ]: # flatten msgs for processing @@ -381,8 +390,12 @@ async def handle_order_updates( } for tid, trade in trades.items(): - # parse-cast - reqid = trade['ordertxid'] + # NOTE: try to get the requid sent in the order + # request message if posssible; it may not be + # provided since this sub also returns generic + # historical trade events. + reqid = trade.get('userref', trade['ordertxid']) + action = trade['type'] price = float(trade['price']) size = float(trade['vol']) @@ -391,6 +404,7 @@ async def handle_order_updates( # send a fill msg for gui update fill_msg = BrokerdFill( reqid=reqid, + time_ns=time.time_ns(), action=action, @@ -539,7 +553,7 @@ async def handle_order_updates( # order state updates resp = BrokerdStatus( - reqid=txid, + reqid=reqid, time_ns=time.time_ns(), # cuz why not account=f'kraken.{acctid}', @@ -597,9 +611,10 @@ async def handle_order_updates( # 'txid': [last.reqid], # txid from submission # }) - msgs.extend(resps) - for resp in resps: - await ems_stream.send(resp.dict()) + if resps: + msgs.extend(resps) + for resp in resps: + await ems_stream.send(resp.dict()) case _: log.warning(f'Unhandled trades update msg: {msg}') @@ -634,7 +649,7 @@ def process_status( resp = BrokerdError( oid=oid, # XXX: use old reqid in case it changed? - reqid=last.reqid, + reqid=reqid, symbol=getattr(last, 'symbol', 'N/A'), reason=f'Failed {action}:\n{errmsg}', @@ -656,12 +671,7 @@ def process_status( f're-mapped reqid: {reqid}\n' f'txid: {txid}\n' ) - resp = BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=last.account, # piker account - ) - return [resp], False + return [], False case { 'event': 'editOrderStatus', @@ -679,12 +689,7 @@ def process_status( f'{descr}' ) # deliver another ack to update the ems-side `.reqid`. - resp = BrokerdOrderAck( - oid=oid, # ems order request id - reqid=txid, # kraken unique order id - account=last.account, # piker account - ) - return [resp], False + return [], False case { "event": "cancelOrderStatus", @@ -700,7 +705,7 @@ def process_status( resps: list[MsgUnion] = [] for txid in rest.get('txid', [last.reqid]): resp = BrokerdStatus( - reqid=txid, + reqid=reqid, account=last.account, time_ns=time.time_ns(), status='cancelled', From ba93f96c710cd618b533c9c8471a8b36fd31c724 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Thu, 7 Jul 2022 16:27:14 -0400 Subject: [PATCH 13/48] Lol, gotta `float()` that vlm before `*` XD --- piker/brokers/kraken/broker.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index eceedf06..dcc1dbe1 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -368,6 +368,10 @@ async def handle_order_updates( defined in the signature clear to the reader. ''' + # transaction records which will be updated + # on new trade clearing events (aka order "fills") + trans: list[pp.Transaction] + async for msg in stream_messages(ws): match msg: # process and relay clearing trade events to ems @@ -379,7 +383,7 @@ async def handle_order_updates( # 'userref': reqid, {'sequence': seq}, ]: - # flatten msgs for processing + # flatten msgs to an {id -> data} table for processing trades = { tid: trade for entry in trades_msgs @@ -725,7 +729,7 @@ def norm_trade_records( records: list[pp.Transaction] = [] for tid, record in ledger.items(): - size = record.get('vol') * { + size = float(record.get('vol')) * { 'buy': 1, 'sell': -1, }[record['type']] @@ -736,7 +740,7 @@ def norm_trade_records( pp.Transaction( fqsn=f'{norm_sym}.kraken', tid=tid, - size=float(size), + size=size, price=float(record['price']), cost=float(record['fee']), dt=pendulum.from_timestamp(float(record['time'])), From a20a8d95d5c7c4cd24bbd07e943b13de72aa476d Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Jul 2022 17:17:28 -0400 Subject: [PATCH 14/48] Use `aclosing()` around ws async gen --- piker/brokers/kraken/broker.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index dcc1dbe1..46fa9d78 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -32,6 +32,7 @@ from typing import ( Union, ) +from async_generator import aclosing from bidict import bidict import pendulum import trio @@ -316,6 +317,7 @@ async def trades_dialogue( ), ) as ws, trio.open_nursery() as n, + aclosing(stream_messages(ws)) as stream, ): # task local msg dialog tracking emsflow: dict[ @@ -339,7 +341,7 @@ async def trades_dialogue( # enter relay loop await handle_order_updates( - ws, + stream, ems_stream, emsflow, ids, @@ -351,7 +353,7 @@ async def trades_dialogue( async def handle_order_updates( - ws: NoBsWs, + ws_stream: NoBsWs, ems_stream: tractor.MsgStream, emsflow: dict[str, list[MsgUnion]], ids: bidict[str, int], @@ -372,7 +374,7 @@ async def handle_order_updates( # on new trade clearing events (aka order "fills") trans: list[pp.Transaction] - async for msg in stream_messages(ws): + async for msg in ws_stream: match msg: # process and relay clearing trade events to ems # https://docs.kraken.com/websockets/#message-ownTrades From caecbaa231c35040f563de9f30cb7a7c71b5bb98 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 8 Jul 2022 23:10:25 -0400 Subject: [PATCH 15/48] Cancel any live orders found on connect More or less just to avoid orders the user wasn't aware of from persisting until we get "open order relaying" through the ems working. Some further fixes which required a new `reqids2txids` map which keeps track of which `kraken` "txid" is mapped to our `reqid: int`; mainly this was needed for cancel requests which require knowing the underlying `txid`s (since apparently kraken doesn't keep track of the "reqid" we pass it). Pass the ws instance into `handle_order_updates()` to enable the cancelling orders on startup. Don't key error on unknown `reqid` values (for eg. when receiving historical trade events on startup). Handle cancel requests first in the ems side loop. --- piker/brokers/kraken/broker.py | 97 ++++++++++++++++++++++------------ 1 file changed, 64 insertions(+), 33 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 46fa9d78..44f82ad8 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -82,6 +82,7 @@ async def handle_order_requests( token: str, emsflow: dict[str, list[MsgUnion]], ids: bidict[str, int], + reqids2txids: dict[int, str], ) -> None: ''' @@ -97,6 +98,23 @@ async def handle_order_requests( async for msg in ems_order_stream: log.info(f'Rx order msg:\n{pformat(msg)}') match msg: + case { + 'action': 'cancel', + }: + cancel = BrokerdCancel(**msg) + last = emsflow[cancel.oid] + reqid = ids[cancel.oid] + txid = reqids2txids[reqid] + + # call ws api to cancel: + # https://docs.kraken.com/websockets/#message-cancelOrder + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [txid], # should be txid from submission + }) + case { 'account': 'kraken.spot' as account, 'action': action, @@ -109,10 +127,9 @@ async def handle_order_requests( if order.oid in ids: ep = 'editOrder' reqid = ids[order.oid] # integer not txid - last = emsflow[order.oid][-1] - assert last.reqid == order.reqid + txid = reqids2txids[reqid] extra = { - 'orderid': last.reqid, # txid + 'orderid': txid, # txid } else: @@ -159,23 +176,6 @@ async def handle_order_requests( # placehold for sanity checking in relay loop emsflow.setdefault(order.oid, []).append(order) - case { - 'account': 'kraken.spot' as account, - 'action': 'cancel', - }: - cancel = BrokerdCancel(**msg) - assert cancel.oid in emsflow - reqid = ids[cancel.oid] - - # call ws api to cancel: - # https://docs.kraken.com/websockets/#message-cancelOrder - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid, - 'txid': [cancel.reqid], # should be txid from submission - }) - case _: account = msg.get('account') if account != 'kraken.spot': @@ -327,6 +327,7 @@ async def trades_dialogue( # 2way map for ems ids to kraken int reqids.. ids: bidict[str, int] = bidict() + reqids2txids: dict[int, str] = {} # task for processing inbound requests from ems n.start_soon( @@ -337,14 +338,17 @@ async def trades_dialogue( token, emsflow, ids, + reqids2txids, ) # enter relay loop await handle_order_updates( + ws, stream, ems_stream, emsflow, ids, + reqids2txids, trans, acctid, acc_name, @@ -353,10 +357,12 @@ async def trades_dialogue( async def handle_order_updates( - ws_stream: NoBsWs, + ws: NoBsWs, + ws_stream: AsyncIterator, ems_stream: tractor.MsgStream, emsflow: dict[str, list[MsgUnion]], ids: bidict[str, int], + reqids2txids: dict[int, str], trans: list[pp.Transaction], acctid: str, acc_name: str, @@ -552,7 +558,29 @@ async def handle_order_updates( submit_vlm = rest.get('vol', 0) exec_vlm = rest.get('vol_exec', 0) - oid = ids.inverse[reqid] + reqids2txids[reqid] = txid + + oid = ids.inverse.get(reqid) + if not oid: + # TODO: handle these and relay them + # through the EMS to the client / UI + # side! + log.warning( + f'Received active order {txid}:\n' + f'{update_msg}\n' + 'Cancelling order for now!..' + ) + + # call ws api to cancel: + # https://docs.kraken.com/websockets/#message-cancelOrder + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [txid], + }) + continue + msgs = emsflow[oid] # send BrokerdStatus messages for all @@ -591,6 +619,7 @@ async def handle_order_updates( 'event': etype, 'status': status, 'reqid': reqid, + **rest, } as event if ( etype in { 'addOrderStatus', @@ -598,7 +627,18 @@ async def handle_order_updates( 'cancelOrderStatus', } ): - oid = ids.inverse[reqid] + oid = ids.inverse.get(reqid) + if not oid: + log.warning( + 'Unknown order status update?:\n' + f'{event}' + ) + continue + + txid = rest.get('txid') + if txid: + reqids2txids[reqid] = txid + msgs = emsflow[oid] last = msgs[-1] resps, errored = process_status( @@ -608,19 +648,10 @@ async def handle_order_updates( msgs, last, ) - # if errored: - # if we rx any error cancel the order again - # await ws.send_msg({ - # 'event': 'cancelOrder', - # 'token': token, - # 'reqid': reqid, - # 'txid': [last.reqid], # txid from submission - # }) - if resps: msgs.extend(resps) for resp in resps: - await ems_stream.send(resp.dict()) + await ems_stream.send(resp) case _: log.warning(f'Unhandled trades update msg: {msg}') From b0d3d9bb017ecda7064f681e7e69b9935fc15716 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 9 Jul 2022 12:59:09 -0400 Subject: [PATCH 16/48] TOSQUASH: lingering `.dict()`s --- piker/brokers/kraken/broker.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 44f82ad8..5cf50ade 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -192,7 +192,7 @@ async def handle_order_requests( 'Invalid request msg:\n{msg}' ), - ).dict() + ) ) @@ -607,7 +607,7 @@ async def handle_order_updates( ), ) msgs.append(resp) - await ems_stream.send(resp.dict()) + await ems_stream.send(resp) case _: log.warning( From 5dc9a61ec4e25a55bb6f6ac6d16cedb76f393b26 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 10 Jul 2022 15:44:52 -0400 Subject: [PATCH 17/48] Use cancel level logging for cancelled orders --- piker/clearing/_ems.py | 3 +++ piker/clearing/_messages.py | 1 + piker/ui/order_mode.py | 15 ++++++++++++--- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/piker/clearing/_ems.py b/piker/clearing/_ems.py index 57b044ad..5f08cfa5 100644 --- a/piker/clearing/_ems.py +++ b/piker/clearing/_ems.py @@ -702,6 +702,9 @@ async def translate_and_relay_brokerd_events( # a live flow now exists oid = entry.oid + # TODO: instead this should be our status set. + # ack, open, fill, closed, cancelled' + resp = None broker_details = {} diff --git a/piker/clearing/_messages.py b/piker/clearing/_messages.py index e5813c78..c30ada54 100644 --- a/piker/clearing/_messages.py +++ b/piker/clearing/_messages.py @@ -186,6 +186,7 @@ class BrokerdStatus(Struct): # XXX: should be best effort set for every update account: str = '' + # TODO: instead (ack, pending, open, fill, clos(ed), cancelled) # { # 'submitted', # 'cancelled', diff --git a/piker/ui/order_mode.py b/piker/ui/order_mode.py index ce08a64c..d7db1bee 100644 --- a/piker/ui/order_mode.py +++ b/piker/ui/order_mode.py @@ -843,16 +843,25 @@ async def process_trades_and_update_ui( # resp to 'cancel' request or error condition # for action request elif resp in ( - 'broker_cancelled', 'broker_inactive', 'broker_errored', + ): + # delete level line from view + mode.on_cancel(oid) + broker_msg = msg['brokerd_msg'] + log.error( + f'Order {oid}->{resp} with:\n{pformat(broker_msg)}' + ) + + elif resp in ( + 'broker_cancelled', 'dark_cancelled' ): # delete level line from view mode.on_cancel(oid) broker_msg = msg['brokerd_msg'] - log.warning( - f'Order {oid} failed with:\n{pformat(broker_msg)}' + log.cancel( + f'Order {oid}->{resp} with:\n{pformat(broker_msg)}' ) elif resp in ( From 57f2478dc7eefa272ba149ce033e24c952c8e0ea Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 10 Jul 2022 16:16:23 -0400 Subject: [PATCH 18/48] Fixes for state updates and clears Turns out the `openOrders` and `ownTrades` subs always return a `reqid` value (the one brokerd sends to the kraken api in order requests) is always set to zero, which seems to be a bug? So this includes patches to work around that as well reliance on the `openOrders` sub to do most `BrokerdStatus` updates since `XOrderStatus` events don't seem to have much data in them at all (they almost look like pure ack events so maybe they aren't affirmative of final state changes anyway..). Other fixes: - respond with a `BrokerdOrderAck` immediately after `requid` generation not after order submission to ensure the ems has a valid `requid` *before* kraken api events are relayed through. - add a `reqids2txids: bidict[int, str]` which maps brokerd genned `requid`s to kraken-side `txid`s since (as mentioned above) the clearing and state endpoints don't relay back this value (it's always 0...) - add log messages for each sub so that (at least for now) we can see exact msg contents coming from kraken. - drop `.remaining` calcs for now since we need to keep record of the order states manually in order to retreive the original submission vlm.. - fix the `openOrders` case for fills, in this case the message includes no `status` field and thus we must catch it in a block *after* the normal state handler to avoid masking. - drop response msg generation from the cancel status case since we can do it again from the `openOrders` handler and sending a double status causes issues on the client side. - add a shite ton of notes around all this missing `requid` stuff. --- piker/brokers/kraken/broker.py | 173 +++++++++++++++++++++++---------- 1 file changed, 124 insertions(+), 49 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 5cf50ade..f98d3314 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -102,7 +102,7 @@ async def handle_order_requests( 'action': 'cancel', }: cancel = BrokerdCancel(**msg) - last = emsflow[cancel.oid] + # last = emsflow[cancel.oid] reqid = ids[cancel.oid] txid = reqids2txids[reqid] @@ -148,6 +148,16 @@ async def handle_order_requests( psym = order.symbol.upper() pair = f'{psym[:3]}/{psym[3:]}' + # XXX: ACK the request **immediately** before sending + # the api side request to ensure the ems maps the oid -> + # reqid correctly! + resp = BrokerdOrderAck( + oid=order.oid, # ems order request id + reqid=reqid, # our custom int mapping + account=account, # piker account + ) + await ems_order_stream.send(resp) + # call ws api to submit the order: # https://docs.kraken.com/websockets/#message-addOrder req = { @@ -166,13 +176,6 @@ async def handle_order_requests( log.info(f'Submitting WS order request:\n{pformat(req)}') await ws.send_msg(req) - resp = BrokerdOrderAck( - oid=order.oid, # ems order request id - reqid=reqid, # our custom int mapping - account=account, # piker account - ) - await ems_order_stream.send(resp) - # placehold for sanity checking in relay loop emsflow.setdefault(order.oid, []).append(order) @@ -327,7 +330,7 @@ async def trades_dialogue( # 2way map for ems ids to kraken int reqids.. ids: bidict[str, int] = bidict() - reqids2txids: dict[int, str] = {} + reqids2txids: bidict[int, str] = bidict() # task for processing inbound requests from ems n.start_soon( @@ -362,7 +365,7 @@ async def handle_order_updates( ems_stream: tractor.MsgStream, emsflow: dict[str, list[MsgUnion]], ids: bidict[str, int], - reqids2txids: dict[int, str], + reqids2txids: bidict[int, str], trans: list[pp.Transaction], acctid: str, acc_name: str, @@ -384,13 +387,22 @@ async def handle_order_updates( match msg: # process and relay clearing trade events to ems # https://docs.kraken.com/websockets/#message-ownTrades + # TODO: turns out you get the fill events from the + # `openOrders` before you get this, so it might be better + # to do all fill/status/pp updates in that sub and just use + # this one for ledger syncs? + # For eg. we could take the "last 50 trades" and do a diff + # with the ledger and then only do a re-sync if something + # seems amiss? case [ trades_msgs, 'ownTrades', - # won't exist for historical values? - # 'userref': reqid, {'sequence': seq}, ]: + log.info( + f'ownTrades update_{seq}:\n' + f'{pformat(trades_msgs)}' + ) # flatten msgs to an {id -> data} table for processing trades = { tid: trade @@ -402,11 +414,13 @@ async def handle_order_updates( } for tid, trade in trades.items(): - # NOTE: try to get the requid sent in the order - # request message if posssible; it may not be - # provided since this sub also returns generic - # historical trade events. - reqid = trade.get('userref', trade['ordertxid']) + txid = trade['ordertxid'] + + # NOTE: yet again, here we don't have any ref to the + # reqid that's generated by us (as the client) and + # sent in the order request, so we have to look it + # up from our own registry... + reqid = reqids2txids.inverse[txid] action = trade['type'] price = float(trade['price']) @@ -415,16 +429,16 @@ async def handle_order_updates( # send a fill msg for gui update fill_msg = BrokerdFill( - reqid=reqid, - time_ns=time.time_ns(), + reqid=reqid, action=action, size=size, price=price, + # TODO: maybe capture more msg data # i.e fees? - broker_details={'name': 'kraken'}, + broker_details={'name': 'kraken'} | trade, broker_time=broker_time ) await ems_stream.send(fill_msg) @@ -455,6 +469,10 @@ async def handle_order_updates( # update ledger and position tracking with open_ledger(acctid, trades) as trans: + # TODO: ideally we can pass in an existingn + # pps state to this right? such that we + # don't have to do a ledger reload all the + # time.. active, closed = pp.update_pps_conf( 'kraken', acctid, @@ -499,7 +517,7 @@ async def handle_order_updates( ]: for order_msg in order_msgs: log.info( - f'Order msg update_{seq}:\n' + f'`openOrders` msg update_{seq}:\n' f'{pformat(order_msg)}' ) txid, update_msg = list(order_msg.items())[0] @@ -510,14 +528,22 @@ async def handle_order_updates( case { 'cancel_reason': 'Order replaced', 'status': status, - 'userref': reqid, - **rest, + # 'userref': reqid, # XXX: always zero bug XD + # **rest, }: + log.info( + f'Order {txid} was replaced' + ) continue case { + # XXX: lol, ws bug, this is always 0! + 'userref': _, + + # during a fill this field is **not** + # provided! but, it is always avail on + # actual status updates.. see case above. 'status': status, - 'userref': reqid, **rest, # XXX: eg. of remaining msg schema: @@ -548,17 +574,23 @@ async def handle_order_updates( }: ems_status = { 'open': 'submitted', - 'closed': 'cancelled', + 'closed': 'filled', 'canceled': 'cancelled', # do we even need to forward # this state to the ems? 'pending': 'pending', }[status] - submit_vlm = rest.get('vol', 0) - exec_vlm = rest.get('vol_exec', 0) + # TODO: store this in a ChainMap instance + # per order dialog. + # submit_vlm = rest.get('vol', 0) + # fee = rest.get('fee', 0) + if status == 'closed': + vlm = 0 + else: + vlm = rest.get('vol_exec', 0) - reqids2txids[reqid] = txid + reqid = reqids2txids.inverse[txid] oid = ids.inverse.get(reqid) if not oid: @@ -594,13 +626,17 @@ async def handle_order_updates( # everyone doin camel case.. status=ems_status, # force lower case - filled=exec_vlm, + filled=vlm, reason='', # why held? - remaining=( - float(submit_vlm) - - - float(exec_vlm) - ), + remaining=vlm, + + # TODO: need to extract the submit vlm + # from a prior msg update.. + # ( + # float(submit_vlm) + # - + # float(exec_vlm) + # ), broker_details=dict( {'name': 'kraken'}, **update_msg @@ -609,12 +645,49 @@ async def handle_order_updates( msgs.append(resp) await ems_stream.send(resp) + # fill event. + # there is no `status` field + case { + 'vol_exec': vlm, + **rest, + }: + # eg. fill msg contents (in total): + # { + # 'vol_exec': '0.84709869', + # 'cost': '101.25370642', + # 'fee': '0.26325964', + # 'avg_price': '119.53000001', + # 'userref': 0, + # } + # TODO: emit fill msg from here + reqid = reqids2txids.inverse[txid] + log.info( + f'openOrders vlm={vlm} Fill for {reqid}:\n' + f'{update_msg}' + ) + continue + case _: log.warning( 'Unknown orders msg:\n' f'{txid}:{order_msg}' ) + # TODO: given the 'openOrders' sub , pretty + # much all the msgs we get for this sub are duplicate + # of the (incremental) updates in that one though we still + # need them because that sub seems to have a bug where the + # `userref` field is always 0 instead of our generated reqid + # value... + # Not sure why kraken devs decided to repeat themselves but + # it almost seems as though we could drop this entire sub + # and get everything we need by just parsing msgs correctly + # above? The only reason for this seems to be remapping + # underlying `txid` values on order "edits" which the + # `openOrders` sub doesn't seem to have any knowledge of. + # I'd also like to ask them which event guarantees that the + # the live order is now in the book, since these status ones + # almost seem more like request-acks then state guarantees. case { 'event': etype, 'status': status, @@ -627,7 +700,13 @@ async def handle_order_updates( 'cancelOrderStatus', } ): + log.info( + f'{etype}:\n' + f'{pformat(msg)}' + ) oid = ids.inverse.get(reqid) + # TODO: relay these to EMS once it supports + # open order loading. if not oid: log.warning( 'Unknown order status update?:\n' @@ -637,6 +716,10 @@ async def handle_order_updates( txid = rest.get('txid') if txid: + # XXX: we **must** do this mapping for edit order + # status updates since the `openOrders` sub above + # never relays back the correct client-side `reqid` + # that is put in the order request.. reqids2txids[reqid] = txid msgs = emsflow[oid] @@ -703,9 +786,9 @@ def process_status( 'descr': descr, # only on success? }: log.info( - f'Submitting order: {descr}\n' + f'Submitted order: {descr}\n' f'ems oid: {oid}\n' - f're-mapped reqid: {reqid}\n' + f'brokerd reqid: {reqid}\n' f'txid: {txid}\n' ) return [], False @@ -722,6 +805,7 @@ def process_status( }: log.info( f'Editting order {oid}[requid={reqid}]:\n' + f'brokerd reqid: {reqid}\n' f'txid: {origtxid} -> {txid}\n' f'{descr}' ) @@ -737,21 +821,12 @@ def process_status( # 'txid': txids, **rest, }: - # TODO: should we support "batch" acking of - # multiple cancels thus avoiding the below loop? - resps: list[MsgUnion] = [] for txid in rest.get('txid', [last.reqid]): - resp = BrokerdStatus( - reqid=reqid, - account=last.account, - time_ns=time.time_ns(), - status='cancelled', - reason='Cancel success: {oid}@{txid}', - broker_details=event, + log.info( + f'Cancelling order {oid}[requid={reqid}]:\n' + f'brokerd reqid: {reqid}\n' ) - resps.append(resp) - - return resps, False + return [], False def norm_trade_records( From 22f9b2552cfefd6064b67d3897b8f49b67488d6c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 10 Jul 2022 19:07:54 -0400 Subject: [PATCH 19/48] Provide symbol norming via a classmethod + global table --- piker/brokers/kraken/api.py | 56 ++++++++++++++++++++----------------- 1 file changed, 31 insertions(+), 25 deletions(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index 17b79027..707ae41a 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -141,6 +141,9 @@ class InvalidKey(ValueError): class Client: + # global symbol normalization table + _ntable: dict[str, str] = {} + def __init__( self, name: str = '', @@ -302,7 +305,9 @@ class Client: async def symbol_info( self, pair: Optional[str] = None, - ): + + ) -> dict[str, dict[str, str]]: + if pair is not None: pairs = {'pair': pair} else: @@ -328,6 +333,12 @@ class Client: if not self._pairs: self._pairs = await self.symbol_info() + ntable = {} + for restapikey, info in self._pairs.items(): + ntable[restapikey] = ntable['wsname'] = info['altname'] + + self._ntable.update(ntable) + return self._pairs async def search_symbols( @@ -425,6 +436,25 @@ class Client: else: raise BrokerError(errmsg) + @classmethod + def normalize_symbol( + cls, + ticker: str + ) -> str: + ''' + Normalize symbol names to to a 3x3 pair from the global + definition map which we build out from the data retreived from + the 'AssetPairs' endpoint, see methods above. + + ''' + symlen = len(ticker) + if symlen != 6: + ticker = cls._ntable[ticker] + else: + raise ValueError(f'Unhandled symbol: {ticker}') + + return ticker.lower() + @acm async def get_client() -> Client: @@ -443,27 +473,3 @@ async def get_client() -> Client: await client.cache_symbols() yield client - - -def normalize_symbol( - ticker: str -) -> str: - ''' - Normalize symbol names to to a 3x3 pair. - - ''' - remap = { - 'XXBTZEUR': 'XBTEUR', - 'XXMRZEUR': 'XMREUR', - - # ws versions? pretty weird.. - 'XBT/EUR': 'XBTEUR', - 'XMR/EUR': 'XMREUR', - } - symlen = len(ticker) - if symlen != 6: - ticker = remap[ticker] - else: - raise ValueError(f'Unhandled symbol: {ticker}') - - return ticker.lower() From abb6854e742118d54bf78e2dc7969c37b658acc8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 10 Jul 2022 20:05:31 -0400 Subject: [PATCH 20/48] Make all `.bsuid`s the normed symbol "altname"s --- piker/brokers/kraken/broker.py | 38 ++++++++++++++++++++++++---------- 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index f98d3314..7d2c3309 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -54,7 +54,6 @@ from .api import ( Client, BrokerError, get_client, - normalize_symbol, ) from .feed import ( get_console_log, @@ -273,7 +272,10 @@ async def trades_dialogue( log.info( f'Loaded {len(trades)} trades from account `{acc_name}`' ) - with open_ledger(acctid, trades) as trans: + with open_ledger( + acctid, + trades, + ) as trans: active, closed = pp.update_pps_conf( 'kraken', acctid, @@ -366,7 +368,7 @@ async def handle_order_updates( emsflow: dict[str, list[MsgUnion]], ids: bidict[str, int], reqids2txids: bidict[int, str], - trans: list[pp.Transaction], + trans: set[pp.Transaction], acctid: str, acc_name: str, token: str, @@ -381,7 +383,7 @@ async def handle_order_updates( ''' # transaction records which will be updated # on new trade clearing events (aka order "fills") - trans: list[pp.Transaction] + trans: set[pp.Transaction] async for msg in ws_stream: match msg: @@ -467,16 +469,27 @@ async def handle_order_updates( ) await ems_stream.send(filled_msg) + if not trades: + # skip pp emissions if we have already + # processed all trades in this msg. + continue + # update ledger and position tracking - with open_ledger(acctid, trades) as trans: - # TODO: ideally we can pass in an existingn + await tractor.breakpoint() + trans: set[pp.Transaction] + with open_ledger( + acctid, + trades, + + ) as trans: + # TODO: ideally we can pass in an existing # pps state to this right? such that we # don't have to do a ledger reload all the # time.. active, closed = pp.update_pps_conf( 'kraken', acctid, - trade_records=trans, + trade_records=list(trans), ledger_reload={}.fromkeys( t.bsuid for t in trans), ) @@ -841,8 +854,9 @@ def norm_trade_records( 'buy': 1, 'sell': -1, }[record['type']] - bsuid = record['pair'] - norm_sym = normalize_symbol(bsuid) + + # we normalize to kraken's `altname` always.. + bsuid = norm_sym = Client.normalize_symbol(record['pair']) records.append( pp.Transaction( @@ -867,7 +881,7 @@ def open_ledger( acctid: str, trade_entries: list[dict[str, Any]], -) -> list[pp.Transaction]: +) -> set[pp.Transaction]: ''' Write recent session's trades to the user's (local) ledger file. @@ -878,8 +892,10 @@ def open_ledger( ) as ledger: # normalize to transaction form + # TODO: cawt damn, we should probably delegate to cryptofeed for + # this insteada of re-hacking kraken's total crap? records = norm_trade_records(trade_entries) - yield records + yield set(records) # update on exit ledger.update(trade_entries) From 5b135fad619e8adf6156d8d7e75a812329b4e3e3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Jul 2022 12:24:28 -0400 Subject: [PATCH 21/48] Handle pre-existing open orders specifically by checking for null `oid` --- piker/brokers/kraken/broker.py | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 7d2c3309..190d73b7 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -202,7 +202,10 @@ async def handle_order_requests( async def subscribe( ws: wsproto.WSConnection, token: str, - subs: list[str] = ['ownTrades', 'openOrders'], + subs: list[str] = [ + 'ownTrades', + 'openOrders', + ], ): ''' Setup ws api subscriptions: @@ -603,10 +606,13 @@ async def handle_order_updates( else: vlm = rest.get('vol_exec', 0) - reqid = reqids2txids.inverse[txid] - + reqid = reqids2txids.inverse.get(txid) oid = ids.inverse.get(reqid) - if not oid: + + if ( + status == 'open' + and oid is None # a non-ems-active order + ): # TODO: handle these and relay them # through the EMS to the client / UI # side! @@ -621,13 +627,11 @@ async def handle_order_updates( await ws.send_msg({ 'event': 'cancelOrder', 'token': token, - 'reqid': reqid, + 'reqid': reqid or 0, 'txid': [txid], }) continue - msgs = emsflow[oid] - # send BrokerdStatus messages for all # order state updates resp = BrokerdStatus( @@ -655,7 +659,11 @@ async def handle_order_updates( {'name': 'kraken'}, **update_msg ), ) - msgs.append(resp) + + # TODO: use collections.ChainMap here + # msgs = emsflow[oid] + # msgs.append(resp) + await ems_stream.send(resp) # fill event. From 2386270cad23de80e23488a37217f20189d566a3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Jul 2022 16:08:47 -0400 Subject: [PATCH 22/48] Handle too-fast-edits, add `ChainMap` msg tracing Since our ems doesn't actually do blocking style client-side submission updates, thus resulting in the client being able to update an existing order's state before knowing its current state, we can run into race conditions where for some backends an order is updated using the wrong order id. For kraken we manually implement detecting this race (lol, for now anyway) such that when a new client side edit comes in before the new `txid` is known, we simply expect the handler loop to cancel the order. Further this adds cancellation on arbitrary status errors, like rate limits. Also this adds 2 leg (ems <-> brokerd <-> kraken) msg tracing using a `collections.ChainMap` which is likely going to end up being the POC for a more general data structure recommended for backends that need to trace msg flow for translation with the ems. --- piker/brokers/kraken/broker.py | 143 +++++++++++++++++++++++---------- 1 file changed, 100 insertions(+), 43 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 190d73b7..75e67bbd 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -18,12 +18,14 @@ Order api and machinery ''' +from collections import ChainMap, defaultdict from contextlib import ( asynccontextmanager as acm, contextmanager as cm, ) from functools import partial -from itertools import chain, count +import itertools +from itertools import count from pprint import pformat import time from typing import ( @@ -73,13 +75,17 @@ MsgUnion = Union[ ] +class TooFastEdit(Exception): + 'Edit requests faster then api submissions' + + async def handle_order_requests( ws: NoBsWs, client: Client, ems_order_stream: tractor.MsgStream, token: str, - emsflow: dict[str, list[MsgUnion]], + apiflows: dict[int, ChainMap[dict[str, dict]]], ids: bidict[str, int], reqids2txids: dict[int, str], @@ -101,7 +107,6 @@ async def handle_order_requests( 'action': 'cancel', }: cancel = BrokerdCancel(**msg) - # last = emsflow[cancel.oid] reqid = ids[cancel.oid] txid = reqids2txids[reqid] @@ -126,7 +131,21 @@ async def handle_order_requests( if order.oid in ids: ep = 'editOrder' reqid = ids[order.oid] # integer not txid - txid = reqids2txids[reqid] + try: + txid = reqids2txids.pop(reqid) + except KeyError: + reqids2txids[reqid] = TooFastEdit(reqid) + await ems_order_stream.send( + BrokerdError( + oid=msg['oid'], + symbol=msg['symbol'], + reason=( + f'TooFastEdit reqid:{reqid}, cancelling..' + ), + + ) + ) + extra = { 'orderid': txid, # txid } @@ -176,7 +195,7 @@ async def handle_order_requests( await ws.send_msg(req) # placehold for sanity checking in relay loop - emsflow.setdefault(order.oid, []).append(order) + apiflows[reqid].maps.append(msg) case _: account = msg.get('account') @@ -328,10 +347,10 @@ async def trades_dialogue( aclosing(stream_messages(ws)) as stream, ): # task local msg dialog tracking - emsflow: dict[ - str, - list[MsgUnion], - ] = {} + apiflows: defaultdict[ + int, + ChainMap[dict[str, dict]], + ] = defaultdict(ChainMap) # 2way map for ems ids to kraken int reqids.. ids: bidict[str, int] = bidict() @@ -344,7 +363,7 @@ async def trades_dialogue( client, ems_stream, token, - emsflow, + apiflows, ids, reqids2txids, ) @@ -354,7 +373,7 @@ async def trades_dialogue( ws, stream, ems_stream, - emsflow, + apiflows, ids, reqids2txids, trans, @@ -368,7 +387,7 @@ async def handle_order_updates( ws: NoBsWs, ws_stream: AsyncIterator, ems_stream: tractor.MsgStream, - emsflow: dict[str, list[MsgUnion]], + apiflows: dict[int, ChainMap[dict[str, dict]]], ids: bidict[str, int], reqids2txids: bidict[int, str], trans: set[pp.Transaction], @@ -478,7 +497,6 @@ async def handle_order_updates( continue # update ledger and position tracking - await tractor.breakpoint() trans: set[pp.Transaction] with open_ledger( acctid, @@ -500,7 +518,7 @@ async def handle_order_updates( # emit any new pp msgs to ems for pos in filter( bool, - chain(active.values(), closed.values()), + itertools.chain(active.values(), closed.values()), ): pp_msg = BrokerdPosition( broker='kraken', @@ -611,16 +629,40 @@ async def handle_order_updates( if ( status == 'open' - and oid is None # a non-ems-active order - ): - # TODO: handle these and relay them - # through the EMS to the client / UI - # side! - log.warning( - f'Received active order {txid}:\n' - f'{update_msg}\n' - 'Cancelling order for now!..' + and ( + + # TOO fast edit handled by the + # request handler task. + (toofast := isinstance( + reqids2txids.get(reqid), + TooFastEdit + )) + + # pre-existing open order NOT from + # this EMS session. + or (noid := oid is None) ) + ): + if toofast: + # TODO: don't even allow this case + # by not moving the client side line + # until an edit confirmation + # arrives... + log.warning( + f'Received too fast edit {txid}:\n' + f'{update_msg}\n' + 'Cancelling order for now!..' + ) + + elif noid: # a non-ems-active order + # TODO: handle these and relay them + # through the EMS to the client / UI + # side! + log.warning( + f'Rx unknown active order {txid}:\n' + f'{update_msg}\n' + 'Cancelling order for now!..' + ) # call ws api to cancel: # https://docs.kraken.com/websockets/#message-cancelOrder @@ -660,10 +702,7 @@ async def handle_order_updates( ), ) - # TODO: use collections.ChainMap here - # msgs = emsflow[oid] - # msgs.append(resp) - + apiflows[reqid].maps.append(update_msg) await ems_stream.send(resp) # fill event. @@ -735,28 +774,47 @@ async def handle_order_updates( ) continue + lasttxid = reqids2txids.get(reqid) txid = rest.get('txid') - if txid: - # XXX: we **must** do this mapping for edit order - # status updates since the `openOrders` sub above - # never relays back the correct client-side `reqid` - # that is put in the order request.. - reqids2txids[reqid] = txid - msgs = emsflow[oid] - last = msgs[-1] + # update the msg chain + chain = apiflows[reqid] + chain.maps.append(event) + + # pretxid = chain['txid'] + # print(f'pretxid: {pretxid}') + resps, errored = process_status( event, oid, token, - msgs, - last, + chain, ) if resps: - msgs.extend(resps) for resp in resps: await ems_stream.send(resp) + if txid: + if ( + isinstance(lasttxid, TooFastEdit) + or errored + ): + # client was editting too quickly + # so we instead cancel this order + print("SENDING CANCEL") + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid or 0, + 'txid': [txid], + }) + else: + # XXX: we **must** do this mapping for edit order + # status updates since the `openOrders` sub above + # never relays back the correct client-side `reqid` + # that is put in the order request.. + reqids2txids[reqid] = txid + case _: log.warning(f'Unhandled trades update msg: {msg}') @@ -765,8 +823,7 @@ def process_status( event: dict[str, str], oid: str, token: str, - msgs: list[MsgUnion], - last: MsgUnion, + chain: ChainMap, ) -> tuple[list[MsgUnion], bool]: ''' @@ -782,7 +839,7 @@ def process_status( 'errorMessage': errmsg, }: # any of ``{'add', 'edit', 'cancel'}`` - action = etype.rstrip('OrderStatus') + action = etype.removesuffix('OrderStatus') log.error( f'Failed to {action} order {reqid}:\n' f'{errmsg}' @@ -791,7 +848,7 @@ def process_status( oid=oid, # XXX: use old reqid in case it changed? reqid=reqid, - symbol=getattr(last, 'symbol', 'N/A'), + symbol=chain.get('symbol', 'N/A'), reason=f'Failed {action}:\n{errmsg}', broker_details=event @@ -842,7 +899,7 @@ def process_status( # 'txid': txids, **rest, }: - for txid in rest.get('txid', [last.reqid]): + for txid in rest.get('txid', [chain['reqid']]): log.info( f'Cancelling order {oid}[requid={reqid}]:\n' f'brokerd reqid: {reqid}\n' From 0fca1b3e1a710d44060ac02635311d072716cfe3 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 13 Jul 2022 21:10:12 -0400 Subject: [PATCH 23/48] Also map the ws symbol set to the alt set --- piker/brokers/kraken/api.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index 707ae41a..b4ea23e5 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -335,7 +335,7 @@ class Client: ntable = {} for restapikey, info in self._pairs.items(): - ntable[restapikey] = ntable['wsname'] = info['altname'] + ntable[restapikey] = ntable[info['wsname']] = info['altname'] self._ntable.update(ntable) From b96b7a8b9cbdf1342ada6b8c54ab3ae68d7dfc79 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 15 Jul 2022 16:26:18 -0400 Subject: [PATCH 24/48] Use `aclosing()` on all msg async-gens --- piker/brokers/kraken/feed.py | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/piker/brokers/kraken/feed.py b/piker/brokers/kraken/feed.py index 4966db8a..e8cfd9b6 100644 --- a/piker/brokers/kraken/feed.py +++ b/piker/brokers/kraken/feed.py @@ -28,6 +28,7 @@ from typing import ( ) import time +from async_generator import aclosing from fuzzywuzzy import process as fuzzy import numpy as np import pendulum @@ -407,17 +408,15 @@ async def stream_quotes( # see the tips on reconnection logic: # https://support.kraken.com/hc/en-us/articles/360044504011-WebSocket-API-unexpected-disconnections-from-market-data-feeds ws: NoBsWs - async with open_autorecon_ws( - 'wss://ws.kraken.com/', - fixture=subscribe, - ) as ws: - + async with ( + open_autorecon_ws( + 'wss://ws.kraken.com/', + fixture=subscribe, + ) as ws, + aclosing(process_data_feed_msgs(ws)) as msg_gen, + ): # pull a first quote and deliver - msg_gen = process_data_feed_msgs(ws) - - # TODO: use ``anext()`` when it lands in 3.10! typ, ohlc_last = await anext(msg_gen) - topic, quote = normalize(ohlc_last) task_status.started((init_msgs, quote)) From 54008a19769d39118e8c37652f1f86006b5af847 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 18 Jul 2022 12:28:52 -0400 Subject: [PATCH 25/48] Add balance and assets retreival methods, cache assets on startup Pass config dict into client and assign to `.conf`. --- piker/brokers/kraken/api.py | 57 ++++++++++++++++++++++++++++++------- 1 file changed, 47 insertions(+), 10 deletions(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index b4ea23e5..46c05f5b 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -29,8 +29,7 @@ from typing import ( ) import time -# import trio -# import tractor +from bidict import bidict import pendulum import asks from fuzzywuzzy import process as fuzzy @@ -40,6 +39,7 @@ import urllib.parse import hashlib import hmac import base64 +import trio from piker import config from piker.brokers._util import ( @@ -143,9 +143,11 @@ class Client: # global symbol normalization table _ntable: dict[str, str] = {} + _atable: bidict[str, str] = bidict() def __init__( self, + config: dict[str, str], name: str = '', api_key: str = '', secret: str = '' @@ -156,6 +158,7 @@ class Client: 'User-Agent': 'krakenex/2.1.0 (+https://github.com/veox/python3-krakenex)' }) + self.conf: dict[str, str] = config self._pairs: list[str] = [] self._name = name self._api_key = api_key @@ -215,8 +218,36 @@ class Client: data['nonce'] = str(int(1000*time.time())) return await self._private(method, data, uri_path) + async def get_balances( + self, + ) -> dict[str, float]: + ''' + Return the set of asset balances for this account + by symbol. + + ''' + resp = await self.endpoint( + 'Balance', + {}, + ) + by_bsuid = resp['result'] + return { + self._atable[sym].lower(): float(bal) + for sym, bal in by_bsuid.items() + } + + async def get_assets(self) -> dict[str, dict]: + resp = await self._public('Assets', {}) + return resp['result'] + + async def cache_assets(self) -> None: + assets = self.assets = await self.get_assets() + for bsuid, info in assets.items(): + self._atable[bsuid] = info['altname'] + async def get_trades( self, + fetch_limit: int = 10, ) -> dict[str, Any]: ''' @@ -228,6 +259,8 @@ class Client: trades_by_id: dict[str, Any] = {} for i in itertools.count(): + if i >= fetch_limit: + break # increment 'ofs' pagination offset ofs = i*50 @@ -459,17 +492,21 @@ class Client: @acm async def get_client() -> Client: - section = get_config() - if section: + conf = get_config() + if conf: client = Client( - name=section['key_descr'], - api_key=section['api_key'], - secret=section['secret'] + conf, + name=conf['key_descr'], + api_key=conf['api_key'], + secret=conf['secret'] ) else: - client = Client() + client = Client({}) - # at startup, load all symbols locally for fast search - await client.cache_symbols() + # at startup, load all symbols, and asset info in + # batch requests. + async with trio.open_nursery() as nurse: + nurse.start_soon(client.cache_assets) + await client.cache_symbols() yield client From 3b79743c7b44ca7c6f80cd9e1b7b40f68094196c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Jul 2022 08:43:33 -0400 Subject: [PATCH 26/48] Finally get real-time pp updates workin for `kraken` This ended up driving the rework of the `piker.pp` apis to use context manager + table style which resulted in a much easier to follow state/update system B). Also added is a flag to do a manual simulation of a "fill triggered rt pp msg" which requires the user to delete the last ledgered trade entry from config files and then allowing that trade to emit through the `openOrders` sub and update client shortly after order mode boot; this is how the rt updates were verified to work without doing even more live orders :joy:. Patch details: - open both `open_trade_ledger()` and `open_pps()` inside the trade dialog startup and conduct a "pp state sync" logic phase where we now pull the account balances and incrementally load pp data (in order, from `pps.toml`, ledger, api) until we can generate the asset balance by reverse incrementing through trade history eventually erroring out if we can't reproduce the balance value. - rework the `trade2pps()` to take in the `PpTable` and generate new ems msgs from table updates. - return the new `dict[str, Transaction]` expected from `norm_trade_records()` - only update pp config and ledger on dialog exit. --- piker/brokers/kraken/broker.py | 462 +++++++++++++++++++++------------ 1 file changed, 294 insertions(+), 168 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 75e67bbd..2126d886 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -24,13 +24,14 @@ from contextlib import ( contextmanager as cm, ) from functools import partial -import itertools from itertools import count +import math from pprint import pformat import time from typing import ( Any, AsyncIterator, + # Optional, Union, ) @@ -41,7 +42,14 @@ import trio import tractor import wsproto -from piker import pp +from piker.pp import ( + Position, + PpTable, + Transaction, + # update_pps_conf, + open_trade_ledger, + open_pps, +) from piker.clearing._messages import ( BrokerdCancel, BrokerdError, @@ -265,10 +273,56 @@ async def subscribe( # await ws.recv_msg() +def trades2pps( + table: PpTable, + acctid: str, + new_trans: dict[str, Transaction] = {}, + +) -> tuple[ + list[BrokerdPosition], + list[Transaction], +]: + if new_trans: + updated = table.update_from_trans( + new_trans, + ) + log.info(f'Updated pps:\n{pformat(updated)}') + + pp_entries, closed_pp_objs = table.dump_active('kraken') + pp_objs: dict[Union[str, int], Position] = table.pps + + pps: dict[int, Position] + position_msgs: list[dict] = [] + + for pps in [pp_objs, closed_pp_objs]: + for tid, p in pps.items(): + msg = BrokerdPosition( + broker='kraken', + # XXX: ok so this is annoying, we're + # relaying an account name with the + # backend suffix prefixed but when + # reading accounts from ledgers we + # don't need it and/or it's prefixed + # in the section table.. we should + # just strip this from the message + # right since `.broker` is already + # included? + account='kraken.' + acctid, + symbol=p.symbol.front_fqsn(), + size=p.size, + avg_price=p.be_price, + currency='', + ) + position_msgs.append(msg) + + return position_msgs + + @tractor.context async def trades_dialogue( ctx: tractor.Context, loglevel: str = None, + ) -> AsyncIterator[dict[str, Any]]: # XXX: required to propagate ``tractor`` loglevel to piker logging @@ -289,98 +343,193 @@ async def trades_dialogue( acctid = client._name acc_name = 'kraken.' + acctid - # pull and deliver trades ledger - trades = await client.get_trades() - log.info( - f'Loaded {len(trades)} trades from account `{acc_name}`' - ) - with open_ledger( - acctid, - trades, - ) as trans: - active, closed = pp.update_pps_conf( + # task local msg dialog tracking + apiflows: defaultdict[ + int, + ChainMap[dict[str, dict]], + ] = defaultdict(ChainMap) + + # 2way map for ems ids to kraken int reqids.. + ids: bidict[str, int] = bidict() + reqids2txids: bidict[int, str] = bidict() + + # NOTE: testing code for making sure the rt incremental update + # of positions, via newly generated msgs works. In order to test + # this, + # - delete the *ABSOLUTE LAST* entry from accont's corresponding + # trade ledgers file (NOTE this MUST be the last record + # delivered from the + # api ledger), + # - open you ``pps.toml`` and find that same tid and delete it + # from the pp's clears table, + # - set this flag to `True` + # + # You should see an update come in after the order mode + # boots up which shows your latest correct asset + # balance size after the "previously unknown simulating a live + # fill" update comes in on the relay loop even though the fill + # will be ignored by the ems (no known reqid) the pp msg should + # update things correctly. + simulate_pp_update: bool = False + + with ( + open_pps( 'kraken', acctid, - trade_records=trans, - ledger_reload={}.fromkeys(t.bsuid for t in trans), - ) + ) as table, - position_msgs: list[dict] = [] - pps: dict[int, pp.Position] - for pps in [active, closed]: - for tid, p in pps.items(): - msg = BrokerdPosition( - broker='kraken', - account=acc_name, - symbol=p.symbol.front_fqsn(), - size=p.size, - avg_price=p.be_price, - currency='', - ) - position_msgs.append(msg) - - await ctx.started( - (position_msgs, [acc_name]) - ) - - # Get websocket token for authenticated data stream - # Assert that a token was actually received. - resp = await client.endpoint('GetWebSocketsToken', {}) - - err = resp.get('error') - if err: - raise BrokerError(err) - - token = resp['result']['token'] - - ws: NoBsWs - async with ( - ctx.open_stream() as ems_stream, - open_autorecon_ws( - 'wss://ws-auth.kraken.com/', - fixture=partial( - subscribe, - token=token, - ), - ) as ws, - trio.open_nursery() as n, - aclosing(stream_messages(ws)) as stream, - ): - # task local msg dialog tracking - apiflows: defaultdict[ - int, - ChainMap[dict[str, dict]], - ] = defaultdict(ChainMap) - - # 2way map for ems ids to kraken int reqids.. - ids: bidict[str, int] = bidict() - reqids2txids: bidict[int, str] = bidict() - - # task for processing inbound requests from ems - n.start_soon( - handle_order_requests, - ws, - client, - ems_stream, - token, - apiflows, - ids, - reqids2txids, - ) - - # enter relay loop - await handle_order_updates( - ws, - stream, - ems_stream, - apiflows, - ids, - reqids2txids, - trans, + open_trade_ledger( + 'kraken', acctid, - acc_name, - token, + ) as ledger_dict, + ): + # transaction-ify the ledger entries + ledger_trans = norm_trade_records(ledger_dict) + + # TODO: eventually probably only load + # as far back as it seems is not deliverd in the + # most recent 50 trades and assume that by ordering we + # already have those records in the ledger. + tids2trades = await client.get_trades() + api_trans = norm_trade_records(tids2trades) + + # retrieve kraken reported balances + # and do diff with ledger to determine + # what amount of trades-transactions need + # to be reloaded. + sizes = await client.get_balances() + for dst, size in sizes.items(): + # we don't care about tracking positions + # in the user's source fiat currency. + if dst == client.conf['src_fiat']: + continue + + def has_pp(dst: str) -> Position | bool: + pps_dst_assets = {bsuid[:3]: bsuid for bsuid in table.pps} + pair = pps_dst_assets.get(dst) + pp = table.pps.get(pair) + + if ( + not pair or not pp + or not math.isclose(pp.size, size) + ): + return False + + return pp + + pos = has_pp(dst) + + if not pos: + # we have a balance for which there is no pp + # entry? so we have to likely update from the + # ledger. + updated = table.update_from_trans(ledger_trans) + log.info(f'Updated pps from ledger:\n{pformat(updated)}') + pos = has_pp(dst) + + if not pos and not simulate_pp_update: + # try reloading from API + table.update_from_trans(api_trans) + pos = has_pp(dst) + if not pos: + raise ValueError( + 'Could not reproduce balance:\n' + f'dst: {dst}, {size}\n' + ) + + # only for simulate-testing a "new fill" since + # otherwise we have to actually conduct a live clear. + if simulate_pp_update: + tid = list(tids2trades)[0] + last_trade_dict = tids2trades[tid] + # stage a first reqid of `0` + reqids2txids[0] = last_trade_dict['ordertxid'] + + # reverse the volume on the last trade record so that we can + # use it to update the pptable and thus reverse the last + # trade's effect on the position size. + # last_trade_dict['vol'] = str(float(last_trade_dict['vol']) * -1) + + # convert the reversed trade into transaction format + # fake_tid = ''.join(reversed(tid)) + # reversed_last_tran = norm_trade_records( + # {fake_tid: last_trade_dict})[fake_tid] + # assert reversed_last_tran.size == -1 * ( + # list(api_trans.values())[0].size) + + # update the pp table with the reversed trade thus taking + # the sizing back to "one trade prior" to the last one. + # table.update_from_trans({tid: reversed_last_tran}) + + ppmsgs = trades2pps( + table, + acctid, + # new_trans, ) + await ctx.started((ppmsgs, [acc_name])) + + # XXX: not fucking clue but putting this finally block + # will suppress errors inside the direct await below!?! + # try: + + # Get websocket token for authenticated data stream + # Assert that a token was actually received. + resp = await client.endpoint('GetWebSocketsToken', {}) + + err = resp.get('error') + if err: + raise BrokerError(err) + + token = resp['result']['token'] + + ws: NoBsWs + async with ( + ctx.open_stream() as ems_stream, + open_autorecon_ws( + 'wss://ws-auth.kraken.com/', + fixture=partial( + subscribe, + token=token, + ), + ) as ws, + aclosing(stream_messages(ws)) as stream, + trio.open_nursery() as nurse, + ): + stream = stream_messages(ws) + + # task for processing inbound requests from ems + nurse.start_soon( + handle_order_requests, + ws, + client, + ems_stream, + token, + apiflows, + ids, + reqids2txids, + ) + + # enter relay loop + # try: + try: + await handle_order_updates( + ws, + stream, + ems_stream, + apiflows, + ids, + reqids2txids, + table, + api_trans, + acctid, + acc_name, + token, + ) + # except: + # await tractor.breakpoint() + finally: + # always update ledger on exit + ledger_dict.update(tids2trades) async def handle_order_updates( @@ -390,7 +539,11 @@ async def handle_order_updates( apiflows: dict[int, ChainMap[dict[str, dict]]], ids: bidict[str, int], reqids2txids: bidict[int, str], - trans: set[pp.Transaction], + table: PpTable, + + # transaction records which will be updated + # on new trade clearing events (aka order "fills") + ledger_trans: dict[str, Transaction], acctid: str, acc_name: str, token: str, @@ -403,9 +556,6 @@ async def handle_order_updates( defined in the signature clear to the reader. ''' - # transaction records which will be updated - # on new trade clearing events (aka order "fills") - trans: set[pp.Transaction] async for msg in ws_stream: match msg: @@ -427,17 +577,34 @@ async def handle_order_updates( f'ownTrades update_{seq}:\n' f'{pformat(trades_msgs)}' ) - # flatten msgs to an {id -> data} table for processing + # assert 0 + # format as tid -> trade event map + # eg. msg + # [{'TOKWHY-SMTUB-G5DOI6': {'cost': '95.29047', + # 'fee': '0.24776', + # 'margin': '0.00000', + # 'ordertxid': 'OKSUXM-3OLSB-L7TN72', + # 'ordertype': 'limit', + # 'pair': 'XBT/EUR', + # 'postxid': 'TKH2SE-M7IF5-CFI7LT', + # 'price': '21268.20000', + # 'time': '1657990947.640891', + # 'type': 'buy', + # 'vol': '0.00448042'}}] trades = { tid: trade for entry in trades_msgs for (tid, trade) in entry.items() - - # only emit entries which are already not-in-ledger - if tid not in {r.tid for r in trans} + if tid not in ledger_trans } - for tid, trade in trades.items(): + # if tid in ledger_trans: + # # skip already seen transactions + # log.info(f'Skipping already seen trade {trade}') + # continue + + # await tractor.breakpoint() + for tid, trade in trades.items(): txid = trade['ordertxid'] # NOTE: yet again, here we don't have any ref to the @@ -491,57 +658,22 @@ async def handle_order_updates( ) await ems_stream.send(filled_msg) - if not trades: - # skip pp emissions if we have already - # processed all trades in this msg. - continue + # if not trades: + # # skip pp emissions if we have already + # # processed all trades in this msg. + # continue - # update ledger and position tracking - trans: set[pp.Transaction] - with open_ledger( + new_trans = norm_trade_records(trades) + ppmsgs = trades2pps( + table, acctid, - trades, - - ) as trans: - # TODO: ideally we can pass in an existing - # pps state to this right? such that we - # don't have to do a ledger reload all the - # time.. - active, closed = pp.update_pps_conf( - 'kraken', - acctid, - trade_records=list(trans), - ledger_reload={}.fromkeys( - t.bsuid for t in trans), - ) - - # emit any new pp msgs to ems - for pos in filter( - bool, - itertools.chain(active.values(), closed.values()), - ): - pp_msg = BrokerdPosition( - broker='kraken', - - # XXX: ok so this is annoying, we're - # relaying an account name with the - # backend suffix prefixed but when - # reading accounts from ledgers we - # don't need it and/or it's prefixed - # in the section table.. we should - # just strip this from the message - # right since `.broker` is already - # included? - account=f'kraken.{acctid}', - symbol=pos.symbol.front_fqsn(), - size=pos.size, - avg_price=pos.be_price, - - # TODO - # currency='' - ) + new_trans, + ) + for pp_msg in ppmsgs: await ems_stream.send(pp_msg) + ledger_trans.update(new_trans) + # process and relay order state change events # https://docs.kraken.com/websockets/#message-openOrders case [ @@ -801,7 +933,7 @@ async def handle_order_updates( ): # client was editting too quickly # so we instead cancel this order - print("SENDING CANCEL") + log.cancel(f'Cancelling order for {reqid}@{txid}') await ws.send_msg({ 'event': 'cancelOrder', 'token': token, @@ -910,9 +1042,10 @@ def process_status( def norm_trade_records( ledger: dict[str, Any], -) -> list[pp.Transaction]: +) -> dict[str, Transaction]: + + records: dict[str, Transaction] = {} - records: list[pp.Transaction] = [] for tid, record in ledger.items(): size = float(record.get('vol')) * { @@ -923,19 +1056,17 @@ def norm_trade_records( # we normalize to kraken's `altname` always.. bsuid = norm_sym = Client.normalize_symbol(record['pair']) - records.append( - pp.Transaction( - fqsn=f'{norm_sym}.kraken', - tid=tid, - size=size, - price=float(record['price']), - cost=float(record['fee']), - dt=pendulum.from_timestamp(float(record['time'])), - bsuid=bsuid, + records[tid] = Transaction( + fqsn=f'{norm_sym}.kraken', + tid=tid, + size=size, + price=float(record['price']), + cost=float(record['fee']), + dt=pendulum.from_timestamp(float(record['time'])), + bsuid=bsuid, - # XXX: there are no derivs on kraken right? - # expiry=expiry, - ) + # XXX: there are no derivs on kraken right? + # expiry=expiry, ) return records @@ -946,21 +1077,16 @@ def open_ledger( acctid: str, trade_entries: list[dict[str, Any]], -) -> set[pp.Transaction]: +) -> set[Transaction]: ''' Write recent session's trades to the user's (local) ledger file. ''' - with pp.open_trade_ledger( + with open_trade_ledger( 'kraken', acctid, ) as ledger: - - # normalize to transaction form - # TODO: cawt damn, we should probably delegate to cryptofeed for - # this insteada of re-hacking kraken's total crap? - records = norm_trade_records(trade_entries) - yield set(records) + yield ledger # update on exit ledger.update(trade_entries) From 64f920d7e5a2588bbdcb6e62b6ad22f3cd81fbf7 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Jul 2022 08:59:09 -0400 Subject: [PATCH 27/48] Accept direct fqsn matches on position msg updates --- piker/ui/order_mode.py | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/piker/ui/order_mode.py b/piker/ui/order_mode.py index d7db1bee..41078e05 100644 --- a/piker/ui/order_mode.py +++ b/piker/ui/order_mode.py @@ -794,15 +794,11 @@ async def process_trades_and_update_ui( pp_msg_symbol = msg['symbol'].lower() fqsn = sym.front_fqsn() broker, key = sym.front_feed() - # print( - # f'pp msg symbol: {pp_msg_symbol}\n', - # f'fqsn: {fqsn}\n', - # f'front key: {key}\n', - # ) - if ( - pp_msg_symbol == fqsn.replace(f'.{broker}', '') + pp_msg_symbol == fqsn + or pp_msg_symbol == fqsn.removesuffix(f'.{broker}') ): + log.info(f'{fqsn} matched pp msg: {fmsg}') tracker = mode.trackers[msg['account']] tracker.live_pp.update_from_msg(msg) # update order pane widgets From 319e68c855ded0d597cf40ce519a39fde53bcc8a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Jul 2022 09:03:57 -0400 Subject: [PATCH 28/48] TOSQUASH: revert to 22Hz display throttle --- piker/ui/_display.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/ui/_display.py b/piker/ui/_display.py index f79c56ae..05603c63 100644 --- a/piker/ui/_display.py +++ b/piker/ui/_display.py @@ -63,7 +63,7 @@ from ..log import get_logger log = get_logger(__name__) # TODO: load this from a config.toml! -_quote_throttle_rate: int = 60 # Hz +_quote_throttle_rate: int = 22 # Hz # a working tick-type-classes template From aa7f24b6db923071b15eac953afc10fe03e02a0b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Jul 2022 10:39:01 -0400 Subject: [PATCH 29/48] Drop old reversed order idea for rt-pp msg testing --- piker/brokers/kraken/broker.py | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 2126d886..43dab81c 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -445,22 +445,6 @@ async def trades_dialogue( # stage a first reqid of `0` reqids2txids[0] = last_trade_dict['ordertxid'] - # reverse the volume on the last trade record so that we can - # use it to update the pptable and thus reverse the last - # trade's effect on the position size. - # last_trade_dict['vol'] = str(float(last_trade_dict['vol']) * -1) - - # convert the reversed trade into transaction format - # fake_tid = ''.join(reversed(tid)) - # reversed_last_tran = norm_trade_records( - # {fake_tid: last_trade_dict})[fake_tid] - # assert reversed_last_tran.size == -1 * ( - # list(api_trans.values())[0].size) - - # update the pp table with the reversed trade thus taking - # the sizing back to "one trade prior" to the last one. - # table.update_from_trans({tid: reversed_last_tran}) - ppmsgs = trades2pps( table, acctid, From b1419c850d3fce6219de12929855e1f67b2c8d7e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 19 Jul 2022 11:02:09 -0400 Subject: [PATCH 30/48] Update ledger from api immediately, cruft cleaning --- piker/brokers/kraken/broker.py | 62 +++++++++++++--------------------- 1 file changed, 23 insertions(+), 39 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 43dab81c..6c56a3e7 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -31,7 +31,6 @@ import time from typing import ( Any, AsyncIterator, - # Optional, Union, ) @@ -46,7 +45,6 @@ from piker.pp import ( Position, PpTable, Transaction, - # update_pps_conf, open_trade_ledger, open_pps, ) @@ -391,6 +389,7 @@ async def trades_dialogue( # most recent 50 trades and assume that by ordering we # already have those records in the ledger. tids2trades = await client.get_trades() + ledger_dict.update(tids2trades) api_trans = norm_trade_records(tids2trades) # retrieve kraken reported balances @@ -448,12 +447,13 @@ async def trades_dialogue( ppmsgs = trades2pps( table, acctid, - # new_trans, ) await ctx.started((ppmsgs, [acc_name])) # XXX: not fucking clue but putting this finally block # will suppress errors inside the direct await below!?! + # likely something to do with the exist stack inside + # the nobsws stuff... # try: # Get websocket token for authenticated data stream @@ -494,26 +494,19 @@ async def trades_dialogue( ) # enter relay loop - # try: - try: - await handle_order_updates( - ws, - stream, - ems_stream, - apiflows, - ids, - reqids2txids, - table, - api_trans, - acctid, - acc_name, - token, - ) - # except: - # await tractor.breakpoint() - finally: - # always update ledger on exit - ledger_dict.update(tids2trades) + await handle_order_updates( + ws, + stream, + ems_stream, + apiflows, + ids, + reqids2txids, + table, + api_trans, + acctid, + acc_name, + token, + ) async def handle_order_updates( @@ -561,9 +554,13 @@ async def handle_order_updates( f'ownTrades update_{seq}:\n' f'{pformat(trades_msgs)}' ) + # XXX: a fix / todo + # see the comment in the caller about weird error + # suppression around a commented `try:` # assert 0 + # format as tid -> trade event map - # eg. msg + # eg. received msg format, # [{'TOKWHY-SMTUB-G5DOI6': {'cost': '95.29047', # 'fee': '0.24776', # 'margin': '0.00000', @@ -579,15 +576,10 @@ async def handle_order_updates( tid: trade for entry in trades_msgs for (tid, trade) in entry.items() + + # don't re-process datums we've already seen if tid not in ledger_trans } - - # if tid in ledger_trans: - # # skip already seen transactions - # log.info(f'Skipping already seen trade {trade}') - # continue - - # await tractor.breakpoint() for tid, trade in trades.items(): txid = trade['ordertxid'] @@ -642,11 +634,6 @@ async def handle_order_updates( ) await ems_stream.send(filled_msg) - # if not trades: - # # skip pp emissions if we have already - # # processed all trades in this msg. - # continue - new_trans = norm_trade_records(trades) ppmsgs = trades2pps( table, @@ -897,9 +884,6 @@ async def handle_order_updates( chain = apiflows[reqid] chain.maps.append(event) - # pretxid = chain['txid'] - # print(f'pretxid: {pretxid}') - resps, errored = process_status( event, oid, From d502274eb9203d21122b7fc87619d41057279de8 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 22 Jul 2022 19:09:12 -0400 Subject: [PATCH 31/48] Add a `Client.get_xfers()` to retreive withdrawal transactions --- piker/brokers/kraken/api.py | 56 +++++++++++++++++++++++++++++++++++++ 1 file changed, 56 insertions(+) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index 46c05f5b..b08d9f52 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -48,6 +48,7 @@ from piker.brokers._util import ( BrokerError, DataThrottle, ) +from piker.pp import Transaction from . import log # // @@ -290,6 +291,61 @@ class Client: assert count == len(trades_by_id.values()) return trades_by_id + async def get_xfers( + self, + asset: str, + src_asset: str = '', + + ) -> dict[str, Transaction]: + ''' + Get asset balance transfer transactions. + + Currently only withdrawals are supported. + + ''' + xfers: list[dict] = (await self.endpoint( + 'WithdrawStatus', + {'asset': asset}, + ))['result'] + + # eg. resp schema: + # 'result': [{'method': 'Bitcoin', 'aclass': 'currency', 'asset': + # 'XXBT', 'refid': 'AGBJRMB-JHD2M4-NDI3NR', 'txid': + # 'b95d66d3bb6fd76cbccb93f7639f99a505cb20752c62ea0acc093a0e46547c44', + # 'info': 'bc1qc8enqjekwppmw3g80p56z5ns7ze3wraqk5rl9z', + # 'amount': '0.00300726', 'fee': '0.00001000', 'time': + # 1658347714, 'status': 'Success'}]} + + trans: dict[str, Transaction] = {} + for entry in xfers: + # look up the normalized name + asset = self._atable[entry['asset']].lower() + + # XXX: this is in the asset units (likely) so it isn't + # quite the same as a commisions cost necessarily..) + cost = float(entry['fee']) + + tran = Transaction( + fqsn=asset + '.kraken', + tid=entry['txid'], + dt=pendulum.from_timestamp(entry['time']), + bsuid=f'{asset}{src_asset}', + size=-1*( + float(entry['amount']) + + + cost + ), + # since this will be treated as a "sell" it + # shouldn't be needed to compute the be price. + price='NaN', + + # XXX: see note above + cost=0, + ) + trans[tran.tid] = tran + + return trans + async def submit_limit( self, symbol: str, From 8b609f531b6dc2dc517c78404426eb2518f0ee07 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 22 Jul 2022 19:11:29 -0400 Subject: [PATCH 32/48] Add transfers knowledge to positions validation --- piker/brokers/kraken/broker.py | 27 +++++++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 6c56a3e7..ed6b5fc1 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -410,15 +410,38 @@ async def trades_dialogue( if ( not pair or not pp - or not math.isclose(pp.size, size) + or not math.isclose(pp.calc_size(), size) ): return False return pp pos = has_pp(dst) - if not pos: + # get transfers to make sense of abs balances. + likely_pair = { + bsuid[:3]: bsuid + for bsuid in table.pps + }.get(dst) + + if likely_pair: + # this was likely pp that had a withdrawal + # from the dst asset out of the account. + + xfer_trans = await client.get_xfers( + dst, + src_asset=likely_pair[3:], + ) + if xfer_trans: + updated = table.update_from_trans( + xfer_trans, + cost_scalar=1, + ) + log.info( + 'Updated {dst} from transfers:\n' + f'{pformat(updated)}' + ) + # we have a balance for which there is no pp # entry? so we have to likely update from the # ledger. From 0fb31586fd3497edd2630739f91ef1bab98427d2 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 25 Jul 2022 12:02:57 -0400 Subject: [PATCH 33/48] Go back to using `Position.size` property in pp loading audits --- piker/brokers/kraken/broker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index ed6b5fc1..6205e955 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -410,7 +410,7 @@ async def trades_dialogue( if ( not pair or not pp - or not math.isclose(pp.calc_size(), size) + or not math.isclose(pp.size, size) ): return False From 168c9863cbd6aa799eb88300e5fb1893e6b161bf Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Tue, 26 Jul 2022 14:14:57 -0400 Subject: [PATCH 34/48] Look for transfers after ledger + api trans load If we don't have a pos table built out already (in mem) we can't figure out the likely dst asset (since there's no pair entry to guide us) that we should use to search for withdrawal transactions; so move it later. Further this ports to the new api changes in `piker.pp`` that will land with #365. --- piker/brokers/kraken/broker.py | 62 +++++++++++++++++++++------------- 1 file changed, 38 insertions(+), 24 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 6205e955..160bfca1 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -286,7 +286,7 @@ def trades2pps( ) log.info(f'Updated pps:\n{pformat(updated)}') - pp_entries, closed_pp_objs = table.dump_active('kraken') + pp_entries, closed_pp_objs = table.dump_active() pp_objs: dict[Union[str, int], Position] = table.pps pps: dict[int, Position] @@ -418,29 +418,6 @@ async def trades_dialogue( pos = has_pp(dst) if not pos: - # get transfers to make sense of abs balances. - likely_pair = { - bsuid[:3]: bsuid - for bsuid in table.pps - }.get(dst) - - if likely_pair: - # this was likely pp that had a withdrawal - # from the dst asset out of the account. - - xfer_trans = await client.get_xfers( - dst, - src_asset=likely_pair[3:], - ) - if xfer_trans: - updated = table.update_from_trans( - xfer_trans, - cost_scalar=1, - ) - log.info( - 'Updated {dst} from transfers:\n' - f'{pformat(updated)}' - ) # we have a balance for which there is no pp # entry? so we have to likely update from the @@ -454,6 +431,43 @@ async def trades_dialogue( table.update_from_trans(api_trans) pos = has_pp(dst) if not pos: + + # get transfers to make sense of abs balances. + # NOTE: we do this after ledger and API + # loading since we might not have an entry + # in the ``pps.toml`` for the necessary pair + # yet and thus this likely pair grabber will + # likely fail. + likely_pair = { + bsuid[:3]: bsuid + for bsuid in table.pps + }.get(dst) + if not likely_pair: + raise ValueError( + 'Could not find a position pair in ' + 'ledger for likely widthdrawal ' + f'candidate: {dst}' + ) + + if likely_pair: + # this was likely pp that had a withdrawal + # from the dst asset out of the account. + + xfer_trans = await client.get_xfers( + dst, + src_asset=likely_pair[3:], + ) + if xfer_trans: + updated = table.update_from_trans( + xfer_trans, + cost_scalar=1, + ) + log.info( + 'Updated {dst} from transfers:\n' + f'{pformat(updated)}' + ) + + if not has_pp(dst): raise ValueError( 'Could not reproduce balance:\n' f'dst: {dst}, {size}\n' From d43ba47ebe56d4ecdd445a16bbee213d9b4d3805 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Wed, 27 Jul 2022 12:28:22 -0400 Subject: [PATCH 35/48] Renames to `ppu` --- piker/brokers/kraken/README.rst | 2 +- piker/brokers/kraken/broker.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/piker/brokers/kraken/README.rst b/piker/brokers/kraken/README.rst index afeaeb2b..80e56913 100644 --- a/piker/brokers/kraken/README.rst +++ b/piker/brokers/kraken/README.rst @@ -57,7 +57,7 @@ your ``pps.toml`` file will have position entries like, [kraken.spot."xmreur.kraken"] size = 4.80907954 - be_price = 103.97000000 + ppu = 103.97000000 bsuid = "XXMRZEUR" clears = [ { tid = "TFJBKK-SMBZS-VJ4UWS", cost = 0.8, price = 103.97, size = 4.80907954, dt = "2022-05-20T02:26:33.413397+00:00" }, diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 160bfca1..3d7e2d91 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -308,7 +308,7 @@ def trades2pps( account='kraken.' + acctid, symbol=p.symbol.front_fqsn(), size=p.size, - avg_price=p.be_price, + avg_price=p.ppu, currency='', ) position_msgs.append(msg) From e6a3e8b65ac33bd50bac591ab9a0a3cbf667a821 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 29 Jul 2022 12:52:56 -0400 Subject: [PATCH 36/48] Add warning msg for `openOrders.userref` always being 0 --- piker/brokers/kraken/broker.py | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 3d7e2d91..9cb3e9b3 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -188,6 +188,9 @@ async def handle_order_requests( 'event': ep, 'token': token, + # XXX: this seems to always get an error response? + # 'userref': f"'{reqid}'", + 'reqid': reqid, # remapped-to-int uid from ems 'pair': pair, 'price': str(order.price), @@ -711,8 +714,7 @@ async def handle_order_updates( continue case { - # XXX: lol, ws bug, this is always 0! - 'userref': _, + 'userref': reqid, # during a fill this field is **not** # provided! but, it is always avail on @@ -764,7 +766,15 @@ async def handle_order_updates( else: vlm = rest.get('vol_exec', 0) - reqid = reqids2txids.inverse.get(txid) + ourreqid = reqids2txids.inverse.get(txid) + + if ourreqid != reqid: + log.warning( + 'REQID MISMATCH due to kraken api bugs..\n' + f'msg:{reqid}, ours:{ourreqid}' + ) + reqid = ourreqid + oid = ids.inverse.get(reqid) if ( From db564d797704b4d16542871a41694f17b71182a6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 30 Jul 2022 14:27:50 -0400 Subject: [PATCH 37/48] Add casting method to our struct variant --- piker/data/types.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/piker/data/types.py b/piker/data/types.py index c6cba61d..d8926610 100644 --- a/piker/data/types.py +++ b/piker/data/types.py @@ -66,3 +66,10 @@ class Struct( ).decode( msgspec.msgpack.Encoder().encode(self) ) + + def typecast( + self, + # fields: Optional[list[str]] = None, + ) -> None: + for fname, ftype in self.__annotations__.items(): + setattr(self, fname, ftype(getattr(self, fname))) From b3058b8c7886abefe6baf192a663061c223e3b01 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 30 Jul 2022 14:30:48 -0400 Subject: [PATCH 38/48] Drop remaining `pydantic` usage, convert `OHLC` to our struct variant --- piker/brokers/kraken/api.py | 26 -------------------------- piker/brokers/kraken/feed.py | 32 ++++++++++++++++++++++++++++---- 2 files changed, 28 insertions(+), 30 deletions(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index b08d9f52..8bef1995 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -34,7 +34,6 @@ import pendulum import asks from fuzzywuzzy import process as fuzzy import numpy as np -from pydantic.dataclasses import dataclass import urllib.parse import hashlib import hmac @@ -78,31 +77,6 @@ _symbol_info_translation: dict[str, str] = { } -@dataclass -class OHLC: - ''' - Description of the flattened OHLC quote format. - - For schema details see: - https://docs.kraken.com/websockets/#message-ohlc - - ''' - chan_id: int # internal kraken id - chan_name: str # eg. ohlc-1 (name-interval) - pair: str # fx pair - time: float # Begin time of interval, in seconds since epoch - etime: float # End time of interval, in seconds since epoch - open: float # Open price of interval - high: float # High price within interval - low: float # Low price within interval - close: float # Close price of interval - vwap: float # Volume weighted average price within interval - volume: float # Accumulated volume **within interval** - count: int # Number of trades within interval - # (sampled) generated tick data - ticks: list[Any] = field(default_factory=list) - - def get_config() -> dict[str, Any]: conf, path = config.load() diff --git a/piker/brokers/kraken/feed.py b/piker/brokers/kraken/feed.py index e8cfd9b6..0f41a3ec 100644 --- a/piker/brokers/kraken/feed.py +++ b/piker/brokers/kraken/feed.py @@ -19,7 +19,6 @@ Real-time and historical data feed endpoints. ''' from contextlib import asynccontextmanager as acm -from dataclasses import asdict from datetime import datetime from typing import ( Any, @@ -50,7 +49,6 @@ from piker.data._web_bs import open_autorecon_ws, NoBsWs from . import log from .api import ( Client, - OHLC, ) @@ -88,6 +86,30 @@ class Pair(Struct): ordermin: float # minimum order volume for pair +class OHLC(Struct): + ''' + Description of the flattened OHLC quote format. + + For schema details see: + https://docs.kraken.com/websockets/#message-ohlc + + ''' + chan_id: int # internal kraken id + chan_name: str # eg. ohlc-1 (name-interval) + pair: str # fx pair + time: float # Begin time of interval, in seconds since epoch + etime: float # End time of interval, in seconds since epoch + open: float # Open price of interval + high: float # High price within interval + low: float # Low price within interval + close: float # Close price of interval + vwap: float # Volume weighted average price within interval + volume: float # Accumulated volume **within interval** + count: int # Number of trades within interval + # (sampled) generated tick data + ticks: list[Any] = [] + + async def stream_messages( ws: NoBsWs, ): @@ -176,12 +198,14 @@ async def process_data_feed_msgs( pair ]: if 'ohlc' in chan_name: - yield 'ohlc', OHLC( + ohlc = OHLC( chan_id, chan_name, pair, *payload_array[0] ) + ohlc.typecast() + yield 'ohlc', ohlc elif 'spread' in chan_name: @@ -214,7 +238,7 @@ def normalize( ohlc: OHLC, ) -> dict: - quote = asdict(ohlc) + quote = ohlc.to_dict() quote['broker_ts'] = quote['time'] quote['brokerd_ts'] = time.time() quote['symbol'] = quote['pair'] = quote['pair'].replace('/', '') From 44e21b1de9eee4a88ed866974f1fe615190015c6 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 30 Jul 2022 16:52:08 -0400 Subject: [PATCH 39/48] Drop field import --- piker/brokers/kraken/api.py | 1 - 1 file changed, 1 deletion(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index 8bef1995..36b0199a 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -19,7 +19,6 @@ Kraken web API wrapping. ''' from contextlib import asynccontextmanager as acm -from dataclasses import field from datetime import datetime import itertools from typing import ( From 808dbb12e6d17de8b4404b340e98d1d22d80572b Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 31 Jul 2022 14:35:07 -0400 Subject: [PATCH 40/48] Drop forgotten `pydantic` dataclass in binance backend.. --- piker/brokers/binance.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/piker/brokers/binance.py b/piker/brokers/binance.py index add23b18..68c7238e 100644 --- a/piker/brokers/binance.py +++ b/piker/brokers/binance.py @@ -33,7 +33,6 @@ import asks from fuzzywuzzy import process as fuzzy import numpy as np import tractor -from pydantic.dataclasses import dataclass import wsproto from .._cacheables import open_cached_client @@ -106,14 +105,14 @@ class Pair(Struct, frozen=True): permissions: list[str] -@dataclass -class OHLC: - """Description of the flattened OHLC quote format. +class OHLC(Struct): + ''' + Description of the flattened OHLC quote format. For schema details see: https://binance-docs.github.io/apidocs/spot/en/#kline-candlestick-streams - """ + ''' time: int open: float @@ -262,6 +261,7 @@ class Client: for i, bar in enumerate(bars): bar = OHLC(*bar) + bar.typecast() row = [] for j, (name, ftype) in enumerate(_ohlc_dtype[1:]): From dc8072c6dbb022534ea4bb6b0e85f059a861a71a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 29 Jul 2022 19:37:02 -0400 Subject: [PATCH 41/48] WIP: use `userref` field over `reqid`... --- piker/brokers/kraken/broker.py | 72 +++++++++++++++++++++------------- 1 file changed, 44 insertions(+), 28 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 9cb3e9b3..0c2c4531 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -31,6 +31,7 @@ import time from typing import ( Any, AsyncIterator, + Optional, Union, ) @@ -188,10 +189,10 @@ async def handle_order_requests( 'event': ep, 'token': token, - # XXX: this seems to always get an error response? - # 'userref': f"'{reqid}'", + # XXX: Lol, you can only send one of these.. + 'userref': str(reqid), + # 'reqid': reqid, # remapped-to-int uid from ems - 'reqid': reqid, # remapped-to-int uid from ems 'pair': pair, 'price': str(order.price), 'volume': str(order.size), @@ -601,7 +602,8 @@ async def handle_order_updates( # format as tid -> trade event map # eg. received msg format, - # [{'TOKWHY-SMTUB-G5DOI6': {'cost': '95.29047', + # [{'TOKWHY-SMTUB-G5DOI6': { + # 'cost': '95.29047', # 'fee': '0.24776', # 'margin': '0.00000', # 'ordertxid': 'OKSUXM-3OLSB-L7TN72', @@ -611,7 +613,8 @@ async def handle_order_updates( # 'price': '21268.20000', # 'time': '1657990947.640891', # 'type': 'buy', - # 'vol': '0.00448042'}}] + # 'vol': '0.00448042' + # }}] trades = { tid: trade for entry in trades_msgs @@ -622,12 +625,14 @@ async def handle_order_updates( } for tid, trade in trades.items(): txid = trade['ordertxid'] + reqid = trade.get('userref') - # NOTE: yet again, here we don't have any ref to the - # reqid that's generated by us (as the client) and - # sent in the order request, so we have to look it - # up from our own registry... - reqid = reqids2txids.inverse[txid] + if not reqid: + # NOTE: yet again, here we don't have any ref to the + # reqid that's generated by us (as the client) and + # sent in the order request, so we have to look it + # up from our own registry... + reqid = reqids2txids.inverse[txid] action = trade['type'] price = float(trade['price']) @@ -705,11 +710,11 @@ async def handle_order_updates( case { 'cancel_reason': 'Order replaced', 'status': status, - # 'userref': reqid, # XXX: always zero bug XD + 'userref': reqid, # XXX: always zero bug XD # **rest, }: log.info( - f'Order {txid} was replaced' + f'Order {txid}@reqid={reqid} was replaced' ) continue @@ -768,12 +773,17 @@ async def handle_order_updates( ourreqid = reqids2txids.inverse.get(txid) + # XXX: abs necessary in order to enable + # mapping status response messages to the + # reqid-dialog.. + reqids2txids[reqid] = txid + if ourreqid != reqid: log.warning( - 'REQID MISMATCH due to kraken api bugs..\n' + 'REQID MISMATCH due to design mess..\n' f'msg:{reqid}, ours:{ourreqid}' ) - reqid = ourreqid + # reqid = ourreqid oid = ids.inverse.get(reqid) @@ -901,7 +911,7 @@ async def handle_order_updates( case { 'event': etype, 'status': status, - 'reqid': reqid, + # 'reqid': reqid, **rest, } as event if ( etype in { @@ -914,9 +924,14 @@ async def handle_order_updates( f'{etype}:\n' f'{pformat(msg)}' ) - oid = ids.inverse.get(reqid) + + txid = rest.get('txid') + reqid = reqids2txids.inverse.get(txid) + lasttxid = reqids2txids.get(reqid) + # TODO: relay these to EMS once it supports # open order loading. + oid = ids.inverse.get(reqid) if not oid: log.warning( 'Unknown order status update?:\n' @@ -924,18 +939,17 @@ async def handle_order_updates( ) continue - lasttxid = reqids2txids.get(reqid) - txid = rest.get('txid') - - # update the msg chain - chain = apiflows[reqid] - chain.maps.append(event) + if reqid is not None: + # update the msg chain + chain = apiflows[reqid] + chain.maps.append(event) resps, errored = process_status( event, oid, token, chain, + reqid=reqid, ) if resps: for resp in resps: @@ -955,12 +969,13 @@ async def handle_order_updates( 'reqid': reqid or 0, 'txid': [txid], }) - else: + + # else: # XXX: we **must** do this mapping for edit order # status updates since the `openOrders` sub above # never relays back the correct client-side `reqid` # that is put in the order request.. - reqids2txids[reqid] = txid + # reqids2txids[reqid] = txid case _: log.warning(f'Unhandled trades update msg: {msg}') @@ -971,6 +986,7 @@ def process_status( oid: str, token: str, chain: ChainMap, + reqid: Optional[int] = None, ) -> tuple[list[MsgUnion], bool]: ''' @@ -982,7 +998,7 @@ def process_status( case { 'event': etype, 'status': 'error', - 'reqid': reqid, + # 'reqid': reqid, 'errorMessage': errmsg, }: # any of ``{'add', 'edit', 'cancel'}`` @@ -1006,7 +1022,7 @@ def process_status( case { 'event': 'addOrderStatus', 'status': "ok", - 'reqid': reqid, # oid from ems side + # 'reqid': reqid, # oid from ems side 'txid': txid, 'descr': descr, # only on success? }: @@ -1021,7 +1037,7 @@ def process_status( case { 'event': 'editOrderStatus', 'status': "ok", - 'reqid': reqid, # oid from ems side + # 'reqid': reqid, # oid from ems side 'descr': descr, # NOTE: for edit request this is a new value @@ -1040,7 +1056,7 @@ def process_status( case { "event": "cancelOrderStatus", "status": "ok", - 'reqid': reqid, + # 'reqid': reqid, # XXX: sometimes this isn't provided!? # 'txid': txids, From 227a80469ef6b52f52bd0997cd073d8dacfcc685 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sat, 30 Jul 2022 16:32:03 -0400 Subject: [PATCH 42/48] Use both `reqid` and `userref` in order requests Turns out you can pass both thus making mapping an ems `oid` to a brokerd-side `reqid` much more simple. This allows us to avoid keeping as much local dialog state but with still the following caveats: - ok `editOrder` msgs must update the reqid<->txid map - only pop `reqids2txids` entries inside the `cancelOrderStatus` handler --- piker/brokers/kraken/broker.py | 138 +++++++++++++++++++++------------ 1 file changed, 90 insertions(+), 48 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 0c2c4531..d1397e62 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -31,7 +31,6 @@ import time from typing import ( Any, AsyncIterator, - Optional, Union, ) @@ -105,7 +104,7 @@ async def handle_order_requests( # XXX: UGH, let's unify this.. with ``msgspec``. msg: dict[str, Any] order: BrokerdOrder - counter = count() + counter = count(1) async for msg in ems_order_stream: log.info(f'Rx order msg:\n{pformat(msg)}') @@ -139,7 +138,8 @@ async def handle_order_requests( ep = 'editOrder' reqid = ids[order.oid] # integer not txid try: - txid = reqids2txids.pop(reqid) + # txid = reqids2txids.pop(reqid) + txid = reqids2txids[reqid] except KeyError: reqids2txids[reqid] = TooFastEdit(reqid) await ems_order_stream.send( @@ -152,10 +152,11 @@ async def handle_order_requests( ) ) - - extra = { - 'orderid': txid, # txid - } + else: + extra = { + 'orderid': txid, # txid + # 'newuserref': reqid, + } else: ep = 'addOrder' @@ -189,9 +190,16 @@ async def handle_order_requests( 'event': ep, 'token': token, - # XXX: Lol, you can only send one of these.. + 'reqid': reqid, # remapped-to-int uid from ems + # XXX: we set these to the same value since for us + # a request dialog and an order's state-liftime are + # treated the same. Also this used to not work, the + # values used to be mutex for some odd reason until + # we dealt with support about it, and then they + # fixed it and pretended like we were crazy and the + # issue was never there lmao... coorps bro. + # 'userref': str(reqid), 'userref': str(reqid), - # 'reqid': reqid, # remapped-to-int uid from ems 'pair': pair, 'price': str(order.price), @@ -633,6 +641,8 @@ async def handle_order_updates( # sent in the order request, so we have to look it # up from our own registry... reqid = reqids2txids.inverse[txid] + if not reqid: + log.warning(f'Unknown trade dialog: {txid}') action = trade['type'] price = float(trade['price']) @@ -713,6 +723,9 @@ async def handle_order_updates( 'userref': reqid, # XXX: always zero bug XD # **rest, }: + # TODO: + # - put the edit order status update code here. + # - send open order status msg. log.info( f'Order {txid}@reqid={reqid} was replaced' ) @@ -771,28 +784,52 @@ async def handle_order_updates( else: vlm = rest.get('vol_exec', 0) + # XXX: keep kraken engine's ``txid`` synced + # with the ems dialog's ``reqid``. ourreqid = reqids2txids.inverse.get(txid) + if reqid > 0: + if ourreqid is None: + log.info( + 'Mapping new txid to our reqid:\n' + f'{reqid} -> {txid}' + ) + reqids2txids[reqid] = txid - # XXX: abs necessary in order to enable - # mapping status response messages to the - # reqid-dialog.. - reqids2txids[reqid] = txid - - if ourreqid != reqid: - log.warning( - 'REQID MISMATCH due to design mess..\n' - f'msg:{reqid}, ours:{ourreqid}' - ) - # reqid = ourreqid + else: + # NOTE: if is to hack around edit order not + # realying userref field + reqid = ourreqid oid = ids.inverse.get(reqid) if ( status == 'open' and ( + # XXX: too fast edit handled by the + # request handler task: this + # scenario occurs when ems side + # requests are coming in too quickly + # such that there is no known txid + # yet established for the ems + # dialog's last reqid when the + # request handler task is already + # receceiving a new update for that + # reqid. In this case we simply mark + # the reqid as being "too fast" and + # then when we get the next txid + # update from kraken's backend, and + # thus the new txid, we simply + # cancel the order for now. - # TOO fast edit handled by the - # request handler task. + # TODO: Ideally we eventually + # instead make the client side of + # the ems block until a submission + # is confirmed by the backend + # instead of this hacky throttle + # style approach and avoid requests + # coming in too quickly on the other + # side of the ems, aka the client + # <-> ems dialog. (toofast := isinstance( reqids2txids.get(reqid), TooFastEdit @@ -869,6 +906,7 @@ async def handle_order_updates( # there is no `status` field case { 'vol_exec': vlm, + 'userref': reqid, **rest, }: # eg. fill msg contents (in total): @@ -880,7 +918,8 @@ async def handle_order_updates( # 'userref': 0, # } # TODO: emit fill msg from here - reqid = reqids2txids.inverse[txid] + ourreqid = reqids2txids.inverse[txid] + assert reqid == ourreqid log.info( f'openOrders vlm={vlm} Fill for {reqid}:\n' f'{update_msg}' @@ -899,19 +938,21 @@ async def handle_order_updates( # need them because that sub seems to have a bug where the # `userref` field is always 0 instead of our generated reqid # value... - # Not sure why kraken devs decided to repeat themselves but - # it almost seems as though we could drop this entire sub - # and get everything we need by just parsing msgs correctly - # above? The only reason for this seems to be remapping + # SOLVED: pass both a reqid and a userref in the init + # request msg. + + # NOTE: The only reason for this seems to be remapping # underlying `txid` values on order "edits" which the # `openOrders` sub doesn't seem to have any knowledge of. + # I'd also like to ask them which event guarantees that the # the live order is now in the book, since these status ones # almost seem more like request-acks then state guarantees. + # ANSWER the `openOrders` is more indicative of "liveness". case { 'event': etype, 'status': status, - # 'reqid': reqid, + 'reqid': reqid, **rest, } as event if ( etype in { @@ -926,8 +967,8 @@ async def handle_order_updates( ) txid = rest.get('txid') - reqid = reqids2txids.inverse.get(txid) lasttxid = reqids2txids.get(reqid) + print(f'txids: {(txid, lasttxid)}') # TODO: relay these to EMS once it supports # open order loading. @@ -939,23 +980,23 @@ async def handle_order_updates( ) continue - if reqid is not None: - # update the msg chain - chain = apiflows[reqid] - chain.maps.append(event) + # if reqid is not None: + # update the msg chain + chain = apiflows[reqid] + chain.maps.append(event) resps, errored = process_status( event, oid, token, chain, - reqid=reqid, + reqids2txids, ) if resps: for resp in resps: await ems_stream.send(resp) - if txid: + if txid or lasttxid: if ( isinstance(lasttxid, TooFastEdit) or errored @@ -969,14 +1010,6 @@ async def handle_order_updates( 'reqid': reqid or 0, 'txid': [txid], }) - - # else: - # XXX: we **must** do this mapping for edit order - # status updates since the `openOrders` sub above - # never relays back the correct client-side `reqid` - # that is put in the order request.. - # reqids2txids[reqid] = txid - case _: log.warning(f'Unhandled trades update msg: {msg}') @@ -986,7 +1019,7 @@ def process_status( oid: str, token: str, chain: ChainMap, - reqid: Optional[int] = None, + reqids2txids: dict[int, str], ) -> tuple[list[MsgUnion], bool]: ''' @@ -998,7 +1031,7 @@ def process_status( case { 'event': etype, 'status': 'error', - # 'reqid': reqid, + 'reqid': reqid, 'errorMessage': errmsg, }: # any of ``{'add', 'edit', 'cancel'}`` @@ -1022,7 +1055,7 @@ def process_status( case { 'event': 'addOrderStatus', 'status': "ok", - # 'reqid': reqid, # oid from ems side + 'reqid': reqid, # oid from ems side 'txid': txid, 'descr': descr, # only on success? }: @@ -1037,7 +1070,7 @@ def process_status( case { 'event': 'editOrderStatus', 'status': "ok", - # 'reqid': reqid, # oid from ems side + 'reqid': reqid, # oid from ems side 'descr': descr, # NOTE: for edit request this is a new value @@ -1050,13 +1083,19 @@ def process_status( f'txid: {origtxid} -> {txid}\n' f'{descr}' ) + + # XXX: update the expected txid since the ``openOrders`` sub + # doesn't relay through the ``userref`` value.. + # (hopefully kraken will fix this so we don't need this + # line.) + reqids2txids[reqid] = txid # deliver another ack to update the ems-side `.reqid`. return [], False case { "event": "cancelOrderStatus", "status": "ok", - # 'reqid': reqid, + 'reqid': reqid, # XXX: sometimes this isn't provided!? # 'txid': txids, @@ -1067,6 +1106,9 @@ def process_status( f'Cancelling order {oid}[requid={reqid}]:\n' f'brokerd reqid: {reqid}\n' ) + if txid == reqids2txids[reqid]: + reqids2txids.pop(reqid) + return [], False From 1cbf45b4c4a49431ddb708caf287debc90132bc9 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Sun, 31 Jul 2022 14:29:33 -0400 Subject: [PATCH 43/48] Use the ``newuserref`` field on order edits Why we need so many fields to accomplish passing through a dialog key to orders is beyond me but this is how they do it with edits.. Allows not having to handle `editOrderStatus` msgs to update the dialog key table and instead just do it in the `openOrders` sub by checking the canceled msg for a 'cancel_reason' of 'Order replaced', in which case we just pop the txid and wait for the new order the kraken backend engine will submit automatically, which will now have the correct 'userref' value we passed in via the `newuserref`, and then we add that new `txid` to our table. --- piker/brokers/kraken/broker.py | 282 ++++++++++++++++----------------- 1 file changed, 137 insertions(+), 145 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index d1397e62..8a6fa84c 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -138,9 +138,13 @@ async def handle_order_requests( ep = 'editOrder' reqid = ids[order.oid] # integer not txid try: - # txid = reqids2txids.pop(reqid) txid = reqids2txids[reqid] except KeyError: + assert 0 + + # XXX: not sure if this block ever gets hit now? + log.error('TOO FAST EDIT') + reqids2txids[reqid] = TooFastEdit(reqid) await ems_order_stream.send( BrokerdError( @@ -155,7 +159,7 @@ async def handle_order_requests( else: extra = { 'orderid': txid, # txid - # 'newuserref': reqid, + 'newuserref': str(reqid), } else: @@ -200,15 +204,12 @@ async def handle_order_requests( # issue was never there lmao... coorps bro. # 'userref': str(reqid), 'userref': str(reqid), - 'pair': pair, 'price': str(order.price), 'volume': str(order.size), - - # only ensures request is valid, nothing more - # validate: 'true', - + # validate: 'true', # validity check, nothing more } | extra + log.info(f'Submitting WS order request:\n{pformat(req)}') await ws.send_msg(req) @@ -229,9 +230,7 @@ async def handle_order_requests( symbol=msg['symbol'], reason=( 'Invalid request msg:\n{msg}' - ), - - ) + )) ) @@ -253,7 +252,6 @@ async def subscribe( ''' # more specific logic for this in kraken's sync client: # https://github.com/krakenfx/kraken-wsclient-py/blob/master/kraken_wsclient_py/kraken_wsclient_py.py#L188 - assert token for sub in subs: msg = { @@ -499,16 +497,9 @@ async def trades_dialogue( ) await ctx.started((ppmsgs, [acc_name])) - # XXX: not fucking clue but putting this finally block - # will suppress errors inside the direct await below!?! - # likely something to do with the exist stack inside - # the nobsws stuff... - # try: - # Get websocket token for authenticated data stream # Assert that a token was actually received. resp = await client.endpoint('GetWebSocketsToken', {}) - err = resp.get('error') if err: raise BrokerError(err) @@ -582,18 +573,37 @@ async def handle_order_updates( defined in the signature clear to the reader. ''' - async for msg in ws_stream: match msg: - # process and relay clearing trade events to ems - # https://docs.kraken.com/websockets/#message-ownTrades + # TODO: turns out you get the fill events from the # `openOrders` before you get this, so it might be better # to do all fill/status/pp updates in that sub and just use # this one for ledger syncs? + + # XXX: ASK SUPPORT ABOUT THIS! + # For eg. we could take the "last 50 trades" and do a diff # with the ledger and then only do a re-sync if something # seems amiss? + + # process and relay clearing trade events to ems + # https://docs.kraken.com/websockets/#message-ownTrades + # format as tid -> trade event map + # eg. received msg format, + # [{'TOKWHY-SMTUB-G5DOI6': { + # 'cost': '95.29047', + # 'fee': '0.24776', + # 'margin': '0.00000', + # 'ordertxid': 'OKSUXM-3OLSB-L7TN72', + # 'ordertype': 'limit', + # 'pair': 'XBT/EUR', + # 'postxid': 'TKH2SE-M7IF5-CFI7LT', + # 'price': '21268.20000', + # 'time': '1657990947.640891', + # 'type': 'buy', + # 'vol': '0.00448042' + # }}] case [ trades_msgs, 'ownTrades', @@ -603,26 +613,6 @@ async def handle_order_updates( f'ownTrades update_{seq}:\n' f'{pformat(trades_msgs)}' ) - # XXX: a fix / todo - # see the comment in the caller about weird error - # suppression around a commented `try:` - # assert 0 - - # format as tid -> trade event map - # eg. received msg format, - # [{'TOKWHY-SMTUB-G5DOI6': { - # 'cost': '95.29047', - # 'fee': '0.24776', - # 'margin': '0.00000', - # 'ordertxid': 'OKSUXM-3OLSB-L7TN72', - # 'ordertype': 'limit', - # 'pair': 'XBT/EUR', - # 'postxid': 'TKH2SE-M7IF5-CFI7LT', - # 'price': '21268.20000', - # 'time': '1657990947.640891', - # 'type': 'buy', - # 'vol': '0.00448042' - # }}] trades = { tid: trade for entry in trades_msgs @@ -665,7 +655,7 @@ async def handle_order_updates( ) await ems_stream.send(fill_msg) - filled_msg = BrokerdStatus( + status_msg = BrokerdStatus( reqid=reqid, time_ns=time.time_ns(), @@ -687,7 +677,7 @@ async def handle_order_updates( # https://github.com/pikers/piker/issues/296 remaining=0, ) - await ems_stream.send(filled_msg) + await ems_stream.send(status_msg) new_trans = norm_trade_records(trades) ppmsgs = trades2pps( @@ -715,22 +705,31 @@ async def handle_order_updates( txid, update_msg = list(order_msg.items())[0] match update_msg: - # we ignore internal order updates triggered by - # kraken's "edit" endpoint. - case { - 'cancel_reason': 'Order replaced', - 'status': status, - 'userref': reqid, # XXX: always zero bug XD - # **rest, - }: - # TODO: - # - put the edit order status update code here. - # - send open order status msg. - log.info( - f'Order {txid}@reqid={reqid} was replaced' - ) - continue - + # XXX: eg. of full msg schema: + # {'avg_price': _, + # 'cost': _, + # 'descr': { + # 'close': None, + # 'leverage': None, + # 'order': descr, + # 'ordertype': 'limit', + # 'pair': 'XMR/EUR', + # 'price': '74.94000000', + # 'price2': '0.00000000', + # 'type': 'buy' + # }, + # 'expiretm': None, + # 'fee': '0.00000000', + # 'limitprice': '0.00000000', + # 'misc': '', + # 'oflags': 'fciq', + # 'opentm': '1656966131.337344', + # 'refid': None, + # 'starttm': None, + # 'stopprice': '0.00000000', + # 'timeinforce': 'GTC', + # 'vol': submit_vlm, # '13.34400854', + # 'vol_exec': exec_vlm} # 0.0000 case { 'userref': reqid, @@ -739,42 +738,7 @@ async def handle_order_updates( # actual status updates.. see case above. 'status': status, **rest, - - # XXX: eg. of remaining msg schema: - # 'avg_price': _, - # 'cost': _, - # 'descr': { - # 'close': None, - # 'leverage': None, - # 'order': descr, - # 'ordertype': 'limit', - # 'pair': 'XMR/EUR', - # 'price': '74.94000000', - # 'price2': '0.00000000', - # 'type': 'buy' - # }, - # 'expiretm': None, - # 'fee': '0.00000000', - # 'limitprice': '0.00000000', - # 'misc': '', - # 'oflags': 'fciq', - # 'opentm': '1656966131.337344', - # 'refid': None, - # 'starttm': None, - # 'stopprice': '0.00000000', - # 'timeinforce': 'GTC', - # 'vol': submit_vlm, # '13.34400854', - # 'vol_exec': exec_vlm, # 0.0000 }: - ems_status = { - 'open': 'submitted', - 'closed': 'filled', - 'canceled': 'cancelled', - # do we even need to forward - # this state to the ems? - 'pending': 'pending', - }[status] - # TODO: store this in a ChainMap instance # per order dialog. # submit_vlm = rest.get('vol', 0) @@ -784,21 +748,39 @@ async def handle_order_updates( else: vlm = rest.get('vol_exec', 0) - # XXX: keep kraken engine's ``txid`` synced - # with the ems dialog's ``reqid``. - ourreqid = reqids2txids.inverse.get(txid) - if reqid > 0: + if status == 'canceled': + reqids2txids.pop(reqid) + + # we specially ignore internal order + # updates triggered by kraken's "edit" + # endpoint. + if rest['cancel_reason'] == 'Order replaced': + # TODO: + # - put the edit order status update + # code here? + # - send open order status msg. + log.info( + f'Order replaced: {txid}@reqid={reqid}' + ) + + # we don't do normal msg emission on + # a replacement cancel since it's + # the result of an "edited order" + # and thus we mask the kraken + # backend cancel then create details + # from the ems side. + continue + else: + # XXX: keep kraken engine's ``txid`` synced + # with the ems dialog's ``reqid``. + reqids2txids[reqid] = txid + + ourreqid = reqids2txids.inverse.get(txid) if ourreqid is None: log.info( 'Mapping new txid to our reqid:\n' f'{reqid} -> {txid}' ) - reqids2txids[reqid] = txid - - else: - # NOTE: if is to hack around edit order not - # realying userref field - reqid = ourreqid oid = ids.inverse.get(reqid) @@ -845,7 +827,7 @@ async def handle_order_updates( # by not moving the client side line # until an edit confirmation # arrives... - log.warning( + log.cancel( f'Received too fast edit {txid}:\n' f'{update_msg}\n' 'Cancelling order for now!..' @@ -855,7 +837,7 @@ async def handle_order_updates( # TODO: handle these and relay them # through the EMS to the client / UI # side! - log.warning( + log.cancel( f'Rx unknown active order {txid}:\n' f'{update_msg}\n' 'Cancelling order for now!..' @@ -871,6 +853,18 @@ async def handle_order_updates( }) continue + # remap statuses to ems set. + ems_status = { + 'open': 'submitted', + 'closed': 'filled', + 'canceled': 'cancelled', + # do we even need to forward + # this state to the ems? + 'pending': 'pending', + }[status] + # TODO: i like the open / closed semantics + # more we should consider them for internals + # send BrokerdStatus messages for all # order state updates resp = BrokerdStatus( @@ -902,22 +896,22 @@ async def handle_order_updates( apiflows[reqid].maps.append(update_msg) await ems_stream.send(resp) - # fill event. - # there is no `status` field + # fill msg. + # eg. contents (in total): + # { + # 'vol_exec': '0.84709869', + # 'cost': '101.25370642', + # 'fee': '0.26325964', + # 'avg_price': '119.53000001', + # 'userref': 0, + # } + # NOTE: there is no `status` field case { 'vol_exec': vlm, 'userref': reqid, **rest, }: - # eg. fill msg contents (in total): - # { - # 'vol_exec': '0.84709869', - # 'cost': '101.25370642', - # 'fee': '0.26325964', - # 'avg_price': '119.53000001', - # 'userref': 0, - # } - # TODO: emit fill msg from here + # TODO: emit fill msg from here? ourreqid = reqids2txids.inverse[txid] assert reqid == ourreqid log.info( @@ -932,18 +926,12 @@ async def handle_order_updates( f'{txid}:{order_msg}' ) - # TODO: given the 'openOrders' sub , pretty - # much all the msgs we get for this sub are duplicate - # of the (incremental) updates in that one though we still - # need them because that sub seems to have a bug where the - # `userref` field is always 0 instead of our generated reqid - # value... - # SOLVED: pass both a reqid and a userref in the init - # request msg. - # NOTE: The only reason for this seems to be remapping # underlying `txid` values on order "edits" which the # `openOrders` sub doesn't seem to have any knowledge of. + # UPDATE: seems like we don't need this any more thanks to + # passing through the dialog key / reqid in the `newuserref` + # field on edit requests. # I'd also like to ask them which event guarantees that the # the live order is now in the book, since these status ones @@ -968,7 +956,6 @@ async def handle_order_updates( txid = rest.get('txid') lasttxid = reqids2txids.get(reqid) - print(f'txids: {(txid, lasttxid)}') # TODO: relay these to EMS once it supports # open order loading. @@ -980,7 +967,6 @@ async def handle_order_updates( ) continue - # if reqid is not None: # update the msg chain chain = apiflows[reqid] chain.maps.append(event) @@ -992,24 +978,30 @@ async def handle_order_updates( chain, reqids2txids, ) + if resps: for resp in resps: await ems_stream.send(resp) - if txid or lasttxid: - if ( - isinstance(lasttxid, TooFastEdit) - or errored - ): - # client was editting too quickly - # so we instead cancel this order - log.cancel(f'Cancelling order for {reqid}@{txid}') - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid or 0, - 'txid': [txid], - }) + txid = txid or lasttxid + if ( + # errored likely on a rate limit or bad input + errored + and txid + + # we throttle too-fast-requests on the ems side + or (txid and isinstance(txid, TooFastEdit)) + ): + # client was editting too quickly + # so we instead cancel this order + log.cancel( + f'Cancelling {reqid}@{txid} due to error:\n {event}') + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid or 0, + 'txid': [txid], + }) case _: log.warning(f'Unhandled trades update msg: {msg}') @@ -1088,7 +1080,7 @@ def process_status( # doesn't relay through the ``userref`` value.. # (hopefully kraken will fix this so we don't need this # line.) - reqids2txids[reqid] = txid + # reqids2txids[reqid] = txid # deliver another ack to update the ems-side `.reqid`. return [], False @@ -1106,8 +1098,8 @@ def process_status( f'Cancelling order {oid}[requid={reqid}]:\n' f'brokerd reqid: {reqid}\n' ) - if txid == reqids2txids[reqid]: - reqids2txids.pop(reqid) + # if txid == reqids2txids[reqid]: + # reqids2txids.pop(reqid) return [], False From 69e501764a9ed6fefa4f0dd6880af88e32daa61a Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 1 Aug 2022 14:08:45 -0400 Subject: [PATCH 44/48] Drop status event processing at large Since we figured out how to pass through ems dialog ids to the `openOrders` sub we don't really need to do much with status updates other then error handling. This drops `process_status()` and moves the error handling logic into a status handler sub-block; we now just info-log status updates for troubleshooting purposes. --- piker/brokers/kraken/broker.py | 162 +++++++-------------------------- piker/data/_ahab.py | 40 +++++--- 2 files changed, 59 insertions(+), 143 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 8a6fa84c..ac22c8df 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -140,11 +140,8 @@ async def handle_order_requests( try: txid = reqids2txids[reqid] except KeyError: - assert 0 - # XXX: not sure if this block ever gets hit now? log.error('TOO FAST EDIT') - reqids2txids[reqid] = TooFastEdit(reqid) await ems_order_stream.send( BrokerdError( @@ -971,139 +968,46 @@ async def handle_order_updates( chain = apiflows[reqid] chain.maps.append(event) - resps, errored = process_status( - event, - oid, - token, - chain, - reqids2txids, - ) + if status == 'error': + # any of ``{'add', 'edit', 'cancel'}`` + action = etype.removesuffix('OrderStatus') + errmsg = rest['errorMessage'] + log.error( + f'Failed to {action} order {reqid}:\n' + f'{errmsg}' + ) + await ems_stream.send(BrokerdError( + oid=oid, + # XXX: use old reqid in case it changed? + reqid=reqid, + symbol=chain.get('symbol', 'N/A'), - if resps: - for resp in resps: - await ems_stream.send(resp) + reason=f'Failed {action}:\n{errmsg}', + broker_details=event + )) - txid = txid or lasttxid - if ( - # errored likely on a rate limit or bad input - errored - and txid + txid = txid or lasttxid + if ( + txid + + # we throttle too-fast-requests on the ems side + or (txid and isinstance(txid, TooFastEdit)) + ): + # client was editting too quickly + # so we instead cancel this order + log.cancel( + f'Cancelling {reqid}@{txid} due to:\n {event}') + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid or 0, + 'txid': [txid], + }) - # we throttle too-fast-requests on the ems side - or (txid and isinstance(txid, TooFastEdit)) - ): - # client was editting too quickly - # so we instead cancel this order - log.cancel( - f'Cancelling {reqid}@{txid} due to error:\n {event}') - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid or 0, - 'txid': [txid], - }) case _: log.warning(f'Unhandled trades update msg: {msg}') -def process_status( - event: dict[str, str], - oid: str, - token: str, - chain: ChainMap, - reqids2txids: dict[int, str], - -) -> tuple[list[MsgUnion], bool]: - ''' - Process `'[add/edit/cancel]OrderStatus'` events by translating to - and returning the equivalent EMS-msg responses. - - ''' - match event: - case { - 'event': etype, - 'status': 'error', - 'reqid': reqid, - 'errorMessage': errmsg, - }: - # any of ``{'add', 'edit', 'cancel'}`` - action = etype.removesuffix('OrderStatus') - log.error( - f'Failed to {action} order {reqid}:\n' - f'{errmsg}' - ) - resp = BrokerdError( - oid=oid, - # XXX: use old reqid in case it changed? - reqid=reqid, - symbol=chain.get('symbol', 'N/A'), - - reason=f'Failed {action}:\n{errmsg}', - broker_details=event - ) - return [resp], True - - # successful request cases - case { - 'event': 'addOrderStatus', - 'status': "ok", - 'reqid': reqid, # oid from ems side - 'txid': txid, - 'descr': descr, # only on success? - }: - log.info( - f'Submitted order: {descr}\n' - f'ems oid: {oid}\n' - f'brokerd reqid: {reqid}\n' - f'txid: {txid}\n' - ) - return [], False - - case { - 'event': 'editOrderStatus', - 'status': "ok", - 'reqid': reqid, # oid from ems side - 'descr': descr, - - # NOTE: for edit request this is a new value - 'txid': txid, - 'originaltxid': origtxid, - }: - log.info( - f'Editting order {oid}[requid={reqid}]:\n' - f'brokerd reqid: {reqid}\n' - f'txid: {origtxid} -> {txid}\n' - f'{descr}' - ) - - # XXX: update the expected txid since the ``openOrders`` sub - # doesn't relay through the ``userref`` value.. - # (hopefully kraken will fix this so we don't need this - # line.) - # reqids2txids[reqid] = txid - # deliver another ack to update the ems-side `.reqid`. - return [], False - - case { - "event": "cancelOrderStatus", - "status": "ok", - 'reqid': reqid, - - # XXX: sometimes this isn't provided!? - # 'txid': txids, - **rest, - }: - for txid in rest.get('txid', [chain['reqid']]): - log.info( - f'Cancelling order {oid}[requid={reqid}]:\n' - f'brokerd reqid: {reqid}\n' - ) - # if txid == reqids2txids[reqid]: - # reqids2txids.pop(reqid) - - return [], False - - def norm_trade_records( ledger: dict[str, Any], diff --git a/piker/data/_ahab.py b/piker/data/_ahab.py index 6910d206..218d46e0 100644 --- a/piker/data/_ahab.py +++ b/piker/data/_ahab.py @@ -39,7 +39,11 @@ from docker.errors import ( APIError, # ContainerError, ) -from requests.exceptions import ConnectionError, ReadTimeout +import requests +from requests.exceptions import ( + ConnectionError, + ReadTimeout, +) from ..log import get_logger, get_console_log from .. import config @@ -188,13 +192,12 @@ class Container: def hard_kill(self, start: float) -> None: delay = time.time() - start - log.error( - f'Failed to kill container {self.cntr.id} after {delay}s\n' - 'sending SIGKILL..' - ) # get out the big guns, bc apparently marketstore # doesn't actually know how to terminate gracefully # :eyeroll:... + log.error( + f'SIGKILL-ing: {self.cntr.id} after {delay}s\n' + ) self.try_signal('SIGKILL') self.cntr.wait( timeout=3, @@ -218,20 +221,25 @@ class Container: self.try_signal('SIGINT') start = time.time() - for _ in range(30): + for _ in range(6): with trio.move_on_after(0.5) as cs: - cs.shield = True log.cancel('polling for CNTR logs...') try: await self.process_logs_until(stop_msg) except ApplicationLogError: hard_kill = True + else: + # if we aren't cancelled on above checkpoint then we + # assume we read the expected stop msg and + # terminated. + break - # if we aren't cancelled on above checkpoint then we - # assume we read the expected stop msg and terminated. - break + if cs.cancelled_caught: + # on timeout just try a hard kill after + # a quick container sync-wait. + hard_kill = True try: log.info(f'Polling for container shutdown:\n{cid}') @@ -254,9 +262,16 @@ class Container: except ( docker.errors.APIError, ConnectionError, + requests.exceptions.ConnectionError, + trio.Cancelled, ): log.exception('Docker connection failure') self.hard_kill(start) + raise + + except trio.Cancelled: + log.exception('trio cancelled...') + self.hard_kill(start) else: hard_kill = True @@ -305,16 +320,13 @@ async def open_ahabd( )) try: - # TODO: we might eventually want a proxy-style msg-prot here # to allow remote control of containers without needing # callers to have root perms? await trio.sleep_forever() finally: - # needed? - with trio.CancelScope(shield=True): - await cntr.cancel(stop_msg) + await cntr.cancel(stop_msg) async def start_ahab( From 1a291939c3a59aae196f1054da074b98bd564d4c Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 1 Aug 2022 16:55:04 -0400 Subject: [PATCH 45/48] Drop subs ack handling from streamer --- piker/brokers/kraken/feed.py | 13 +------------ 1 file changed, 1 insertion(+), 12 deletions(-) diff --git a/piker/brokers/kraken/feed.py b/piker/brokers/kraken/feed.py index 0f41a3ec..7c589d85 100644 --- a/piker/brokers/kraken/feed.py +++ b/piker/brokers/kraken/feed.py @@ -152,19 +152,8 @@ async def stream_messages( continue - case { - 'connectionID': _, - 'event': 'systemStatus', - 'status': 'online', - 'version': _, - } as msg: - log.info( - 'WS connection is up:\n' - f'{msg}' - ) - continue - case _: + # passthrough sub msgs yield msg From 30bcfdcc8373999149ac8258fc4d039bd3cb495e Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Mon, 1 Aug 2022 19:22:31 -0400 Subject: [PATCH 46/48] Emit fills from `openOrders` block The (partial) fills from this sub are most indicative of clears (also says support) whereas the msgs in the `ownTrades` sub are only emitted after the entire order request has completed - there is no size-vlm remaining. Further enhancements: - this also includes proper subscription-syncing inside `subscribe()` with a small pre-msg-loop which waits on ack-msgs for each sub and raises any errors. This approach should probably be implemented for the data feed streams as well. - configure the `ownTrades` sub to not bother sending historical data on startup. - make the `openOrders` sub include rate limit counters. - handle the rare case where the ems is trying to cancel an order which was just edited and hasn't yet had it's new `txid` registered. --- piker/brokers/kraken/broker.py | 139 ++++++++++++++++++++++----------- 1 file changed, 93 insertions(+), 46 deletions(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index ac22c8df..25c8c1a7 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -114,16 +114,32 @@ async def handle_order_requests( }: cancel = BrokerdCancel(**msg) reqid = ids[cancel.oid] - txid = reqids2txids[reqid] - # call ws api to cancel: - # https://docs.kraken.com/websockets/#message-cancelOrder - await ws.send_msg({ - 'event': 'cancelOrder', - 'token': token, - 'reqid': reqid, - 'txid': [txid], # should be txid from submission - }) + try: + txid = reqids2txids[reqid] + except KeyError: + # XXX: not sure if this block ever gets hit now? + log.error('TOO FAST CANCEL/EDIT') + reqids2txids[reqid] = TooFastEdit(reqid) + await ems_order_stream.send( + BrokerdError( + oid=msg['oid'], + symbol=msg['symbol'], + reason=( + f'TooFastEdit reqid:{reqid}, could not cancelling..' + ), + + ) + ) + else: + # call ws api to cancel: + # https://docs.kraken.com/websockets/#message-cancelOrder + await ws.send_msg({ + 'event': 'cancelOrder', + 'token': token, + 'reqid': reqid, + 'txid': [txid], # should be txid from submission + }) case { 'account': 'kraken.spot' as account, @@ -235,9 +251,17 @@ async def handle_order_requests( async def subscribe( ws: wsproto.WSConnection, token: str, - subs: list[str] = [ - 'ownTrades', - 'openOrders', + subs: list[tuple[str, dict]] = [ + ('ownTrades', { + # don't send first 50 trades on startup, + # we already pull this manually from the rest endpoint. + 'snapshot': False, + },), + ('openOrders', { + # include rate limit counters + 'ratecounter': True, + },), + ], ): ''' @@ -250,12 +274,15 @@ async def subscribe( # more specific logic for this in kraken's sync client: # https://github.com/krakenfx/kraken-wsclient-py/blob/master/kraken_wsclient_py/kraken_wsclient_py.py#L188 assert token - for sub in subs: + subnames: set[str] = set() + + for name, sub_opts in subs: msg = { 'event': 'subscribe', 'subscription': { - 'name': sub, + 'name': name, 'token': token, + **sub_opts, } } @@ -264,7 +291,34 @@ async def subscribe( # since internally the ws methods appear to be FIFO # locked. await ws.send_msg(msg) + subnames.add(name) + # wait on subscriptionn acks + with trio.move_on_after(5): + while True: + match (msg := await ws.recv_msg()): + case { + 'event': 'subscriptionStatus', + 'status': 'subscribed', + 'subscription': sub_opts, + } as msg: + log.info( + f'Sucessful subscribe for {sub_opts}:\n' + f'{pformat(msg)}' + ) + subnames.remove(sub_opts['name']) + if not subnames: + break + + case { + 'event': 'subscriptionStatus', + 'status': 'error', + 'errorMessage': errmsg, + } as msg: + raise RuntimeError( + f'{errmsg}\n\n' + f'{pformat(msg)}' + ) yield for sub in subs: @@ -616,9 +670,10 @@ async def handle_order_updates( for (tid, trade) in entry.items() # don't re-process datums we've already seen - if tid not in ledger_trans + # if tid not in ledger_trans } for tid, trade in trades.items(): + assert tid not in ledger_trans txid = trade['ordertxid'] reqid = trade.get('userref') @@ -636,22 +691,8 @@ async def handle_order_updates( size = float(trade['vol']) broker_time = float(trade['time']) - # send a fill msg for gui update - fill_msg = BrokerdFill( - time_ns=time.time_ns(), - reqid=reqid, - - action=action, - size=size, - price=price, - - # TODO: maybe capture more msg data - # i.e fees? - broker_details={'name': 'kraken'} | trade, - broker_time=broker_time - ) - await ems_stream.send(fill_msg) - + # TODO: we can emit this on the "closed" state in + # the `openOrders` sub-block below. status_msg = BrokerdStatus( reqid=reqid, time_ns=time.time_ns(), @@ -905,17 +946,33 @@ async def handle_order_updates( # NOTE: there is no `status` field case { 'vol_exec': vlm, + 'avg_price': price, 'userref': reqid, **rest, - }: - # TODO: emit fill msg from here? + } as msg: + ourreqid = reqids2txids.inverse[txid] assert reqid == ourreqid log.info( f'openOrders vlm={vlm} Fill for {reqid}:\n' f'{update_msg}' ) - continue + + fill_msg = BrokerdFill( + time_ns=time.time_ns(), + reqid=reqid, + + # action=action, # just use size value + # for now? + size=vlm, + price=price, + + # TODO: maybe capture more msg data + # i.e fees? + broker_details={'name': 'kraken'} | trade, + broker_time=broker_time + ) + await ems_stream.send(fill_msg) case _: log.warning( @@ -923,17 +980,7 @@ async def handle_order_updates( f'{txid}:{order_msg}' ) - # NOTE: The only reason for this seems to be remapping - # underlying `txid` values on order "edits" which the - # `openOrders` sub doesn't seem to have any knowledge of. - # UPDATE: seems like we don't need this any more thanks to - # passing through the dialog key / reqid in the `newuserref` - # field on edit requests. - - # I'd also like to ask them which event guarantees that the - # the live order is now in the book, since these status ones - # almost seem more like request-acks then state guarantees. - # ANSWER the `openOrders` is more indicative of "liveness". + # order request status updates case { 'event': etype, 'status': status, @@ -1003,8 +1050,8 @@ async def handle_order_updates( 'reqid': reqid or 0, 'txid': [txid], }) - case _: + log.warning(f'Unhandled trades update msg: {msg}') From d280a592b1d0b46bb2133185d8ad20852006ad72 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 5 Aug 2022 11:04:31 -0400 Subject: [PATCH 47/48] Repair normalize method logic to only error on lookup failure --- piker/brokers/kraken/api.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/piker/brokers/kraken/api.py b/piker/brokers/kraken/api.py index 36b0199a..80feab49 100644 --- a/piker/brokers/kraken/api.py +++ b/piker/brokers/kraken/api.py @@ -509,10 +509,9 @@ class Client: the 'AssetPairs' endpoint, see methods above. ''' + ticker = cls._ntable[ticker] symlen = len(ticker) if symlen != 6: - ticker = cls._ntable[ticker] - else: raise ValueError(f'Unhandled symbol: {ticker}') return ticker.lower() From 3a0987e0be23042a18972e49155c0716396cb163 Mon Sep 17 00:00:00 2001 From: Tyler Goodlet Date: Fri, 5 Aug 2022 21:00:54 -0400 Subject: [PATCH 48/48] Fix to-fast-edit guard case --- piker/brokers/kraken/broker.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/piker/brokers/kraken/broker.py b/piker/brokers/kraken/broker.py index 25c8c1a7..3641934a 100644 --- a/piker/brokers/kraken/broker.py +++ b/piker/brokers/kraken/broker.py @@ -1038,7 +1038,7 @@ async def handle_order_updates( txid # we throttle too-fast-requests on the ems side - or (txid and isinstance(txid, TooFastEdit)) + and not isinstance(txid, TooFastEdit) ): # client was editting too quickly # so we instead cancel this order