Add context-styled `asyncio` client proxy for ib
This adds a new client manager-factory: `open_client_proxy()` which uses the newer `tractor.to_asyncio.open_channel_from()` (and thus the inter-loop-task-channel style) a `aio_client_method_relay()` and a re-implemented `MethodProxy` wrapper to allow transparently calling `asyncio` client methods from `trio` tasks. Use this proxy in the history backfiller task and add a new (prototype) `open_history_client()` which will be used in the new storage management layer. Drop `get_client()` which was the portal wrapping equivalent of the same proxy but with a one-task-per-call approach. Oh, and `Client.bars()` can take `datetime`, so let's use it B)mkts_backup
							parent
							
								
									cf589c840d
								
							
						
					
					
						commit
						820dfff08a
					
				| 
						 | 
					@ -29,8 +29,9 @@ from functools import partial
 | 
				
			||||||
import itertools
 | 
					import itertools
 | 
				
			||||||
from math import isnan
 | 
					from math import isnan
 | 
				
			||||||
from typing import (
 | 
					from typing import (
 | 
				
			||||||
    Any, Optional,
 | 
					    Any, Callable, Optional,
 | 
				
			||||||
    AsyncIterator, Awaitable,
 | 
					    AsyncIterator, Awaitable,
 | 
				
			||||||
 | 
					    Union,
 | 
				
			||||||
)
 | 
					)
 | 
				
			||||||
import asyncio
 | 
					import asyncio
 | 
				
			||||||
from pprint import pformat
 | 
					from pprint import pformat
 | 
				
			||||||
| 
						 | 
					@ -43,6 +44,7 @@ import time
 | 
				
			||||||
import trio
 | 
					import trio
 | 
				
			||||||
from trio_typing import TaskStatus
 | 
					from trio_typing import TaskStatus
 | 
				
			||||||
import tractor
 | 
					import tractor
 | 
				
			||||||
 | 
					from tractor import to_asyncio
 | 
				
			||||||
from async_generator import aclosing
 | 
					from async_generator import aclosing
 | 
				
			||||||
from ib_insync.wrapper import RequestError
 | 
					from ib_insync.wrapper import RequestError
 | 
				
			||||||
from ib_insync.contract import Contract, ContractDetails, Option
 | 
					from ib_insync.contract import Contract, ContractDetails, Option
 | 
				
			||||||
| 
						 | 
					@ -58,7 +60,7 @@ import numpy as np
 | 
				
			||||||
 | 
					
 | 
				
			||||||
from .. import config
 | 
					from .. import config
 | 
				
			||||||
from ..log import get_logger, get_console_log
 | 
					from ..log import get_logger, get_console_log
 | 
				
			||||||
from .._daemon import maybe_spawn_brokerd
 | 
					# from .._daemon import maybe_spawn_brokerd
 | 
				
			||||||
from ..data._source import from_df
 | 
					from ..data._source import from_df
 | 
				
			||||||
from ..data._sharedmem import ShmArray
 | 
					from ..data._sharedmem import ShmArray
 | 
				
			||||||
from ._util import SymbolNotFound, NoData
 | 
					from ._util import SymbolNotFound, NoData
 | 
				
			||||||
