Ensure tests are running and working up until asserting pps
parent
fcd8b8eb78
commit
36f466fff8
|
@ -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)
|
||||||
|
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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,
|
||||||
),
|
),
|
||||||
)
|
)
|
||||||
|
|
Loading…
Reference in New Issue