Ensure tests are running and working up until asserting pps

paper_trade_improvements_rebase
jaredgoldman 2023-02-26 15:59:55 -05:00
parent 178fb8b56e
commit 5ccc72111b
3 changed files with 198 additions and 177 deletions

View File

@ -531,7 +531,7 @@ async def trades_dialogue(
): ):
with open_pps(broker, 'piker-paper', False) as table: with open_pps(broker, 'paper', False) as table:
# save pps in local state # save pps in local state
_positions.update(table.pps) _positions.update(table.pps)

View File

@ -1,6 +1,9 @@
from contextlib import asynccontextmanager as acm from contextlib import asynccontextmanager as acm
from functools import partial from functools import partial
import os import os
from typing import AsyncContextManager
from pathlib import Path
from shutil import rmtree
import pytest import pytest
import tractor import tractor
@ -11,6 +14,7 @@ from piker import (
from piker._daemon import ( from piker._daemon import (
Services, Services,
) )
from piker.clearing._client import open_ems
def pytest_addoption(parser): def pytest_addoption(parser):
@ -132,3 +136,49 @@ def open_test_pikerd(
# - no leaked subprocs or shm buffers # - no leaked subprocs or shm buffers
# - all requested container service are torn down # - all requested container service are torn down
# - certain ``tractor`` runtime state? # - certain ``tractor`` runtime state?
@acm
async def _open_test_pikerd_and_ems(
fqsn,
mode,
loglevel,
open_test_pikerd
):
async with (
open_test_pikerd() as (_, _, _, services),
open_ems(
fqsn,
mode=mode,
loglevel=loglevel,
) as ems_services):
yield (services, ems_services)
@pytest.fixture
def open_test_pikerd_and_ems(
open_test_pikerd,
fqsn: str = 'xbtusdt.kraken',
mode: str = 'paper',
loglevel: str = 'info',
):
yield partial(
_open_test_pikerd_and_ems,
fqsn,
mode,
loglevel,
open_test_pikerd
)
@pytest.fixture(scope='session')
def delete_testing_dir():
'''This fixture removes the temp directory
used for storing all config/ledger/pp data
created during testing sessions
'''
yield
app_dir = Path(config.get_app_dir('piker')).resolve()
if app_dir.is_dir():
rmtree(str(app_dir))
assert not app_dir.is_dir()

View File

@ -1,17 +1,13 @@
""" '''
Paper-mode testing Paper-mode testing
""" '''
import trio import trio
import math
from shutil import rmtree
from exceptiongroup import BaseExceptionGroup from exceptiongroup import BaseExceptionGroup
from typing import ( from typing import (
AsyncContextManager, AsyncContextManager,
Literal, Literal,
) )
from pathlib import Path
from operator import attrgetter
import pytest import pytest
import tractor import tractor
@ -19,219 +15,194 @@ from tractor._exceptions import ContextCancelled
from uuid import uuid4 from uuid import uuid4
from functools import partial from functools import partial
from piker.config import get_app_dir
from piker.log import get_logger from piker.log import get_logger
from piker.clearing._messages import Order from piker.clearing._messages import Order
from piker.pp import ( from piker.pp import (
PpTable,
open_trade_ledger, open_trade_ledger,
open_pps, open_pps,
) )
from piker.clearing import (
open_ems,
)
from piker.clearing._client import (
OrderBook,
)
from piker._cacheables import open_cached_client
from piker.clearing._messages import BrokerdPosition
log = get_logger(__name__) log = get_logger(__name__)
@pytest.fixture(scope="session")
def delete_testing_dir():
"""This fixture removes the temp directory
used for storing all config/ledger/pp data
created during testing sessions
"""
yield
app_dir = Path(get_app_dir("piker")).resolve()
if app_dir.is_dir():
rmtree(str(app_dir))
assert not app_dir.is_dir()
def get_fqsn(broker, symbol): def get_fqsn(broker, symbol):
fqsn = f"{symbol}.{broker}" fqsn = f'{symbol}.{broker}'
return (fqsn, symbol, broker) return (fqsn, symbol, broker)
def test_paper_trade(open_test_pikerd: AsyncContextManager, delete_testing_dir): oid = ''
oid = "" test_exec_mode = 'live'
test_exec_mode = "live" (fqsn, symbol, broker) = get_fqsn('kraken', 'xbtusdt')
test_account = "paper" brokers = [broker]
(fqsn, symbol, broker) = get_fqsn("kraken", "xbtusdt") account = 'paper'
brokers = [broker]
account = "paper"
positions: dict[
# brokername, acctid
tuple[str, str],
list[BrokerdPosition],
]
async def _async_main(
action: Literal["buy", "sell"] | None = None,
price: int = 30000,
assert_entries: bool = False,
assert_pps: bool = False,
assert_zeroed_pps: bool = False,
assert_msg: bool = True,
executions: int = 1,
size: float = 0.01,
) -> None:
"""Start piker, place a trade and assert entries are present
in both trade ledger and pps tomls. Then restart piker and ensure
that pps from previous trade exists in the ems pps.
Finally close the position and ensure that the position in pps.toml is closed.
"""
nonlocal oid
nonlocal positions
book: OrderBook
msg = {}
# Set up piker and EMS
async with (
open_test_pikerd() as (_, _, _, services),
open_ems(fqsn, mode="paper") as (
book,
trades_stream,
pps,
accounts,
dialogs,
),
):
# Send order to EMS
if action:
for x in range(executions):
print(f"Sending {action} order num {x}")
oid = str(uuid4())
order = Order(
exec_mode=test_exec_mode,
action=action,
oid=oid,
account=test_account,
size=size,
symbol=fqsn,
price=price,
brokers=brokers,
)
# This is actually a syncronous call to push a message
# to the async ems clue - hence why we call trio.sleep afterwards
book.send(order)
async for msg in trades_stream: async def _async_main(
msg = await trades_stream.receive() open_test_pikerd_and_ems: AsyncContextManager,
try: action: Literal['buy', 'sell'] | None = None,
if msg["name"] == "position": price: int = 30000,
break assert_entries: bool = False,
except (NameError, AttributeError): assert_pps: bool = False,
pass assert_zeroed_pps: bool = False,
# Do nothing, message isn't a position assert_msg: bool = False,
executions: int = 1,
await trio.sleep(1) size: float = 0.01,
# Assert entries are made in both ledger and PPS ) -> None:
if assert_entries or assert_pps or assert_zeroed_pps or assert_msg: '''Start piker, place a trade and assert data in pps stream, ledger and position table. Then restart piker and ensure
_assert( that pps from previous trade exists in the ems pps.
assert_entries, Finally close the position and ensure that the position in pps.toml is closed.
assert_pps, '''
assert_zeroed_pps, oid: str = ''
assert_msg, last_msg = {}
pps, async with open_test_pikerd_and_ems() as (
msg, services,
size, (book, trades_stream, pps, accounts, dialogs),
)
# Close piker like a user would
raise KeyboardInterrupt
def _assert(
assert_entries, assert_pps, assert_zerod_pps, assert_msg, pps, msg, size
): ):
with ( # Set up piker and EMS
open_trade_ledger(broker, test_account) as ledger, # Send order to EMS
open_pps(broker, test_account) as table, if action:
): for x in range(executions):
# TODO: Assert between msg and pp, ledger and pp, ledger and message oid = str(uuid4())
# for proper values order = Order(
print(f"assertion msg: {msg}") exec_mode=test_exec_mode,
# assert that entires are have been written action=action,
if assert_entries: oid=oid,
latest_ledger_entry = ledger[oid] account=account,
latest_position = pps[(broker, test_account)][-1] size=size,
pp_price = table.conf[broker][account][fqsn]["ppu"] symbol=fqsn,
# assert most price=price,
assert list(ledger.keys())[-1] == oid brokers=brokers,
assert latest_ledger_entry["size"] == test_size )
assert latest_ledger_entry["fqsn"] == fqsn # This is actually a syncronous call to push a message
# to the async ems clue - hence why we call trio.sleep afterwards
book.send(order)
# Ensure the price-per-unit (breakeven) price is close to our clearing price async for msg in trades_stream:
assert math.isclose(pp_price, latest_ledger_entry["size"], rel_tol=1) last_msg = msg
assert table.brokername == broker match msg:
assert table.acctid == account case {'name': 'position'}:
break
# assert that the last pps price is the same as the ledger price if assert_entries or assert_pps or assert_zeroed_pps or assert_msg:
if assert_pps: _assert(
latest_ledger_entry = ledger[oid] assert_entries,
latest_position = pps[(broker, test_account)][-1] assert_pps,
assert latest_position["avg_price"] == latest_ledger_entry["price"] assert_zeroed_pps,
pps,
if assert_zerod_pps: last_msg,
# assert that positions are not present size,
assert not bool(table.pps) executions,
if assert_msg and msg["name"] == "position":
latest_position = pps[(broker, test_account)][-1]
breakpoint()
assert msg["broker"] == broker
assert msg["account"]== test_account
assert msg["symbol"] == fqsn
assert msg["avg_price"]== latest_position["avg_price"]
# Close position and assert empty position in pps
def _run_test_and_check(exception, fn):
with pytest.raises(exception) as exc_info:
trio.run(fn)
for exception in exc_info.value.exceptions:
assert (
isinstance(exception, KeyboardInterrupt)
or isinstance(exception, ContextCancelled)
or isinstance(exception, KeyError)
) )
# Teardown piker like a user would
raise KeyboardInterrupt
def _assert(
assert_entries,
assert_pps,
assert_zerod_pps,
pps,
last_msg,
size,
executions,
):
with (
open_trade_ledger(broker, account) as ledger,
open_pps(broker, account) as table,
):
'''
Assert multiple cases including pps, ledger and final position message state
'''
if assert_entries:
assert last_msg['broker'] == broker
assert last_msg['account'] == account
assert last_msg['symbol'] == fqsn
assert last_msg['size'] == size * executions
assert last_msg['currency'] == symbol
assert last_msg['avg_price'] == table.pps[symbol].ppu
if assert_pps:
last_ppu = pps[(broker, account)][-1]
assert last_ppu['avg_price'] == table.pps[symbol].ppu
if assert_zerod_pps:
assert not bool(table.pps)
# Close position and assert empty position in pps
def _run_test_and_check(fn):
with pytest.raises(BaseExceptionGroup) as exc_info:
trio.run(fn)
for exception in exc_info.value.exceptions:
assert isinstance(exception, KeyboardInterrupt) or isinstance(
exception, ContextCancelled
)
def test_buy(open_test_pikerd_and_ems: AsyncContextManager, delete_testing_dir):
# Enter a trade and assert entries are made in pps and ledger files # Enter a trade and assert entries are made in pps and ledger files
_run_test_and_check( _run_test_and_check(
BaseExceptionGroup, partial(
partial(_async_main, action="buy", assert_entries=True), _async_main,
open_test_pikerd_and_ems=open_test_pikerd_and_ems,
action='buy',
assert_entries=True,
),
) )
# Open ems and assert existence of pps entries # Open ems and assert existence of pps entries
_run_test_and_check( _run_test_and_check(
BaseExceptionGroup, partial(
partial(_async_main, assert_pps=True), _async_main,
open_test_pikerd_and_ems=open_test_pikerd_and_ems,
assert_pps=True,
),
) )
def test_sell(open_test_pikerd_and_ems: AsyncContextManager, delete_testing_dir):
# Sell position # Sell position
_run_test_and_check( _run_test_and_check(
BaseExceptionGroup, partial(
partial(_async_main, action="sell", price=1), _async_main,
open_test_pikerd_and_ems=open_test_pikerd_and_ems,
action='sell',
price=1,
),
) )
# Ensure pps are zeroed # Ensure pps are zeroed
_run_test_and_check( _run_test_and_check(
BaseExceptionGroup, partial(
partial(_async_main, assert_zeroed_pps=True), _async_main,
open_test_pikerd_and_ems=open_test_pikerd_and_ems,
assert_zeroed_pps=True,
),
) )
#
def test_multi_sell(open_test_pikerd_and_ems: AsyncContextManager, delete_testing_dir):
# Make 5 market limit buy orders # Make 5 market limit buy orders
_run_test_and_check( _run_test_and_check(
BaseExceptionGroup, partial(_async_main, action="buy", executions=5) partial(
_async_main,
open_test_pikerd_and_ems=open_test_pikerd_and_ems,
action='buy',
executions=5,
),
) )
# Sell 5 slots at the same price, assert cleared positions # Sell 5 slots at the same price, assert cleared positions
_run_test_and_check( _run_test_and_check(
BaseExceptionGroup,
partial( partial(
_async_main, action="sell", executions=5, price=1, assert_zeroed_pps=True _async_main,
open_test_pikerd_and_ems=open_test_pikerd_and_ems,
action='sell',
executions=5,
price=1,
assert_zeroed_pps=True,
), ),
) )