| 
						 | 
					@ -217,11 +219,12 @@ _enters = 0
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
class Client:
 | 
					class Client:
 | 
				
			||||||
    """IB wrapped for our broker backend API.
 | 
					    '''
 | 
				
			||||||
 | 
					    IB wrapped for our broker backend API.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    Note: this client requires running inside an ``asyncio`` loop.
 | 
					    Note: this client requires running inside an ``asyncio`` loop.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    """
 | 
					    '''
 | 
				
			||||||
    _contracts: dict[str, Contract] = {}
 | 
					    _contracts: dict[str, Contract] = {}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    def __init__(
 | 
					    def __init__(
 | 
				
			||||||
| 
						 | 
					@ -242,20 +245,22 @@ class Client:
 | 
				
			||||||
        self,
 | 
					        self,
 | 
				
			||||||
        symbol: str,
 | 
					        symbol: str,
 | 
				
			||||||
        # EST in ISO 8601 format is required... below is EPOCH
 | 
					        # EST in ISO 8601 format is required... below is EPOCH
 | 
				
			||||||
        start_dt: str = "1970-01-01T00:00:00.000000-05:00",
 | 
					        start_dt: Union[datetime, str] = "1970-01-01T00:00:00.000000-05:00",
 | 
				
			||||||
        end_dt: str = "",
 | 
					        end_dt: Union[datetime, str ] = "",
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        sample_period_s: str = 1,  # ohlc sample period
 | 
					        sample_period_s: str = 1,  # ohlc sample period
 | 
				
			||||||
        period_count: int = int(2e3),  # <- max per 1s sample query
 | 
					        period_count: int = int(2e3),  # <- max per 1s sample query
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        is_paid_feed: bool = False,  # placeholder
 | 
					        is_paid_feed: bool = False,  # placeholder
 | 
				
			||||||
    ) -> list[dict[str, Any]]:
 | 
					    ) -> list[dict[str, Any]]:
 | 
				
			||||||
        """Retreive OHLCV bars for a symbol over a range to the present.
 | 
					        '''
 | 
				
			||||||
        """
 | 
					        Retreive OHLCV bars for a symbol over a range to the present.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        '''
 | 
				
			||||||
        bars_kwargs = {'whatToShow': 'TRADES'}
 | 
					        bars_kwargs = {'whatToShow': 'TRADES'}
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        global _enters
 | 
					        global _enters
 | 
				
			||||||
        print(f'ENTER BARS {_enters}')
 | 
					        print(f'ENTER BARS {_enters} @ end={end_dt}')
 | 
				
			||||||
        _enters += 1
 | 
					        _enters += 1
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        contract = await self.find_contract(symbol)
 | 
					        contract = await self.find_contract(symbol)
 | 
				
			||||||
| 
						 | 
					@ -984,7 +989,7 @@ async def _trio_run_client_method(
 | 
				
			||||||
    # ):
 | 
					    # ):
 | 
				
			||||||
    #     kwargs['_treat_as_stream'] = True
 | 
					    #     kwargs['_treat_as_stream'] = True
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    return await tractor.to_asyncio.run_task(
 | 
					    return await to_asyncio.run_task(
 | 
				
			||||||
        _aio_run_client_method,
 | 
					        _aio_run_client_method,
 | 
				
			||||||
        meth=method,
 | 
					        meth=method,
 | 
				
			||||||
        client=client,
 | 
					        client=client,
 | 
				
			||||||
| 
						 | 
					@ -992,60 +997,119 @@ async def _trio_run_client_method(
 | 
				
			||||||
    )
 | 
					    )
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
class _MethodProxy:
 | 
					class MethodProxy:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    def __init__(
 | 
					    def __init__(
 | 
				
			||||||
        self,
 | 
					        self,
 | 
				
			||||||
        portal: tractor.Portal
 | 
					        chan: to_asyncio.LinkedTaskChannel,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    ) -> None:
 | 
					    ) -> None:
 | 
				
			||||||
        self._portal = portal
 | 
					        self.chan = chan
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    async def _run_method(
 | 
					    async def _run_method(
 | 
				
			||||||
        self,
 | 
					        self,
 | 
				
			||||||
        *,
 | 
					        *,
 | 
				
			||||||
        meth: str = None,
 | 
					        meth: str = None,
 | 
				
			||||||
        **kwargs
 | 
					        **kwargs
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    ) -> Any:
 | 
					    ) -> Any:
 | 
				
			||||||
        return await self._portal.run(
 | 
					        '''
 | 
				
			||||||
            _trio_run_client_method,
 | 
					        Make a ``Client`` method call by requesting through the
 | 
				
			||||||
            method=meth,
 | 
					        ``tractor.to_asyncio`` layer.
 | 
				
			||||||
            **kwargs
 | 
					
 | 
				
			||||||
        )
 | 
					        '''
 | 
				
			||||||
 | 
					        chan = self.chan
 | 
				
			||||||
 | 
					        # send through method + ``kwargs: dict`` as pair
 | 
				
			||||||
 | 
					        await chan.send((meth, kwargs))
 | 
				
			||||||
 | 
					        return await chan.receive()
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
def get_client_proxy(
 | 
					async def open_aio_client_method_relay(
 | 
				
			||||||
 | 
					    from_trio: asyncio.Queue,
 | 
				
			||||||
 | 
					    to_trio: trio.abc.SendChannel,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    portal: tractor.Portal,
 | 
					) -> None:
 | 
				
			||||||
    target=Client,
 | 
					 | 
				
			||||||
 | 
					
 | 
				
			||||||
) -> _MethodProxy:
 | 
					    async with load_aio_clients() as (
 | 
				
			||||||
 | 
					        client,
 | 
				
			||||||
 | 
					        clients,
 | 
				
			||||||
 | 
					        accts2clients,
 | 
				
			||||||
 | 
					    ):
 | 
				
			||||||
 | 
					        to_trio.send_nowait(client)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    proxy = _MethodProxy(portal)
 | 
					        # relay all method requests to ``asyncio``-side client and
 | 
				
			||||||
 | 
					        # deliver back results
 | 
				
			||||||
 | 
					        while True:
 | 
				
			||||||
 | 
					            msg = await from_trio.get()
 | 
				
			||||||
 | 
					            meth_name, kwargs = msg
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    # mock all remote methods
 | 
					            meth = getattr(client, meth_name)
 | 
				
			||||||
 | 
					            resp = await meth(**kwargs)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            # echo the msg back
 | 
				
			||||||
 | 
					            to_trio.send_nowait(resp)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					@acm
 | 
				
			||||||
 | 
					async def open_client_proxy() -> MethodProxy:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    try:
 | 
				
			||||||
 | 
					        async with to_asyncio.open_channel_from(
 | 
				
			||||||
 | 
					            open_aio_client_method_relay,
 | 
				
			||||||
 | 
					        ) as (first, chan):
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            assert isinstance(first, Client)
 | 
				
			||||||
 | 
					            proxy = MethodProxy(chan)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            # mock all remote methods on ib ``Client``.
 | 
				
			||||||
            for name, method in inspect.getmembers(
 | 
					            for name, method in inspect.getmembers(
 | 
				
			||||||
        target, predicate=inspect.isfunction
 | 
					                Client, predicate=inspect.isfunction
 | 
				
			||||||
            ):
 | 
					            ):
 | 
				
			||||||
                if '_' == name[0]:
 | 
					                if '_' == name[0]:
 | 
				
			||||||
                    continue
 | 
					                    continue
 | 
				
			||||||
                setattr(proxy, name, partial(proxy._run_method, meth=name))
 | 
					                setattr(proxy, name, partial(proxy._run_method, meth=name))
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    return proxy
 | 
					            yield proxy
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    except RequestError as err:
 | 
				
			||||||
 | 
					        code, msg = err.code, err.message
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        # TODO: retreive underlying ``ib_insync`` error?
 | 
				
			||||||
 | 
					        if (
 | 
				
			||||||
 | 
					            code == 162 and (
 | 
				
			||||||
 | 
					                'HMDS query returned no data' in msg
 | 
				
			||||||
 | 
					                or 'No market data permissions for' in msg
 | 
				
			||||||
 | 
					            )
 | 
				
			||||||
 | 
					        ):
 | 
				
			||||||
 | 
					            # these cases should not cause a task crash
 | 
				
			||||||
 | 
					            log.warning(msg)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        else:
 | 
				
			||||||
 | 
					            raise
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
@acm
 | 
					# @acm
 | 
				
			||||||
async def get_client(
 | 
					# async def get_client(
 | 
				
			||||||
    **kwargs,
 | 
					#     **kwargs,
 | 
				
			||||||
) -> Client:
 | 
					
 | 
				
			||||||
    """Init the ``ib_insync`` client in another actor and return
 | 
					# ) -> Client:
 | 
				
			||||||
    a method proxy to it.
 | 
					#     '''
 | 
				
			||||||
    """
 | 
					#     Init the ``ib_insync`` client in another actor and return
 | 
				
			||||||
    async with maybe_spawn_brokerd(
 | 
					#     a method proxy to it.
 | 
				
			||||||
        brokername='ib',
 | 
					
 | 
				
			||||||
        infect_asyncio=True,
 | 
					#     '''
 | 
				
			||||||
        **kwargs
 | 
					#     async with (
 | 
				
			||||||
    ) as portal:
 | 
					#         maybe_spawn_brokerd(
 | 
				
			||||||
        proxy_client = get_client_proxy(portal)
 | 
					#             brokername='ib',
 | 
				
			||||||
        yield proxy_client
 | 
					#             infect_asyncio=True,
 | 
				
			||||||
 | 
					#             **kwargs
 | 
				
			||||||
 | 
					#         ) as portal,
 | 
				
			||||||
 | 
					#     ):
 | 
				
			||||||
 | 
					#         assert 0
 | 
				
			||||||
 | 
					        # TODO: the IPC via portal relay layer for when this current
 | 
				
			||||||
 | 
					        # actor isn't in aio mode.
 | 
				
			||||||
 | 
					        # open_client_proxy() as proxy,
 | 
				
			||||||
 | 
					        #     yield proxy
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
# https://interactivebrokers.github.io/tws-api/tick_types.html
 | 
					# https://interactivebrokers.github.io/tws-api/tick_types.html
 | 
				
			||||||
| 
						 | 
					@ -1126,27 +1190,32 @@ def normalize(
 | 
				
			||||||
 | 
					
 | 
				
			||||||
async def get_bars(
 | 
					async def get_bars(
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    proxy: MethodProxy,
 | 
				
			||||||
    sym: str,
 | 
					    sym: str,
 | 
				
			||||||
    end_dt: str = "",
 | 
					    end_dt: str = "",
 | 
				
			||||||
 | 
					
 | 
				
			||||||
) -> (dict, np.ndarray):
 | 
					) -> (dict, np.ndarray):
 | 
				
			||||||
 | 
					    '''
 | 
				
			||||||
 | 
					    Retrieve historical data from a ``trio``-side task using
 | 
				
			||||||
 | 
					    a ``MethoProxy``.
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    '''
 | 
				
			||||||
    _err: Optional[Exception] = None
 | 
					    _err: Optional[Exception] = None
 | 
				
			||||||
 | 
					 | 
				
			||||||
    fails = 0
 | 
					    fails = 0
 | 
				
			||||||
 | 
					    bars: Optional[list] = None
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    for _ in range(2):
 | 
					    for _ in range(2):
 | 
				
			||||||
        try:
 | 
					        try:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            bars, bars_array = await _trio_run_client_method(
 | 
					            bars, bars_array = await proxy.bars(
 | 
				
			||||||
                method='bars',
 | 
					 | 
				
			||||||
                symbol=sym,
 | 
					                symbol=sym,
 | 
				
			||||||
                end_dt=end_dt,
 | 
					                end_dt=end_dt,
 | 
				
			||||||
            )
 | 
					            )
 | 
				
			||||||
 | 
					 | 
				
			||||||
            if bars_array is None:
 | 
					            if bars_array is None:
 | 
				
			||||||
                raise SymbolNotFound(sym)
 | 
					                raise SymbolNotFound(sym)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            next_dt = bars[0].date
 | 
					            next_dt = bars[0].date
 | 
				
			||||||
 | 
					            print(f'ib datetime {next_dt}')
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            return (bars, bars_array, next_dt), fails
 | 
					            return (bars, bars_array, next_dt), fails
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -1169,8 +1238,16 @@ async def get_bars(
 | 
				
			||||||
                    # error?
 | 
					                    # error?
 | 
				
			||||||
                    # OLDER: seem to always cause throttling despite low rps
 | 
					                    # OLDER: seem to always cause throttling despite low rps
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                    # raise err
 | 
					                    # TODO: if there is not bars returned from the first
 | 
				
			||||||
                    break
 | 
					                    # query we need to manually calculate the next step
 | 
				
			||||||
 | 
					                    # back and convert to an expected datetime format.
 | 
				
			||||||
 | 
					                    # if not bars:
 | 
				
			||||||
 | 
					                    #     raise
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					                    # try to decrement start point and look further back
 | 
				
			||||||
 | 
					                    next_dt = bars[0].date
 | 
				
			||||||
 | 
					                    print(f'ib datetime {next_dt}')
 | 
				
			||||||
 | 
					                    continue
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                elif 'No market data permissions for' in err.message:
 | 
					                elif 'No market data permissions for' in err.message:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -1197,6 +1274,41 @@ async def get_bars(
 | 
				
			||||||
    #     raise _err
 | 
					    #     raise _err
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					@acm
 | 
				
			||||||
 | 
					async def open_history_client(
 | 
				
			||||||
 | 
					    symbol: str,
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					) -> tuple[Callable, int]:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					    async with open_client_proxy() as proxy:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        async def get_hist(
 | 
				
			||||||
 | 
					            end_dt: str,
 | 
				
			||||||
 | 
					            start_dt: str = '',
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        ) -> tuple[np.ndarray, str]:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            out, fails = await get_bars(proxy, symbol, end_dt=end_dt)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            # TODO: add logic here to handle tradable hours and only grab
 | 
				
			||||||
 | 
					            # valid bars in the range
 | 
				
			||||||
 | 
					            if out == (None, None):
 | 
				
			||||||
 | 
					                # could be trying to retreive bars over weekend
 | 
				
			||||||
 | 
					                log.error(f"Can't grab bars starting at {end_dt}!?!?")
 | 
				
			||||||
 | 
					                raise NoData(f'{end_dt}')
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            bars, bars_array, next_dt = out
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            # volume cleaning since there's -ve entries,
 | 
				
			||||||
 | 
					            # wood luv to know what crookery that is..
 | 
				
			||||||
 | 
					            vlm = bars_array['volume']
 | 
				
			||||||
 | 
					            vlm[vlm < 0] = 0
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					            return bars_array, next_dt
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					        yield get_hist
 | 
				
			||||||
 | 
					
 | 
				
			||||||
 | 
					
 | 
				
			||||||
async def backfill_bars(
 | 
					async def backfill_bars(
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    sym: str,
 | 
					    sym: str,
 | 
				
			||||||
| 
						 | 
					@ -1219,12 +1331,15 @@ async def backfill_bars(
 | 
				
			||||||
    https://github.com/pikers/piker/issues/128
 | 
					    https://github.com/pikers/piker/issues/128
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    '''
 | 
					    '''
 | 
				
			||||||
 | 
					    # async with open_history_client(sym) as proxy:
 | 
				
			||||||
 | 
					    async with open_client_proxy() as proxy:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        if platform.system() == 'Windows':
 | 
					        if platform.system() == 'Windows':
 | 
				
			||||||
            log.warning(
 | 
					            log.warning(
 | 
				
			||||||
                'Decreasing history query count to 4 since, windows...')
 | 
					                'Decreasing history query count to 4 since, windows...')
 | 
				
			||||||
            count = 4
 | 
					            count = 4
 | 
				
			||||||
 | 
					
 | 
				
			||||||
    out, fails = await get_bars(sym)
 | 
					        out, fails = await get_bars(proxy, sym)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
        if out is None:
 | 
					        if out is None:
 | 
				
			||||||
            raise RuntimeError("Could not pull currrent history?!")
 | 
					            raise RuntimeError("Could not pull currrent history?!")
 | 
				
			||||||
| 
						 | 
					@ -1243,15 +1358,15 @@ async def backfill_bars(
 | 
				
			||||||
            i = 0
 | 
					            i = 0
 | 
				
			||||||
            while i < count:
 | 
					            while i < count:
 | 
				
			||||||
 | 
					
 | 
				
			||||||
            out, fails = await get_bars(sym, end_dt=next_dt)
 | 
					                out, fails = await get_bars(proxy, sym, end_dt=next_dt)
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                if fails is None or fails > 1:
 | 
					                if fails is None or fails > 1:
 | 
				
			||||||
                    break
 | 
					                    break
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                if out == (None, None):
 | 
					                if out == (None, None):
 | 
				
			||||||
                    # could be trying to retreive bars over weekend
 | 
					                    # could be trying to retreive bars over weekend
 | 
				
			||||||
                # TODO: add logic here to handle tradable hours and only grab
 | 
					                    # TODO: add logic here to handle tradable hours and
 | 
				
			||||||
                # valid bars in the range
 | 
					                    # only grab valid bars in the range
 | 
				
			||||||
                    log.error(f"Can't grab bars starting at {next_dt}!?!?")
 | 
					                    log.error(f"Can't grab bars starting at {next_dt}!?!?")
 | 
				
			||||||
                    continue
 | 
					                    continue
 | 
				
			||||||
 | 
					
 | 
				
			||||||
| 
						 | 
					@ -1261,6 +1376,7 @@ async def backfill_bars(
 | 
				
			||||||
                # wood luv to know what crookery that is..
 | 
					                # wood luv to know what crookery that is..
 | 
				
			||||||
                vlm = bars_array['volume']
 | 
					                vlm = bars_array['volume']
 | 
				
			||||||
                vlm[vlm < 0] = 0
 | 
					                vlm[vlm < 0] = 0
 | 
				
			||||||
 | 
					
 | 
				
			||||||
                # TODO we should probably dig into forums to see what peeps
 | 
					                # TODO we should probably dig into forums to see what peeps
 | 
				
			||||||
                # think this data "means" and then use it as an indicator of
 | 
					                # think this data "means" and then use it as an indicator of
 | 
				
			||||||
                # sorts? dinkus has mentioned that $vlms for the day dont'
 | 
					                # sorts? dinkus has mentioned that $vlms for the day dont'
 | 
				
			||||||
| 
						 | 
					
 | 
				
			||||||
		Loading…
	
		Reference in New Issue