Skip to content

Commit

Permalink
#981 Added first test of send basket to market
Browse files Browse the repository at this point in the history
  • Loading branch information
chrisjstevo committed Nov 21, 2023
1 parent 1013a05 commit b970f52
Show file tree
Hide file tree
Showing 10 changed files with 146 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ object BasketModule extends DefaultModule {
final val BasketTradingConstituentTable = "basketTradingConstituent"
final val BasketTradingConstituentJoin = "basketTradingConstituentJoin"

def apply()(implicit clock: Clock, lifecycle: LifecycleContainer, tableDefContainer: TableDefContainer): ViewServerModule = {
def apply(omsApi: OmsApi)(implicit clock: Clock, lifecycle: LifecycleContainer, tableDefContainer: TableDefContainer): ViewServerModule = {

import org.finos.vuu.core.module.basket.BasketModule.{BasketColumnNames => B, BasketConstituentColumnNames => BC, BasketTradingColumnNames => BT, BasketTradingConstituentColumnNames => BTC, PriceStrategy => PS}

Expand All @@ -39,15 +39,16 @@ object BasketModule extends DefaultModule {
(table, _) => new BasketProvider(table),
(table, _, _, tableContainer) => ViewPortDef(
columns = table.getTableDef.columns,
service = new BasketService(table, tableContainer)
service = new BasketService(table, tableContainer, omsApi)
)
)
.addTable(
TableDef(
name = BasketConstituentTable,
keyField = BC.RicBasketId,
columns = Columns.fromNames(BC.RicBasketId.string(), BC.Ric.string(), BC.BasketId.string(), BC.Weighting.double(), BC.LastTrade.string(), BC.Change.string(),
BC.Volume.string(), BC.Side.string(), BC.Description.string()), // we can join to instruments and other tables to get the rest of the data.....
columns = Columns.fromNames(BC.RicBasketId.string(), BC.Ric.string(), BC.BasketId.string(),
BC.Weighting.double(), BC.LastTrade.string(), BC.Change.string(),
BC.Volume.string(), BC.Side.string(), BC.Description.string()), // we can join to instruments and other tables to get the rest of the data.....
VisualLinks(),
joinFields = BC.RicBasketId, BC.Ric
),
Expand All @@ -57,14 +58,17 @@ object BasketModule extends DefaultModule {
TableDef(
name = BasketTradingTable,
keyField = BT.InstanceId,
columns = Columns.fromNames(BT.InstanceId.string(), BT.BasketId.string(), BT.BasketName.string(), BT.Status.string(), BT.Units.int(), BT.FilledPct.double(), BT.FxRateToUsd.double(), BT.TotalNotional.double(), BT.TotalNotionalUsd.double(), BT.Side.string()), // we can join to instruments and other tables to get the rest of the data.....
columns = Columns.fromNames(BT.InstanceId.string(), BT.BasketId.string(), BT.BasketName.string(),
BT.Status.string(), BT.Units.int(), BT.FilledPct.double(), BT.FxRateToUsd.double(),
BT.TotalNotional.double(), BT.TotalNotionalUsd.double(), BT.Side.string()
), // we can join to instruments and other tables to get the rest of the data.....
VisualLinks(),
joinFields = BT.BasketId
),
(table, vs) => new BasketTradingProvider(table, vs.tableContainer),
(table, _, _, tableContainer) => ViewPortDef(
columns = table.getTableDef.columns,
service = new BasketTradingService(table, tableContainer)
service = new BasketTradingService(table, tableContainer, omsApi)
)
)
.addTable(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,9 @@ import org.finos.vuu.provider.DefaultProvider

class BasketTradingConstituentProvider(val table: DataTable, val omsApi: OmsApi)(implicit lifecycle: LifecycleContainer, clock: Clock) extends DefaultProvider {

import org.finos.vuu.core.module.basket.BasketModule.{BasketTradingConstituentColumnNames => BTC}
private val runner = new LifeCycleRunner("TradingConsProviderRunner", runOnce, 50L)

val runner = new LifeCycleRunner("TradingConsProviderRunner", runOnce, 50L)
import org.finos.vuu.core.module.basket.BasketModule.{BasketTradingConstituentColumnNames => BTC}

omsApi.addListener(new OmsListener {
override def onAck(ack: Ack): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import org.finos.vuu.core.module.basket.service.BasketService.counter
import org.finos.vuu.core.table.{DataTable, RowData, RowWithData, TableContainer}
import org.finos.vuu.net.rpc.{EditRpcHandler, RpcHandler}
import org.finos.vuu.net.{ClientSessionId, RequestContext}
import org.finos.vuu.order.oms.{NewOrder, OmsApi}
import org.finos.vuu.viewport._

import java.util.concurrent.atomic.AtomicInteger
Expand All @@ -20,7 +21,7 @@ trait BasketServiceIF{
def createBasket(basketKey: String, name: String)(ctx: RequestContext): ViewPortAction
}

class BasketService(val table: DataTable, val tableContainer: TableContainer)(implicit clock: Clock) extends RpcHandler with BasketServiceIF with StrictLogging {
class BasketService(val table: DataTable, val tableContainer: TableContainer, val omsApi: OmsApi)(implicit clock: Clock) extends RpcHandler with BasketServiceIF with StrictLogging {

import org.finos.vuu.core.module.basket.BasketModule.{BasketConstituentColumnNames => BC, BasketTradingColumnNames => BT, BasketTradingConstituentColumnNames => BTC}

Expand Down Expand Up @@ -73,6 +74,7 @@ class BasketService(val table: DataTable, val tableContainer: TableContainer)(im
createBasketInternal(basketKey, name, ctx.session)
}


private def createBasketInternal(basketKey: String, name: String, sessionId: ClientSessionId): ViewPortAction = {

val constituents = getConstituentsForBasketKey(basketKey)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,23 +2,55 @@ package org.finos.vuu.core.module.basket.service

import com.typesafe.scalalogging.StrictLogging
import org.finos.toolbox.time.Clock
import org.finos.vuu.core.module.basket.BasketModule
import org.finos.vuu.core.module.basket.BasketModule.{BasketTradingConstituentTable, Sides}
import org.finos.vuu.core.table.{DataTable, RowWithData, TableContainer}
import org.finos.vuu.net.{ClientSessionId, RequestContext}
import org.finos.vuu.net.rpc.{EditRpcHandler, RpcHandler}
import org.finos.vuu.viewport.{NoAction, ViewPort, ViewPortAction, ViewPortAddRowAction, ViewPortDeleteCellAction, ViewPortDeleteRowAction, ViewPortEditAction, ViewPortEditCellAction, ViewPortEditRowAction, ViewPortEditSuccess, ViewPortFormCloseAction, ViewPortFormSubmitAction}
import org.finos.vuu.net.{ClientSessionId, RequestContext}
import org.finos.vuu.order.oms.{NewOrder, OmsApi}
import org.finos.vuu.viewport._

trait BasketTradingServiceIF extends EditRpcHandler {
def sendToMarket(basketInstanceId: String)(ctx: RequestContext): ViewPortAction
}

class BasketTradingService(val table: DataTable, val tableContainer: TableContainer)(implicit clock: Clock) extends RpcHandler with BasketTradingServiceIF with StrictLogging {
class BasketTradingService(val table: DataTable, val tableContainer: TableContainer, val omsApi: OmsApi)(implicit clock: Clock) extends RpcHandler with BasketTradingServiceIF with StrictLogging {

import org.finos.vuu.core.module.basket.BasketModule.{BasketTradingColumnNames => BT, BasketTradingConstituentColumnNames => BTC}

/**
* Send basket to market
*/
override def sendToMarket(name: String)(ctx: RequestContext): ViewPortAction = {
val tableRow = table.asTable.pullRow(name)

logger.info("Sending basket to market:" + name + " (row:" + tableRow + ")")

val tradingConsTable = tableContainer.getTable(BasketModule.BasketTradingConstituentTable)

val constituents = tradingConsTable.primaryKeys.toList
.map(tradingConsTable.pullRow)
.filter(_.get(BTC.InstanceId) == name)

import org.finos.vuu.core.module.basket.BasketModule.{BasketConstituentColumnNames => BC, BasketTradingColumnNames => BT, BasketTradingConstituentColumnNames => BTC}
constituents.foreach(constituentRow => {

override def sendToMarket(basketInstanceId: String)(ctx: RequestContext): ViewPortAction = {
//table.
NoAction()
val quantity = constituentRow.get(BTC.Quantity).asInstanceOf[Long]
val symbol = constituentRow.get(BTC.Ric).toString
val price = constituentRow.get(BTC.LimitPrice).asInstanceOf[Double]
val instanceIdRic = constituentRow.get(BTC.InstanceIdRic).toString
val side = constituentRow.get(BTC.Side).toString

val nos = NewOrder(side, symbol, quantity, price, instanceIdRic)

logger.info(s"Sending constituent to market $nos")

omsApi.createOrder(nos)
})

table.processUpdate(name, RowWithData(name,
Map(BT.InstanceId -> name, BT.Status -> BasketStates.ON_MARKET)), clock.now())

ViewPortEditSuccess()
}

private def onEditCell(key: String, columnName: String, data: Any, vp: ViewPort, session: ClientSessionId): ViewPortEditAction = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,3 +6,8 @@ object OrderStates {
final val CANCELLED = "CANCELLED"
final val FILLED = "FILLED"
}

object BasketStates{
final val OFF_MARKET = "OFF_MARKET"
final val ON_MARKET = "ON_MARKET"
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import org.finos.vuu.core.module.TableDefContainer
import org.finos.vuu.core.module.basket.service.{BasketServiceIF, BasketTradingServiceIF}
import org.finos.vuu.core.module.price.PriceModule
import org.finos.vuu.core.table.TableTestHelper.combineQs
import org.finos.vuu.order.oms.OmsApi
import org.finos.vuu.test.VuuServerTestCase
import org.finos.vuu.util.table.TableAsserts.assertVpEq
import org.scalatest.prop.Tables.Table
Expand All @@ -23,9 +24,11 @@ class BasketCreateTest extends VuuServerTestCase {
implicit val tableDefContainer: TableDefContainer = new TableDefContainer(Map())
implicit val metricsProvider: MetricsProvider = new MetricsProviderImpl

val omsApi = OmsApi()

import BasketModule.{BasketTradingColumnNames => BT, _}

withVuuServer(PriceModule(), BasketModule()) {
withVuuServer(PriceModule(), BasketModule(omsApi)) {
vuuServer =>

vuuServer.login("testUser", "testToken")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ import org.finos.vuu.api.ViewPortDef
import org.finos.vuu.core.module.TableDefContainer
import org.finos.vuu.core.module.basket.service.{BasketServiceIF, BasketTradingServiceIF}
import org.finos.vuu.core.module.price.PriceModule
import org.finos.vuu.order.oms.OmsApi
import org.finos.vuu.test.VuuServerTestCase
import org.finos.vuu.util.table.TableAsserts.assertVpEq
import org.scalatest.prop.Tables.Table
Expand All @@ -24,9 +25,11 @@ class BasketMutateOffMarketTest extends VuuServerTestCase {
implicit val tableDefContainer: TableDefContainer = new TableDefContainer(Map())
implicit val metricsProvider: MetricsProvider = new MetricsProviderImpl

val omsApi = OmsApi()

import BasketModule._

withVuuServer(PriceModule(), BasketModule()) {
withVuuServer(PriceModule(), BasketModule(omsApi)) {
vuuServer =>

vuuServer.login("testUser", "testToken")
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
package org.finos.vuu.core.module.basket

import org.finos.toolbox.jmx.{MetricsProvider, MetricsProviderImpl}
import org.finos.toolbox.lifecycle.LifecycleContainer
import org.finos.toolbox.time.{Clock, TestFriendlyClock}
import org.finos.vuu.api.ViewPortDef
import org.finos.vuu.core.module.TableDefContainer
import org.finos.vuu.core.module.basket.BasketTestCaseHelper.{tickBasketDef, tickConstituentsDef, tickPrices}
import org.finos.vuu.core.module.basket.service.{BasketServiceIF, BasketTradingServiceIF}
import org.finos.vuu.core.module.price.PriceModule
import org.finos.vuu.order.oms.OmsApi
import org.finos.vuu.test.VuuServerTestCase
import org.finos.vuu.util.table.TableAsserts.assertVpEq
import org.scalatest.prop.Tables.Table

class BasketSendToMarketTest extends VuuServerTestCase {

Feature("Basket Service Test Case") {

Scenario("Check the creation of the baskets and constituents") {

implicit val clock: Clock = new TestFriendlyClock(10001L)
implicit val lifecycle: LifecycleContainer = new LifecycleContainer()
implicit val tableDefContainer: TableDefContainer = new TableDefContainer(Map())
implicit val metricsProvider: MetricsProvider = new MetricsProviderImpl

val omsApi = OmsApi()

import BasketModule._

withVuuServer(PriceModule(), BasketModule(omsApi)) {
vuuServer =>

vuuServer.login("testUser", "testToken")

vuuServer.overrideViewPortDef("prices", (table, _, _, _) => ViewPortDef(table.getTableDef.columns, null))

val pricesProvider = vuuServer.getProvider(PriceModule.NAME, "prices")
val basketProvider = vuuServer.getProvider(BasketModule.NAME, BasketTable)
val constituentProvider = vuuServer.getProvider(BasketModule.NAME, BasketConstituentTable)

tickPrices(pricesProvider)
tickBasketDef(basketProvider)
tickConstituentsDef(constituentProvider)

val vpBasket = vuuServer.createViewPort(BasketModule.NAME, BasketTable)

vuuServer.runOnce()

Then("Get the Basket RPC Service and call create basket")
val basketService = vuuServer.getViewPortRpcServiceProxy[BasketServiceIF](vpBasket)

basketService.createBasket(".FTSE", "chris-001")(vuuServer.requestContext)

val vpBasketTrading = vuuServer.createViewPort(BasketModule.NAME, BasketTradingTable)

vuuServer.runOnce()

val tradingService = vuuServer.getViewPortRpcServiceProxy[BasketTradingServiceIF](vpBasketTrading)

And("send the basket to market")
tradingService.sendToMarket("chris-001")(vuuServer.requestContext)

vuuServer.runOnce()

Then("verify basket is on market")
assertVpEq(combineQsForVp(vpBasketTrading)) {
Table(
("instanceId", "basketId", "basketName", "status", "units", "filledPct", "fxRateToUsd", "totalNotional", "totalNotionalUsd", "side"),
("chris-001", ".FTSE", "chris-001", "ON_MARKET", 1, null, null, null, null, "Buy")
)
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ object MaxTimes{
final val MAX_FILL_TIME_MS = 8_000
}

case class NewOrder(symbol: String, qty: Long, price: Double, clientOrderId: String)
case class NewOrder(side: String, symbol: String, qty: Long, price: Double, clientOrderId: String)
case class ReplaceOrder(orderId: Int, newPrice: Double, newQty: Long)
case class CancelOrder(orderId: Int)
case class Ack(orderId: Int, clientOrderId: String, symbol: String, qty: Long, price: Double)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ import org.scalatest.featurespec.AnyFeatureSpec
import org.scalatest.matchers.should.Matchers

import java.util.concurrent.ConcurrentHashMap
import javax.print.attribute.standard.Sides

case class TestOrderState(symbol: String, qty: Long, price: Double, state: String, filledQty: Long, filledPrice: Double)

Expand Down Expand Up @@ -61,7 +62,7 @@ class OmsApiTest extends AnyFeatureSpec with GivenWhenThen with Matchers {

omsApi.addListener(listener)

omsApi.createOrder(NewOrder("VOD.L", 1000L, 100.01, "clOrdId1"))
omsApi.createOrder(NewOrder("Buy","VOD.L", 1000L, 100.01, "clOrdId1"))

clock.sleep(MAX_ACK_TIME_MS)
omsApi.runOnce()
Expand Down

0 comments on commit b970f52

Please sign in to comment.