Compare commits

...

30 Commits

Author SHA1 Message Date
Sam Ricotta
28a7cbe97e wip 2022-11-25 16:15:04 +01:00
Sam Ricotta
f5bfacd2cf wip 2022-11-23 18:05:58 +01:00
Callum Waters
22d04dd19d fix up proto problems with orderbook 2022-11-22 14:24:43 +01:00
Callum Waters
5651a21adb add notes for testing 2022-11-18 16:54:32 +01:00
Callum Waters
bd75306a93 add more documentation 2022-11-18 16:49:33 +01:00
Callum Waters
b76eb6906c add hash and state meta tracking 2022-11-18 16:11:35 +01:00
Callum Waters
b0e07355bd add pairs to the database
Co-authored-by: samricotta <samricotta@users.noreply.github.com>
2022-11-18 14:18:52 +01:00
Callum Waters
f7184192a0 add execution components 2022-11-15 10:20:58 +01:00
Callum Waters
a0c68b4941 add signature verification to process proposal 2022-11-14 15:49:51 +01:00
Callum Waters
662517b0db add logic for signing and signature verification 2022-11-14 15:38:39 +01:00
Sam Ricotta
38897de6d7 orderbook example 2022-11-14 14:54:52 +01:00
Sam Ricotta
e1f3d5c58c wip test 2022-11-10 09:06:34 +01:00
Sam Ricotta
07804ea061 wip 2022-11-07 22:41:16 +01:00
Sam Ricotta
1f6a6176c0 Prepare and process proposal excecution 2022-10-28 16:20:59 +02:00
Sam Ricotta
b8162782c3 prepare proposal wip 2022-10-17 17:06:18 +03:00
Sam Ricotta
18382f97eb update 2022-10-17 17:04:40 +03:00
Sam Ricotta
5bfc6d7a3b process proposal checks 2022-10-17 17:00:06 +03:00
Callum Waters
43ce473402 add function 2022-10-17 16:57:28 +03:00
Callum Waters
3b4c1b903a split out types verification 2022-10-17 16:42:08 +03:00
Callum Waters
8a40180248 add more tests 2022-10-17 16:36:50 +03:00
Callum Waters
920c5ad813 fix some of the match functions and add more tests 2022-10-17 16:06:30 +03:00
Callum Waters
bbf1169aea add tests for market matching 2022-10-17 12:36:36 +03:00
Callum Waters
67d3a6e45b Merge branch 'sam/abci++-tutorial' of github.com:tendermint/tendermint into sam/abci++-tutorial 2022-10-17 12:16:21 +03:00
Callum Waters
0c2483e9f3 add match function 2022-10-17 12:14:40 +03:00
Sam Ricotta
b837ead457 Small fixes 2022-10-16 19:19:44 +03:00
Sam Ricotta
944e9ebdc0 sudo code updates 2022-10-16 18:58:40 +03:00
Callum Waters
28d47d9a38 update msg types 2022-10-16 18:53:39 +03:00
Sam Ricotta
7a81dd25bf update for pairng 2022-10-06 14:29:22 +02:00
Callum Waters
94c874da32 clean up and add descriptions to the protos 2022-10-06 10:37:16 +02:00
Callum Waters
e7e48024ca Start sketching out the orderbook app
Co-authored-by: samricotta <samricotta@users.noreply.github.com>
2022-10-06 09:31:50 +02:00
18 changed files with 5692 additions and 0 deletions

View File

@@ -0,0 +1,573 @@
package orderbook
import (
"bytes"
"encoding/binary"
fmt "fmt"
"github.com/cosmos/gogoproto/proto"
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/tmhash"
)
var _ types.Application = (*StateMachine)(nil)
const Version = 1
const (
// In tendermint a zero code is okay and all non zero codes are errors
StatusOK = iota
StatusErrDecoding
StatusErrUnknownMessage
StatusErrValidateBasic
StatusErrNoAccount
StatusErrAccountExists
StatusErrNoPair
StatusErrPairExists
StatusErrInvalidOrder
StatusErrUnacceptableMessage
StatusErrNoCommodity
)
var (
stateKey = []byte("state")
accountKey = []byte("account")
pairKey = []byte("pair")
)
// StateMachine is the main struct that encompasses the logic of the orderbook
type StateMachine struct {
// inherit all the abci methods so we don't have to implement everything
types.BaseApplication
// persisted state which is a key value store containing:
// accountID -> account
// pairID -> pair
db dbm.DB
// in-memory state
lastHeight int64 // the last height that was persisted
lastHash []byte // the last hash that was persisted
// list of accounts (this is used for the app hash)
accounts []*Account
pairs map[string]*Pair // lookup pairs
commodities map[string]struct{} // lookup commodities
publicKeys map[string]struct{} // lookup existence of an account
// a list of transactions that have been modified by the most recent block
// and will need to result in an update to the db
touchedAccounts map[uint64]struct{}
// new pairs added in this block which will needed to be added to the
// db on "Commit"
newPairs []*Pair
// app-side mempool (also emphemeral)
// this takes ask and bid transactions from `CheckTx`
// and matches them as a "MatchedOrder" which is
// then proposed in a block
//
// it's important to note that there is no garbage collection
// here. Bids and asks, potentially even invalid, will
// continue to stay here until matched
markets map[string]*Market // i.e. ATOM/USDC
}
// New creates a StateMachine from a given database. If the database is
// empty a fresh instance is created else the accounts, pairs and
// state are loaded into memory.
func New(db dbm.DB) (*StateMachine, error) {
// iterate over all the account keys
iter, err := db.Iterator(nil, nil)
if err != nil {
return nil, err
}
defer iter.Close()
var (
accounts = make([]*Account, 0)
publicKeys = make(map[string]struct{})
commodities = make(map[string]struct{})
pairs = make(map[string]*Pair)
markets = make(map[string]*Market)
lastHeight uint64
lastHash []byte
)
for ; iter.Valid(); iter.Next() {
if bytes.HasPrefix(iter.Key(), pairKey) {
var pair Pair
if err := proto.Unmarshal(iter.Value(), &pair); err != nil {
return nil, err
}
pairs[pair.String()] = &pair
commodities[pair.BuyersDenomination] = struct{}{}
markets[pair.String()] = NewMarket(&pair)
}
if bytes.HasPrefix(iter.Key(), accountKey) {
var acc Account
if err := proto.Unmarshal(iter.Value(), &acc); err != nil {
return nil, err
}
accounts = append(accounts, &acc)
publicKeys[string(acc.PublicKey)] = struct{}{}
}
if bytes.HasPrefix(iter.Key(), stateKey) {
state := iter.Value()
lastHeight = binary.BigEndian.Uint64(state[:4])
lastHash = state[4:]
}
}
return &StateMachine{
accounts: accounts,
pairs: pairs,
commodities: commodities,
publicKeys: publicKeys,
markets: markets,
lastHeight: int64(lastHeight),
lastHash: lastHash,
db: db,
touchedAccounts: make(map[uint64]struct{}),
newPairs: make([]*Pair, 0),
}, nil
}
// Info is used by Tendermint to understand the state of the application.
// This is useful for replay and syncing modes.
func (sm *StateMachine) Info(req types.RequestInfo) types.ResponseInfo {
return types.ResponseInfo{
AppVersion: Version,
LastBlockHeight: sm.lastHeight,
LastBlockAppHash: sm.lastHash,
}
}
// CheckTx indicates which transactions should be accepted in the mempool. It is
// not a perfect validity check because we're unsure of the state that the transaction
// will be executed against. We should treat this as a gatekeeper to the mempool.
// Apart from adding transactions to the app-side mempool, this check is stateless.
func (sm *StateMachine) CheckTx(req types.RequestCheckTx) types.ResponseCheckTx {
var msg = new(Msg)
err := proto.Unmarshal(req.Tx, msg)
if err != nil {
return types.ResponseCheckTx{Code: StatusErrDecoding, Log: err.Error()} // decoding error
}
if err := msg.ValidateBasic(); err != nil {
return types.ResponseCheckTx{Code: StatusErrValidateBasic, Log: err.Error()}
}
// add either bids or asks to the market which will match them in PrepareProposal
switch m := msg.Sum.(type) {
case *Msg_MsgAsk:
market, ok := sm.markets[m.MsgAsk.Pair.String()]
if !ok {
return types.ResponseCheckTx{Code: StatusErrNoPair}
}
market.AddAsk(m.MsgAsk.AskOrder)
case *Msg_MsgBid:
market, ok := sm.markets[m.MsgBid.Pair.String()]
if !ok {
return types.ResponseCheckTx{Code: StatusErrNoPair}
}
market.AddBid(m.MsgBid.BidOrder)
}
return types.ResponseCheckTx{Code: StatusOK}
}
// ValidateTx validates the transactions against state.
func (sm *StateMachine) ValidateTx(msg *Msg) uint32 {
if err := msg.ValidateBasic(); err != nil {
return StatusErrValidateBasic
}
switch m := msg.Sum.(type) {
case *Msg_MsgRegisterPair:
pair := m.MsgRegisterPair.Pair
if _, ok := sm.pairs[pair.String()]; ok {
return StatusErrPairExists
}
reversePair := &Pair{BuyersDenomination: pair.SellersDenomination, SellersDenomination: pair.BuyersDenomination}
if _, ok := sm.pairs[reversePair.String()]; ok {
return StatusErrPairExists
}
case *Msg_MsgAsk, *Msg_MsgBid: // MsgAsk and MsgBid are not allowed individually - they need to be matched as a TradeSet
return StatusErrUnacceptableMessage //Todo add logic around msg ask and bid to allow
case *Msg_MsgCreateAccount:
// check for duplicate accounts in state machine
if _, ok := sm.publicKeys[string(m.MsgCreateAccount.PublicKey)]; ok {
return StatusErrAccountExists
}
// check that each of the commodities is present in at least one trading pair
for _, commodity := range m.MsgCreateAccount.Commodities {
if _, exists := sm.commodities[commodity.Denom]; !exists {
return StatusErrNoCommodity
}
}
case *Msg_MsgTradeSet:
// check the pair exists
if _, ok := sm.pairs[m.MsgTradeSet.TradeSet.Pair.String()]; !ok {
return StatusErrNoPair
}
for _, order := range m.MsgTradeSet.TradeSet.MatchedOrders {
// validate matched order i.e. users have funds and signatures are valid
if !sm.isMatchedOrderValid(order, m.MsgTradeSet.TradeSet.Pair) {
return StatusErrInvalidOrder
}
}
default:
return StatusErrUnknownMessage
}
return StatusOK
}
// PrepareProposal is called whenever the validator is the proposer for that round. First, it adds the non order
// transactions provided by tendermint. The orderbook then loops through each market and tries to match as many
// transactions as possible. For each new transaction it checks that the max bytes has not been exceeded.
func (sm *StateMachine) PrepareProposal(req types.RequestPrepareProposal) types.ResponsePrepareProposal {
// declare transaction with the size of 0
txs := make([][]byte, 0)
// go through the transactions passed up via Tendermint first
for _, tx := range req.Txs {
var msg = new(Msg)
err := proto.Unmarshal(tx, msg)
if err != nil {
panic(err)
}
// skip over the bids and asks that are proposed. We already have them
if _, ok := msg.Sum.(*Msg_MsgBid); ok {
continue
}
if _, ok := msg.Sum.(*Msg_MsgAsk); ok {
continue
}
// make sure we're proposing valid transactions
if status := sm.ValidateTx(msg); status != StatusOK {
continue
}
if len(txs)+len(tx) > int(req.MaxTxBytes) {
return types.ResponsePrepareProposal{Txs: txs}
}
txs = append(txs, tx)
}
// fetch and match all the bids and asks for each market and add these
for _, market := range sm.markets {
tradeSet := market.Match()
// tradesets into bytes and bytes into a transaction
if tradeSet == nil {
continue
}
fmt.Println("we have a tradeset")
tradeSet = sm.validateTradeSetAgainstState(tradeSet)
if tradeSet == nil || len(tradeSet.MatchedOrders) == 0 {
continue
}
fmt.Println("we have a valid tradeset")
// wrap this as a message typ
msgTradeSet := &MsgTradeSet{TradeSet: tradeSet}
bz, err := proto.Marshal(msgTradeSet)
if err != nil {
panic(err)
}
// check to see that we don't over populate the block
if len(txs)+len(bz) > int(req.MaxTxBytes) {
return types.ResponsePrepareProposal{Txs: txs}
}
txs = append(txs, bz)
}
return types.ResponsePrepareProposal{Txs: req.Txs}
}
// Process Proposal either rejects or accepts transactions
//
// It uses the same validity function for prepare proposal. This ensures the coherence property
// is adhered to i.e. all honest validators must accept a proposal by an honest proposer
func (sm *StateMachine) ProcessProposal(req types.RequestProcessProposal) types.ResponseProcessProposal {
for _, tx := range req.Txs {
var msg = new(Msg)
err := proto.Unmarshal(tx, msg)
if err != nil {
return rejectProposal()
}
if status := sm.ValidateTx(msg); status != StatusOK {
fmt.Printf("tx failed validation, status: %d\n", status)
return rejectProposal()
}
}
return acceptProposal()
}
func (sm *StateMachine) BeginBlock(req types.RequestBeginBlock) types.ResponseBeginBlock {
// reset the new pairs
sm.newPairs = make([]*Pair, 0)
return types.ResponseBeginBlock{}
}
// DeliverTx is called for each tx in a block once it has been finalized. This is where the
// execution code lives. Most importantly it's where we update the user accounts following
// a successful order.
func (sm *StateMachine) DeliverTx(req types.RequestDeliverTx) types.ResponseDeliverTx {
var msg = new(Msg)
err := proto.Unmarshal(req.Tx, msg)
if err != nil {
return types.ResponseDeliverTx{Code: StatusErrDecoding, Log: err.Error()} // decoding error
}
if status := sm.ValidateTx(msg); status != StatusOK {
return types.ResponseDeliverTx{Code: status}
}
switch m := msg.Sum.(type) {
case *Msg_MsgRegisterPair:
sm.markets[m.MsgRegisterPair.Pair.String()] = NewMarket(m.MsgRegisterPair.Pair)
sm.pairs[m.MsgRegisterPair.Pair.String()] = m.MsgRegisterPair.Pair
sm.commodities[m.MsgRegisterPair.Pair.BuyersDenomination] = struct{}{}
sm.commodities[m.MsgRegisterPair.Pair.SellersDenomination] = struct{}{}
sm.newPairs = append(sm.newPairs, m.MsgRegisterPair.Pair)
case *Msg_MsgCreateAccount:
nextAccountID := uint64(len(sm.accounts))
sm.accounts = append(sm.accounts, &Account{
Index: nextAccountID,
PublicKey: m.MsgCreateAccount.PublicKey,
Commodities: m.MsgCreateAccount.Commodities,
})
sm.touchedAccounts[nextAccountID] = struct{}{}
sm.publicKeys[string(m.MsgCreateAccount.PublicKey)] = struct{}{}
case *Msg_MsgTradeSet:
pair := m.MsgTradeSet.TradeSet.Pair
for _, order := range m.MsgTradeSet.TradeSet.MatchedOrders {
buyer := sm.accounts[order.OrderBid.OwnerId]
seller := sm.accounts[order.OrderAsk.OwnerId]
// the buyer gets quantity of the asset that the seller was selling
buyer.AddCommodity(NewCommodity(pair.SellersDenomination, order.OrderAsk.Quantity))
// the buyer gives up quantity * ask price of the buyers denomination
buyer.SubtractCommodity(NewCommodity(pair.BuyersDenomination, order.OrderAsk.Quantity*order.OrderAsk.AskPrice))
// the seller gets quantity * ask price of the asset that the buyer was paying with
seller.AddCommodity(NewCommodity(pair.BuyersDenomination, order.OrderAsk.Quantity*order.OrderAsk.AskPrice))
// the seller gives up quantity of the commodity they were selling
seller.SubtractCommodity(NewCommodity(pair.SellersDenomination, order.OrderAsk.Quantity))
// mark that these account have been touched
sm.touchedAccounts[order.OrderBid.OwnerId] = struct{}{}
sm.touchedAccounts[order.OrderAsk.OwnerId] = struct{}{}
}
default:
return types.ResponseDeliverTx{Code: StatusErrUnknownMessage}
}
return types.ResponseDeliverTx{Code: 0}
}
// EndBlock is used to update consensus params and the validator set. For the orderbook,
// we keep both the same for thw
func (sm *StateMachine) EndBlock(req types.RequestEndBlock) types.ResponseEndBlock {
return types.ResponseEndBlock{}
}
// Commit is called to tell the app it is safe to persist state to disk.
// We now take the in-memory representation and update the parts that have
// changed on to disk.
func (sm *StateMachine) Commit() types.ResponseCommit {
batch := sm.db.NewBatch()
// write to accounts that were modified by the last block
for accountID := range sm.touchedAccounts {
value, err := proto.Marshal(sm.accounts[accountID])
if err != nil {
panic(err)
}
key := binary.BigEndian.AppendUint64(accountKey, accountID)
if err := batch.Set(key, value); err != nil {
panic(err)
}
}
// write the new pairs that were added by the last block
pairID := len(sm.pairs) - len(sm.newPairs)
for id, pair := range sm.newPairs {
value, err := proto.Marshal(pair)
if err != nil {
panic(err)
}
key := binary.BigEndian.AppendUint64(pairKey, uint64(pairID+id))
if err := batch.Set(key, value); err != nil {
panic(err)
}
}
hash := sm.hash()
err := sm.updateState(batch, sm.lastHeight+1, hash)
if err != nil {
panic(err)
}
err = batch.WriteSync()
if err != nil {
panic(err)
}
return types.ResponseCommit{Data: hash}
}
// hash is just the the sha256 of the byte representation of all accounts.
// remember that this needs to be deterministic for all state machines
func (sm *StateMachine) hash() []byte {
digest := bytes.NewBuffer(nil)
for _, account := range sm.accounts {
bz, err := proto.Marshal(account)
if err != nil {
panic(err)
}
digest.Write(bz)
}
return tmhash.Sum(digest.Bytes())
}
func (sm *StateMachine) updateState(batch dbm.Batch, height int64, hash []byte) error {
sm.lastHash = hash
sm.lastHeight = height
heightBytes := make([]byte, 8)
binary.BigEndian.PutUint64(heightBytes, uint64(height))
return batch.Set(stateKey, append(heightBytes, hash...))
}
func (sm *StateMachine) validateTradeSetAgainstState(tradeSet *TradeSet) *TradeSet {
output := &TradeSet{Pair: tradeSet.Pair}
for _, matchedOrder := range tradeSet.MatchedOrders {
if !sm.isMatchedOrderValid(matchedOrder, tradeSet.Pair) {
continue
}
// yayy! this matched order is still valid and can be executed
output.MatchedOrders = append(output.MatchedOrders, matchedOrder)
}
return output
}
// isMatchedOrderValid is a check against current state to ensure that the order
// is valid and can execute.
//
// This method is also called when preparing a proposal since `CheckTx` doesn't have
// strict validity guarantees and there could be invalid transactions within the mempool
//
// Note: if one of the two orders are invalid we discard both. In the future we could
// improve this by adding back the part of the order that might still be valid.
func (sm *StateMachine) isMatchedOrderValid(order *MatchedOrder, pair *Pair) bool {
if int(order.OrderBid.OwnerId) >= len(sm.accounts) {
return false
}
bidOwner := sm.accounts[order.OrderBid.OwnerId]
if bidOwner == nil {
return false
}
if int(order.OrderAsk.OwnerId) >= len(sm.accounts) {
return false
}
askOwner := sm.accounts[order.OrderAsk.OwnerId]
if askOwner == nil {
return false
}
askCommodities := askOwner.FindCommidity(pair.SellersDenomination)
if askCommodities == nil {
return false
}
buyCommodities := bidOwner.FindCommidity(pair.BuyersDenomination)
if buyCommodities == nil {
return false
}
// Seller has enough of the commodity
if askCommodities.Quantity-order.OrderAsk.Quantity < 0 {
return false
}
// Buyer has enough of the buying commodity
if buyCommodities.Quantity-(order.OrderAsk.AskPrice*order.OrderAsk.Quantity) < 0 {
return false
}
if !order.OrderAsk.ValidateSignature(ed25519.PubKey(askOwner.PublicKey), pair) {
return false
}
if !order.OrderBid.ValidateSignature(ed25519.PubKey(bidOwner.PublicKey), pair) {
return false
}
return true
}
// InitDB takes an empty DB instance and populates it with the
// provided pairs and accounts. Note that the order here is important
func InitDB(db dbm.DB, pairs []*Pair, accounts []*Account) error {
batch := db.NewBatch()
for id, account := range accounts {
value, err := proto.Marshal(account)
if err != nil {
return err
}
key := binary.BigEndian.AppendUint64(accountKey, uint64(id))
if err := batch.Set(key, value); err != nil {
return err
}
}
for id, pair := range pairs {
value, err := proto.Marshal(pair)
if err != nil {
return err
}
key := binary.BigEndian.AppendUint64(pairKey, uint64(id))
fmt.Println(key)
if err := batch.Set(key, value); err != nil {
return err
}
}
return batch.WriteSync()
}
func rejectProposal() types.ResponseProcessProposal {
return types.ResponseProcessProposal{Status: types.ResponseProcessProposal_REJECT}
}
func acceptProposal() types.ResponseProcessProposal {
return types.ResponseProcessProposal{Status: types.ResponseProcessProposal_ACCEPT}
}

View File

@@ -0,0 +1,327 @@
package orderbook_test
import (
fmt "fmt"
"testing"
"github.com/cosmos/gogoproto/proto"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
"github.com/tendermint/tendermint/abci/example/orderbook"
"github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
params "github.com/tendermint/tendermint/types"
)
// TODO: we should also check that CheckTx adds bids and asks to the app-side mempool
func TestCheckTx(t *testing.T) {
db := dbm.NewMemDB()
require.NoError(t, orderbook.InitDB(db, []*orderbook.Pair{testPair}, nil))
app, err := orderbook.New(db)
require.NoError(t, err)
testCases := []struct {
name string
msg *orderbook.Msg
responseCode uint32
expOrderSize int
}{
{
name: "test empty tx",
msg: &orderbook.Msg{},
responseCode: orderbook.StatusErrValidateBasic,
expOrderSize: 0,
},
{
name: "test msg ask",
msg: &orderbook.Msg{
Sum: &orderbook.Msg_MsgAsk{
MsgAsk: &orderbook.MsgAsk{
Pair: testPair,
AskOrder: &orderbook.OrderAsk{
Quantity: 10,
AskPrice: 1,
OwnerId: 1,
Signature: crypto.CRandBytes(ed25519.SignatureSize),
},
},
},
},
responseCode: orderbook.StatusOK,
expOrderSize: 1,
},
{
name: "test msg ask wrong signature",
msg: &orderbook.Msg{
Sum: &orderbook.Msg_MsgAsk{
MsgAsk: &orderbook.MsgAsk{
Pair: testPair,
AskOrder: &orderbook.OrderAsk{
Quantity: 10,
AskPrice: 1,
OwnerId: 1,
Signature: crypto.CRandBytes(62),
},
},
},
},
responseCode: orderbook.StatusErrValidateBasic,
expOrderSize: 1,
},
{
name: "test msg bid",
msg: &orderbook.Msg{Sum: &orderbook.Msg_MsgBid{MsgBid: &orderbook.MsgBid{
Pair: testPair,
BidOrder: &orderbook.OrderBid{
MaxQuantity: 15,
MaxPrice: 5,
OwnerId: 1,
Signature: crypto.CRandBytes(ed25519.SignatureSize),
},
}}},
responseCode: orderbook.StatusOK,
expOrderSize: 2,
},
{
name: "test msg bid blank",
msg: &orderbook.Msg{Sum: &orderbook.Msg_MsgBid{MsgBid: &orderbook.MsgBid{
Pair: testPair,
BidOrder: &orderbook.OrderBid{
MaxQuantity: 0,
MaxPrice: 0,
OwnerId: 0,
Signature: crypto.CRandBytes(ed25519.SignatureSize),
},
}}},
responseCode: orderbook.StatusErrValidateBasic,
expOrderSize: 2,
},
{
name: "test msg register duplicate pair",
msg: &orderbook.Msg{Sum: &orderbook.Msg_MsgRegisterPair{MsgRegisterPair: &orderbook.MsgRegisterPair{
Pair: &orderbook.Pair{BuyersDenomination: "ATOM", SellersDenomination: "ATOM"},
}}},
responseCode: orderbook.StatusErrValidateBasic,
expOrderSize: 2,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
bz, err := proto.Marshal(tc.msg)
require.NoError(t, err)
resp := app.CheckTx(types.RequestCheckTx{Tx: bz})
require.Equal(t, tc.responseCode, resp.Code, resp.Log)
bids, asks := app.Orders(testPair)
require.Equal(t, tc.expOrderSize, len(bids)+len(asks))
})
}
}
// func ValidateTx(t *testing.T) {
// db := dbm.NewMemDB()
// require.NoError(t, orderbook.InitDB(db, []*orderbook.Pair{testPair}, nil))
// app, err := orderbook.New(db)
// require.NoError(t, err)
// for _, tc := range testCases {
// t.Run(tc.name, func(t *testing.T) {
// bz, err := proto.Marshal(tc.msg)
// require.NoError(t, err)
// resp := app.CheckTx(types.RequestCheckTx{Tx: bz})
// require.Equal(t, tc.responseCode, resp.Code, resp.Log)
// bids, asks := app.Orders(testPair)
// require.Equal(t, tc.expOrderSize, len(bids)+len(asks))
// })
// }
// }
// TODO: we should check that transactions in
// a market are being validated and added to the proposal
// // and that other transactions get in
// func TestPrepareProposal(t *testing.T) {
// db := dbm.NewMemDB()
// require.NoError(t, orderbook.InitDB(db, []*orderbook.Pair{testPair}, nil))
// app, err := orderbook.New(db)
// require.NoError(t, err)
// for _, tc := range testCases {
// t.Run(tc.name, func(t *testing.T) {
// bz, err := proto.Marshal(tc.msg)
// require.NoError(t, err)
// resp := app.CheckTx(types.RequestCheckTx{Tx: bz})
// require.Equal(t, tc.responseCode, resp.Code, resp.Log)
// bids, asks := app.Orders(testPair)
// require.Equal(t, tc.expOrderSize, len(bids)+len(asks))
// })
// }
// }
// {
// name: "test msg register pair",
// msg: &orderbook.Msg{Sum: &orderbook.Msg_MsgRegisterPair{MsgRegisterPair: &orderbook.MsgRegisterPair{
// Pair: &orderbook.Pair{BuyersDenomination: "ATOM", SellersDenomination: "AUD"},
// }}},
// responseCode: orderbook.StatusOK,
// expOrderSize: 2,
// pairSize: 2,
// },
// TODO: we should test that transactions are
// always valid i.e. ValidateTx. We could potentially
// combine this with PrepareProposal
// func TestProcessProposal(t *testing.T) {
// app := orderbook.New(dbm.NewMemDB())
// }
// TODO: we should test that a matched order
// correctly updates the accounts. We should
// also test that committing a block persists
// it to the database and that we can now
// query the new state
// func TestFinalizeBlock(t *testing.T) {
// app := orderbook.New(dbm.NewMemDB())
// }
// TODO: test that we can start from new
// and from existing state
// func TestNewStateMachine(t *testing.T) {}
func TestEndToEnd(t *testing.T) {
db := dbm.NewMemDB()
app, err := orderbook.New(db)
require.NoError(t, err)
var (
maxBytes = params.DefaultConsensusParams().Block.MaxBytes
commodityNZD = &orderbook.Commodity{Denom: "NZD", Quantity: 100}
commodityAUD = &orderbook.Commodity{Denom: "AUD", Quantity: 100}
registerPairMsg = newRegisterPair("NZD", "AUD")
pair = registerPairMsg.GetMsgRegisterPair().Pair
pkAlice = ed25519.GenPrivKey()
pkBob = ed25519.GenPrivKey()
pubKeyAlice = pkAlice.PubKey().Bytes()
pubKeyBob = pkBob.PubKey().Bytes()
registerAlice = newRegisterAccount(pubKeyAlice, []*orderbook.Commodity{commodityAUD})
registerBob = newRegisterAccount(pubKeyBob, []*orderbook.Commodity{commodityNZD})
// bob is asking for 25 AUD for 5 NZD
ask = &orderbook.Msg{Sum: &orderbook.Msg_MsgAsk{MsgAsk: orderbook.NewMsgAsk(pair, 5, 5, 1)}}
// alice is bidding for 5 NZD for 25 AUD
bid = &orderbook.Msg{Sum: &orderbook.Msg_MsgBid{MsgBid: orderbook.NewMsgBid(pair, 5, 5, 0)}}
)
require.NoError(t, ask.GetMsgAsk().Sign(pkBob))
require.NoError(t, bid.GetMsgBid().Sign(pkAlice))
testCases := []struct {
txs [][]byte
accepted bool
// assertions to be made about the state of the application
// after each block
assertions func(t *testing.T, app *orderbook.StateMachine)
}{
{
// block 1 sets up the trading pair
txs: asTxs(registerPairMsg),
accepted: true,
assertions: func(t *testing.T, app *orderbook.StateMachine) {
pairs := app.Pairs()
require.Len(t, pairs, 1)
require.Equal(t, pair, &pairs[0])
},
},
{
// block 2 registers two accounts: alice and bob
txs: asTxs(registerAlice, registerBob),
accepted: true,
assertions: func(t *testing.T, app *orderbook.StateMachine) {
alice := app.Account(0)
require.False(t, alice.IsEmpty(), alice)
require.Equal(t, pubKeyAlice, alice.PublicKey)
require.Len(t, alice.Commodities, 1)
require.Equal(t, alice.Commodities[0], commodityAUD)
bob := app.Account(1)
require.False(t, bob.IsEmpty(), bob)
require.Equal(t, pubKeyBob, bob.PublicKey)
require.Len(t, bob.Commodities, 1)
require.Equal(t, bob.Commodities[0], commodityNZD)
require.True(t, app.Account(2).IsEmpty())
},
},
{
// block 3 performs a trade between alice and bob
txs: asTxs(ask, bid),
accepted: true,
assertions: func(t *testing.T, app *orderbook.StateMachine) {
alice := app.Account(0)
require.Equal(t, alice.Commodities[0].Quantity, 75) // 75 AUD
require.Equal(t, alice.Commodities[1].Quantity, 5) // 5 NZD
bob := app.Account(1)
require.Equal(t, bob.Commodities[0].Quantity, 95) // 95 NZD
require.Equal(t, bob.Commodities[0].Quantity, 5) // 5 AUD
},
},
}
for idx, tc := range testCases {
for _, tx := range tc.txs {
resp := app.CheckTx(types.RequestCheckTx{Tx: tx})
require.EqualValues(t, orderbook.StatusOK, resp.Code)
}
txs := app.PrepareProposal(types.RequestPrepareProposal{MaxTxBytes: maxBytes, Txs: tc.txs}).Txs
require.Equal(t, txs, tc.txs)
if idx == 2 {
fmt.Print(tc.txs)
fmt.Println()
fmt.Print(txs)
}
result := app.ProcessProposal(types.RequestProcessProposal{Txs: txs})
if tc.accepted {
require.Equal(t, types.ResponseProcessProposal_ACCEPT, result.Status)
} else {
require.Equal(t, types.ResponseProcessProposal_REJECT, result.Status)
continue
}
app.BeginBlock(types.RequestBeginBlock{})
for _, tx := range txs {
app.DeliverTx(types.RequestDeliverTx{Tx: tx})
}
app.EndBlock(types.RequestEndBlock{})
app.Commit()
if tc.assertions != nil {
tc.assertions(t, app)
}
}
}
func asTxs(msgs ...*orderbook.Msg) [][]byte {
output := make([][]byte, len(msgs))
for i, msg := range msgs {
bz, err := proto.Marshal(msg)
if err != nil {
panic(err)
}
output[i] = bz
}
return output
}
func newRegisterPair(d1, d2 string) *orderbook.Msg {
return &orderbook.Msg{Sum: &orderbook.Msg_MsgRegisterPair{MsgRegisterPair: &orderbook.MsgRegisterPair{
Pair: &orderbook.Pair{BuyersDenomination: d1, SellersDenomination: d2},
}}}
}
func newRegisterAccount(pubkey []byte, commodities []*orderbook.Commodity) *orderbook.Msg {
return &orderbook.Msg{Sum: &orderbook.Msg_MsgCreateAccount{MsgCreateAccount: &orderbook.MsgCreateAccount{
PublicKey: pubkey,
Commodities: commodities,
}}}
}

View File

@@ -0,0 +1,9 @@
version: v1
plugins:
- name: gogofaster
out: .
opt:
- Mgoogle/protobuf/timestamp.proto=github.com/cosmos/gogoproto/types
- Mgoogle/protobuf/duration.proto=github.com/golang/protobuf/ptypes/duration
- plugins=grpc
- paths=source_relative

View File

@@ -0,0 +1,243 @@
package main
import (
"fmt"
"os"
"path/filepath"
"github.com/spf13/cobra"
"github.com/spf13/viper"
"github.com/tendermint/tendermint/abci/example/orderbook"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/libs/log"
tmos "github.com/tendermint/tendermint/libs/os"
tmrand "github.com/tendermint/tendermint/libs/rand"
"github.com/tendermint/tendermint/node"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/privval"
"github.com/tendermint/tendermint/proxy"
"github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
)
func main() {
NewCLI().Run()
}
type CLI struct {
root *cobra.Command
config *cfg.Config
}
func NewCLI() *CLI {
cli := &CLI{}
cli.root = &cobra.Command{
Use: "orderbook",
Short: "orderbook abci++ example",
}
cli.root.AddCommand(&cobra.Command{
Use: "init",
Short: "initialize the file system for an orderbook node",
PersistentPreRunE: func(cmd *cobra.Command, args []string) error {
root, err := os.Getwd()
if err != nil {
return err
}
viper.AddConfigPath(filepath.Join(root, "config"))
viper.SetConfigName("config")
config := cfg.DefaultConfig()
if err := viper.ReadInConfig(); err != nil {
if _, ok := err.(viper.ConfigFileNotFoundError); ok {
// Config file not found; use default
// This often happens when initializing a config for the first time
} else {
return err
}
} else {
if err := viper.Unmarshal(config); err != nil {
return err
}
}
config.SetRoot(root)
cli.config = config
return nil
},
RunE: func(cmd *cobra.Command, args []string) error {
privValKeyFile := cli.config.PrivValidatorKeyFile()
privValStateFile := cli.config.PrivValidatorStateFile()
var pv *privval.FilePV
if tmos.FileExists(privValKeyFile) {
pv = privval.LoadFilePV(privValKeyFile, privValStateFile)
fmt.Print("found private validator", "keyFile", privValKeyFile,
"stateFile", privValStateFile)
} else {
pv = privval.GenFilePV(privValKeyFile, privValStateFile)
pv.Save()
fmt.Print("Generated private validator", "keyFile", privValKeyFile,
"stateFile", privValStateFile)
}
nodeKeyFile := cli.config.NodeKeyFile()
if tmos.FileExists(nodeKeyFile) {
fmt.Print("Found node key", "path", nodeKeyFile)
} else {
if _, err := p2p.LoadOrGenNodeKey(nodeKeyFile); err != nil {
return err
}
fmt.Print("Generated node key", "path", nodeKeyFile)
}
// genesis file
genFile := cli.config.GenesisFile()
if tmos.FileExists(genFile) {
fmt.Print("Found genesis file", "path", genFile)
} else {
genDoc := types.GenesisDoc{
ChainID: fmt.Sprintf("orderbook-chain-%v", tmrand.Int()),
GenesisTime: tmtime.Now(),
ConsensusParams: types.DefaultConsensusParams(),
}
pubKey, err := pv.GetPubKey()
if err != nil {
return fmt.Errorf("can't get pubkey: %w", err)
}
genDoc.Validators = []types.GenesisValidator{{
Address: pubKey.Address(),
PubKey: pubKey,
Power: 10,
}}
if err := genDoc.SaveAs(genFile); err != nil {
return err
}
fmt.Print("Generated genesis file", "path", genFile)
}
return nil
},
})
cli.root.AddCommand(&cobra.Command{
Use: "run",
Short: "runs an orderbook node",
RunE: func(cmd *cobra.Command, args []string) error {
dbProvider := node.DefaultDBProvider
appDB, err := dbProvider(&node.DBContext{"orderbook", cli.config})
if err != nil {
return err
}
app, err := orderbook.New(appDB)
if err != nil {
return err
}
nodeKey, err := p2p.LoadOrGenNodeKey(cli.config.NodeKeyFile())
if err != nil {
return fmt.Errorf("failed to load or gen node key %s: %w", cli.config.NodeKeyFile(), err)
}
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
n, err := node.NewNode(
cli.config,
privval.LoadOrGenFilePV(cli.config.PrivValidatorKeyFile(), cli.config.PrivValidatorStateFile()),
nodeKey,
proxy.NewLocalClientCreator(app),
node.DefaultGenesisDocProviderFunc(cli.config),
dbProvider,
node.DefaultMetricsProvider(cli.config.Instrumentation),
logger,
)
if err != nil {
return err
}
if err := n.Start(); err != nil {
return err
}
tmos.TrapSignal(logger, func() {
if err := n.Stop(); err != nil {
logger.Error("unable to stop the node", "error", err)
}
})
return nil
},
})
cli.root.AddCommand(&cobra.Command{
Use: "create-account [commodities...]",
Short: "creates a new account message and submits it to the chain",
Example: "create-account 500BTC 10000USD",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
cli.root.AddCommand(&cobra.Command{
Use: "create-pair buyers-denomination sellers-denomination",
Short: "creates a new pair message and submits it to the chain",
Example: "create-pair BTC USD",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
cli.root.AddCommand(&cobra.Command{
Use: "bid buying-commodity price",
Short: "creates a bid message and submits it to the chain",
Example: "bid 10BTC 15000BTC/USD",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
cli.root.AddCommand(&cobra.Command{
Use: "ask selling-commodity price",
Short: "creates an ask message and submits it to the chain",
Example: "ask 5BTC 12000BTC/USD",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
querySubcommand := &cobra.Command{
Use: "query",
Short: "query the bal",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
}
querySubcommand.AddCommand(&cobra.Command{
Use: "account pubkey|id",
Short: "query the balance of an account",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
querySubcommand.AddCommand(&cobra.Command{
Use: "pairs",
Short: "list all the trading pairs",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
querySubcommand.AddCommand(&cobra.Command{
Use: "orders pair",
Short: "list all current orders for a given pair",
Example: "orders BTC/USD",
RunE: func(cmd *cobra.Command, args []string) error {
return nil
},
})
cli.root.AddCommand(querySubcommand)
return cli
}
// Run runs the CLI.
func (cli *CLI) Run() {
if err := cli.root.Execute(); err != nil {
fmt.Print(err)
os.Exit(1)
}
}

View File

@@ -0,0 +1,19 @@
//go:generate go install github.com/bufbuild/buf/cmd/buf
//go:generate buf generate
// The orderbook presents a more advanced example of a Tendermint application than the simple kvstore
//
// An orderbook is a tool used in financial markets for enabling trading of various commodities. Without
// delving into too much detail, an orderbook is made of two types of transactions: Bids and Asks. An Ask
// is an offer by a seller for n amount of a commodity at an AskPrice and a bid is an offer from a buyer
// for m amount of a commodity at a BidPrice. When the bid price exceeds the ask price, and the buyer quantity
// is less than or equal to the sellers quantity, the order is matched. In actual terms, we neglect the
// underlying denomination (i.e. USD) and effectively both participants are simultaneously a buyer and seller.
//
// This example falls far short of being a decentralized orderbook, but demonstrates how one can build an
// app-side mempool, how one can use PrepareProposal and ProcessProposal to craft complex transactions,
// how we can use signatures and validate transactions against state. How applications can manage concurrency,
// and demonstrate the lifecycle of transactions from RPC -> CheckTx -> Mempool -> PrepareProposal -> ProcessProposal
// -> DeliverTx -> Commit -> Querying
package orderbook

View File

@@ -0,0 +1,254 @@
package orderbook
import (
"container/heap"
sync "sync"
)
type Market struct {
// immutable
pair *Pair // i.e. EUR/USD (a market is bidirectional)
mtx sync.RWMutex
askOrders *AskOrders // i.e. buying EUR for USD
lowestAsk float64
bidOrders *BidOrders // i.e. selling EUR for USD or buying USD for EUR
highestBid float64
}
func NewMarket(p *Pair) *Market {
askOrders := make(AskOrders, 0)
bidOrders := make(BidOrders, 0)
return &Market{pair: p, askOrders: &askOrders, bidOrders: &bidOrders}
}
func (m *Market) AddBid(b *OrderBid) {
m.mtx.Lock()
defer m.mtx.Unlock()
heap.Push(m.bidOrders, b)
if b.MaxPrice > m.highestBid {
m.highestBid = b.MaxPrice
}
}
func (m *Market) AddAsk(a *OrderAsk) {
m.mtx.Lock()
defer m.mtx.Unlock()
heap.Push(m.askOrders, a)
if a.AskPrice < m.lowestAsk || m.lowestAsk == 0 {
m.lowestAsk = a.AskPrice
}
}
// Match takes the set of bids and asks and matches them together.
// A bid matches an ask when the MaxPrice is greater than the AskPrice
// and the MaxQuantity is greater than the quantity.
func (m *Market) Match() *TradeSet {
m.mtx.Lock()
defer m.mtx.Unlock()
// if one side doesn't have any orders than there is nothing to match
// and we return early
if m.askOrders.Len() == 0 || m.bidOrders.Len() == 0 {
return nil
}
if m.highestBid < m.lowestAsk {
// no orders match, we return early.
return nil
}
t := &TradeSet{Pair: m.pair}
bids := make([]*OrderBid, 0)
asks := make([]*OrderAsk, 0)
// get all the bids that are greater than the lowest ask. In order from heighest bid to lowest bid
for m.bidOrders.Len() > 0 {
bid := heap.Pop(m.bidOrders).(*OrderBid)
if bid.MaxPrice < m.lowestAsk {
// we've reached the limit, push the bid back and break the loop
heap.Push(m.bidOrders, bid)
break
} else {
bids = append(bids, bid)
}
}
// get all the asks that are lower than the highest bid in the bids set. Ordered from lowest to highest ask
for m.askOrders.Len() > 0 {
ask := heap.Pop(m.askOrders).(*OrderAsk)
if ask.AskPrice > bids[0].MaxPrice {
// the ask price is greater than the highest bid; push the ask back and break theh loop
heap.Push(m.askOrders, ask)
break
} else {
asks = append(asks, ask)
}
}
// this is to keep track of the index of the bids that have been matched
reserved := make(map[int]struct{})
// start from the highest ask and the highest bid and for each ask loop downwards through the slice of
// bids until one is matched
OUTER_LOOP:
for i := len(asks) - 1; i >= 0; i-- {
ask := asks[i]
// start with the highest bid and increment down since we're more likely to find a match
for j := len(bids) - 1; j >= 0; j-- {
if _, ok := reserved[j]; ok {
// skip over the bids that have already been reserved
continue
}
bid := bids[j]
if bid.MaxPrice >= ask.AskPrice {
if bid.MaxQuantity >= ask.Quantity {
// yay! we have a match
t.AddFilledOrder(ask, bid)
// reserve the bid so we don't rematch it with another ask
reserved[j] = struct{}{}
continue OUTER_LOOP
}
} else {
// once we've dropped below the ask price there are no more possible bids and so we break
break
}
}
// as we go from highest to lowest, asks that aren't matched become the new lowest ask price
m.lowestAsk = ask.AskPrice
// no match found, add the ask order back into the heap
heap.Push(m.askOrders, ask)
}
// if all available asks were matched then
// we never have the opportunity to update the lowest ask.
// Now we reset it to 0
if m.askOrders.Len() == 0 {
m.lowestAsk = 0
}
// add back the unmatched bids to the heap so they can be matched again in a later round.
// We also neeed to recalculate the new highest bid. First we tackle an edge case whereby all
// selected bids were matched. In this case we grab the next highest and set that as the new
// highest bid
m.highestBid = 0
if len(reserved) == len(bids) && m.bidOrders.Len() > 0 {
newHighestBid := heap.Pop(m.bidOrders).(*OrderBid)
m.highestBid = newHighestBid.MaxPrice
heap.Push(m.bidOrders, newHighestBid)
}
for j := 0; j < len(bids); j++ {
if _, ok := reserved[j]; !ok {
if bids[j].MaxPrice > m.highestBid {
m.highestBid = bids[j].MaxPrice
}
heap.Push(m.bidOrders, bids[j])
}
}
if len(t.MatchedOrders) == 0 {
return nil
}
return t
}
func (m Market) LowestAsk() float64 {
m.mtx.RLock()
defer m.mtx.RUnlock()
return m.lowestAsk
}
func (m Market) HighestBid() float64 {
m.mtx.RLock()
defer m.mtx.RUnlock()
return m.highestBid
}
func (m Market) GetBids() []OrderBid {
m.mtx.RLock()
defer m.mtx.RUnlock()
orders := make([]OrderBid, m.bidOrders.Len())
for idx, order := range *m.bidOrders {
orders[idx] = *order
}
return orders
}
func (m Market) GetAsks() []OrderAsk {
m.mtx.RLock()
defer m.mtx.RUnlock()
orders := make([]OrderAsk, m.askOrders.Len())
for idx, order := range *m.askOrders {
orders[idx] = *order
}
return orders
}
// Heap ordered by lowest price
type AskOrders []*OrderAsk
var _ heap.Interface = (*AskOrders)(nil)
func (a AskOrders) Len() int { return len(a) }
func (a AskOrders) Less(i, j int) bool {
return a[i].AskPrice < a[j].AskPrice
}
func (a AskOrders) Swap(i, j int) {
a[i], a[j] = a[j], a[i]
}
func (a *AskOrders) Push(x any) {
item := x.(*OrderAsk)
*a = append(*a, item)
}
func (a *AskOrders) Pop() any {
old := *a
n := len(old)
item := old[n-1]
old[n-1] = nil
*a = old[0 : n-1]
return item
}
// Heap ordered by highest price
type BidOrders []*OrderBid
var _ heap.Interface = (*BidOrders)(nil)
func (b BidOrders) Len() int { return len(b) }
func (b BidOrders) Less(i, j int) bool {
return b[i].MaxPrice > b[j].MaxPrice
}
func (b BidOrders) Swap(i, j int) {
b[i], b[j] = b[j], b[i]
}
func (b *BidOrders) Push(x any) {
item := x.(*OrderBid)
*b = append(*b, item)
}
func (b *BidOrders) Pop() any {
old := *b
n := len(old)
item := old[n-1]
old[n-1] = nil
*b = old[0 : n-1]
return item
}
func (t *TradeSet) AddFilledOrder(ask *OrderAsk, bid *OrderBid) {
t.MatchedOrders = append(t.MatchedOrders, &MatchedOrder{
OrderAsk: ask,
OrderBid: bid,
})
}

View File

@@ -0,0 +1,179 @@
package orderbook_test
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/abci/example/orderbook"
)
var testPair = &orderbook.Pair{BuyersDenomination: "ATOM", SellersDenomination: "USD"}
func testBid(price, quantity float64) *orderbook.OrderBid {
return &orderbook.OrderBid{
MaxPrice: price,
MaxQuantity: quantity,
}
}
func testAsk(price, quantity float64) *orderbook.OrderAsk {
return &orderbook.OrderAsk{
AskPrice: price,
Quantity: quantity,
}
}
func TestTrackLowestAndHighestPrices(t *testing.T) {
market := orderbook.NewMarket(testPair)
require.Zero(t, market.LowestAsk())
require.Zero(t, market.HighestBid())
market.AddBid(testBid(100, 10))
require.EqualValues(t, 100, market.HighestBid())
market.AddAsk(testAsk(50, 10))
require.EqualValues(t, 50, market.LowestAsk())
market.AddAsk(testAsk(30, 10))
require.EqualValues(t, 30, market.LowestAsk())
market.AddAsk(testAsk(40, 10))
require.EqualValues(t, 30, market.LowestAsk())
}
func TestSimpleOrderMatching(t *testing.T) {
testcases := []struct {
bid *orderbook.OrderBid
ask *orderbook.OrderAsk
match bool
}{
{
bid: testBid(50, 10),
ask: testAsk(50, 10),
match: true,
},
{
bid: testBid(60, 10),
ask: testAsk(50, 10),
match: true,
},
{
bid: testBid(50, 10),
ask: testAsk(60, 10),
match: false,
},
{
bid: testBid(50, 5),
ask: testAsk(40, 10),
match: false,
},
{
bid: testBid(50, 15),
ask: testAsk(40, 10),
match: true,
},
}
for idx, tc := range testcases {
market := orderbook.NewMarket(testPair)
market.AddAsk(tc.ask)
market.AddBid(tc.bid)
resp := market.Match()
if tc.match {
require.Len(t, resp.MatchedOrders, 1, idx)
} else {
require.Nil(t, resp)
}
}
}
func TestMultiOrderMatching(t *testing.T) {
testcases := []struct {
bids []*orderbook.OrderBid
asks []*orderbook.OrderAsk
expected []*orderbook.MatchedOrder
expectedLowestAsk float64
expectedHighestBid float64
}{
{
bids: []*orderbook.OrderBid{
testBid(50, 20),
testBid(40, 10),
testBid(30, 15),
},
asks: []*orderbook.OrderAsk{
testAsk(30, 15),
testAsk(30, 5),
},
expected: []*orderbook.MatchedOrder{
{
OrderAsk: testAsk(30, 5),
OrderBid: testBid(30, 15),
},
{
OrderAsk: testAsk(30, 15),
OrderBid: testBid(50, 20),
},
},
expectedLowestAsk: 0,
expectedHighestBid: 40,
},
{
bids: []*orderbook.OrderBid{
testBid(60, 20),
testBid(80, 5),
},
asks: []*orderbook.OrderAsk{
testAsk(60, 15),
testAsk(70, 10),
testAsk(50, 20),
},
expected: []*orderbook.MatchedOrder{
{
OrderAsk: testAsk(60, 15),
OrderBid: testBid(60, 20),
},
},
expectedLowestAsk: 50,
expectedHighestBid: 80,
},
{
bids: []*orderbook.OrderBid{
testBid(60, 20),
testBid(80, 5),
},
asks: []*orderbook.OrderAsk{},
expected: []*orderbook.MatchedOrder{},
expectedLowestAsk: 0,
expectedHighestBid: 80,
},
{
bids: []*orderbook.OrderBid{},
asks: []*orderbook.OrderAsk{
testAsk(70, 10),
testAsk(50, 20),
},
expected: []*orderbook.MatchedOrder{},
expectedLowestAsk: 50,
expectedHighestBid: 0,
},
}
for idx, tc := range testcases {
market := orderbook.NewMarket(testPair)
for _, ask := range tc.asks {
market.AddAsk(ask)
}
for _, bid := range tc.bids {
market.AddBid(bid)
}
resp := market.Match()
if len(tc.expected) == 0 {
require.Nil(t, resp, idx)
} else {
require.Equal(t, tc.expected, resp.MatchedOrders, idx)
}
require.EqualValues(t, tc.expectedLowestAsk, market.LowestAsk(), idx)
require.EqualValues(t, tc.expectedHighestBid, market.HighestBid(), idx)
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,40 @@
syntax = "proto3";
package orderbook;
option go_package = "github.com/tendermint/tendermint/abci/example/orderbook";
import "wire.proto";
message MsgBid {
Pair pair = 1;
OrderBid bid_order = 2;
}
message MsgAsk {
Pair pair = 1;
OrderAsk ask_order = 2;
}
message MsgCreateAccount {
bytes public_key = 1;
repeated Commodity commodities = 2;
}
message MsgRegisterPair {
Pair pair = 1;
}
message MsgTradeSet {
TradeSet trade_set = 1;
}
message Msg {
//a Msg has to be one of the below
oneof sum {
MsgBid msg_bid = 1;
MsgAsk msg_ask = 2;
MsgRegisterPair msg_register_pair = 3;
MsgCreateAccount msg_create_account = 4;
MsgTradeSet msg_trade_set = 5;
}
}

BIN
abci/example/orderbook/orderbook Executable file

Binary file not shown.

View File

@@ -0,0 +1,31 @@
package orderbook
// Query the state of an account (returns a concrete copy)
func (sm *StateMachine) Account(id uint64) Account {
if int(id) >= len(sm.accounts) {
return Account{}
}
return *sm.accounts[id]
}
// Query all the pairs that the orderbook has (returns a concrete copy)
func (sm *StateMachine) Pairs() []Pair {
pairs := make([]Pair, len(sm.pairs))
idx := 0
for _, pair := range sm.pairs {
pairs[idx] = *pair
idx++
}
return pairs
}
// Query the current orders for a pair (returns concrete copies)
func (sm *StateMachine) Orders(pair *Pair) ([]OrderBid, []OrderAsk) {
market, ok := sm.markets[pair.String()]
if !ok {
return nil, nil
}
return market.GetBids(), market.GetAsks()
}
func (sm *StateMachine) Height() int64 { return sm.lastHeight }

View File

@@ -0,0 +1,299 @@
package orderbook
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"math"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
)
func NewMsgBid(pair *Pair, maxPrice, maxQuantity float64, ownerId uint64) *MsgBid {
return &MsgBid{
Pair: pair,
BidOrder: &OrderBid{
MaxPrice: maxPrice,
MaxQuantity: maxQuantity,
OwnerId: ownerId,
},
}
}
func (msg *MsgBid) Sign(pk crypto.PrivKey) error {
sig, err := pk.Sign(msg.BidOrder.DeterministicSignatureBytes(msg.Pair))
if err != nil {
return err
}
msg.BidOrder.Signature = sig
return nil
}
func (msg *MsgBid) ValidateBasic() error {
if err := msg.BidOrder.ValidateBasic(); err != nil {
return err
}
if err := msg.Pair.ValidateBasic(); err != nil {
return err
}
if len(msg.BidOrder.Signature) != ed25519.SignatureSize {
return errors.New("invalid signature size")
}
return nil
}
func NewMsgAsk(pair *Pair, askPrice, quantity float64, ownerId uint64) *MsgAsk {
return &MsgAsk{
Pair: pair,
AskOrder: &OrderAsk{
AskPrice: askPrice,
Quantity: quantity,
OwnerId: ownerId,
},
}
}
func (msg *MsgAsk) Sign(pk crypto.PrivKey) error {
sig, err := pk.Sign(msg.AskOrder.DeterministicSignatureBytes(msg.Pair))
if err != nil {
return err
}
msg.AskOrder.Signature = sig
return nil
}
func (msg *MsgAsk) ValidateBasic() error {
if err := msg.AskOrder.ValidateBasic(); err != nil {
return err
}
if err := msg.Pair.ValidateBasic(); err != nil {
return err
}
return nil
}
func NewMsgCreateAccount(commodities ...*Commodity) (*MsgCreateAccount, crypto.PrivKey) {
pk := ed25519.GenPrivKey()
return &MsgCreateAccount{
PublicKey: pk.PubKey().Bytes(),
Commodities: commodities,
}, pk
}
func (msg *MsgCreateAccount) ValidateBasic() error {
if len(msg.PublicKey) != ed25519.PubKeySize {
return errors.New("invalid pub key size")
}
uniqueMap := make(map[string]struct{}, len(msg.Commodities))
for _, c := range msg.Commodities {
if err := c.ValidateBasic(); err != nil {
return err
}
if _, ok := uniqueMap[c.Denom]; ok {
return fmt.Errorf("commodity %s declared twice", c.Denom)
}
uniqueMap[c.Denom] = struct{}{}
}
return nil
}
func NewMsgRegisterPair(pair *Pair) *MsgRegisterPair {
return &MsgRegisterPair{Pair: pair}
}
func (msg *MsgRegisterPair) ValidateBasic() error {
return msg.Pair.ValidateBasic()
}
func NewCommodity(denom string, quantity float64) *Commodity {
return &Commodity{
Denom: denom,
Quantity: quantity,
}
}
func (c *Commodity) ValidateBasic() error {
if c.Quantity <= 0 {
return errors.New("quantity must be greater than zero")
}
return nil
}
func (p *Pair) ValidateBasic() error {
if p.BuyersDenomination == "" || p.SellersDenomination == "" {
return errors.New("inbound and outbound commodities must be present")
}
if p.BuyersDenomination == p.SellersDenomination {
return errors.New("commodities must not be the same")
}
return nil
}
func (o *OrderBid) ValidateBasic() error {
if o.MaxQuantity == 0 {
return errors.New("max quantity must be non zero")
}
if o.MaxPrice <= 0 {
return errors.New("min price must be greater than 0")
}
if len(o.Signature) != ed25519.SignatureSize {
return errors.New("invalid signature size")
}
return nil
}
func (o *OrderBid) ValidateSignature(pk crypto.PubKey, pair *Pair) bool {
return pk.VerifySignature(o.DeterministicSignatureBytes(pair), o.Signature)
}
func (o *OrderBid) DeterministicSignatureBytes(pair *Pair) []byte {
buf := bytes.NewBuffer(nil)
buf.WriteString(pair.SellersDenomination)
buf.WriteString(pair.BuyersDenomination)
bz := buf.Bytes()
bz = binary.BigEndian.AppendUint64(bz, math.Float64bits(o.MaxQuantity))
bz = binary.BigEndian.AppendUint64(bz, math.Float64bits(o.MaxPrice))
return bz
}
func (m *MatchedOrder) ValidateBasic() error {
if err := m.OrderAsk.ValidateBasic(); err != nil {
return err
}
if err := m.OrderBid.ValidateBasic(); err != nil {
return err
}
return nil
}
func (t *TradeSet) ValidateBasic() error {
for _, matchedOrder := range t.MatchedOrders {
if err := matchedOrder.ValidateBasic(); err != nil {
return err
}
// checking if there is an account
if matchedOrder.OrderAsk.OwnerId == 0 {
return errors.New("must have an owner id more than zero")
}
}
// validate the pairs are valid
if err := t.Pair.ValidateBasic(); err != nil {
return err
}
return nil
}
func (o *OrderAsk) ValidateBasic() error {
if o.Quantity == 0 {
return errors.New("quantity outbound must be non zero")
}
if o.AskPrice <= 0 {
return errors.New("min price must be greater than 0")
}
if len(o.Signature) != ed25519.SignatureSize {
return errors.New("invalid signature size")
}
return nil
}
func (o *OrderAsk) ValidateSignature(pk crypto.PubKey, pair *Pair) bool {
return pk.VerifySignature(o.DeterministicSignatureBytes(pair), o.Signature)
}
func (o *OrderAsk) DeterministicSignatureBytes(pair *Pair) []byte {
buf := bytes.NewBuffer(nil)
buf.WriteString(pair.BuyersDenomination)
buf.WriteString(pair.SellersDenomination)
bz := buf.Bytes()
bz = binary.BigEndian.AppendUint64(bz, math.Float64bits(o.Quantity))
bz = binary.BigEndian.AppendUint64(bz, math.Float64bits(o.AskPrice))
return bz
}
func (a Account) IsEmpty() bool {
return len(a.PublicKey) == 0
}
func (a *Account) FindCommidity(denom string) *Commodity {
for _, c := range a.Commodities {
if c.Denom == denom {
return c
}
}
return nil
}
func (a *Account) AddCommodity(c *Commodity) {
curr := a.FindCommidity(c.Denom)
if curr == nil {
a.Commodities = append(a.Commodities, c)
} else {
curr.Quantity += c.Quantity
}
}
func (a *Account) SubtractCommodity(c *Commodity) {
curr := a.FindCommidity(c.Denom)
if curr == nil {
panic("trying to remove a commodity the account does not have")
}
curr.Quantity -= c.Quantity
}
func (msg *Msg) ValidateBasic() error {
switch m := msg.Sum.(type) {
case *Msg_MsgRegisterPair:
if err := m.MsgRegisterPair.ValidateBasic(); err != nil {
return err
}
case *Msg_MsgCreateAccount:
if err := m.MsgCreateAccount.ValidateBasic(); err != nil {
return err
}
case *Msg_MsgBid:
if err := m.MsgBid.ValidateBasic(); err != nil {
return err
}
case *Msg_MsgAsk:
if err := m.MsgAsk.ValidateBasic(); err != nil {
return err
}
case *Msg_MsgTradeSet:
if err := m.MsgTradeSet.TradeSet.ValidateBasic(); err != nil {
return err
}
default:
return errors.New("unknown tx")
}
return nil
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,52 @@
syntax = "proto3";
package orderbook;
option go_package = "github.com/tendermint/tendermint/abci/example/orderbook";
message OrderAsk {
double quantity = 1;
double ask_price = 2;
uint64 owner_id = 3;
bytes signature = 4;
}
message OrderBid {
double max_quantity = 1;
double max_price = 2;
uint64 owner_id = 3;
bytes signature = 4;
}
message Pair {
// the denomination that the buyer receives i.e. EUR
string buyers_denomination = 1;
// the denomination that the seller receives i.e. USD
string sellers_denomination = 2;
}
message Commodity {
string denom = 1;
double quantity = 2;
}
// Accounts is the atomic piece of information that is persisted to disk.
message Account {
uint64 index = 1;
bytes public_key = 2;
// the set of commodities that the account has
repeated Commodity commodities = 3;
}
// TradeSet is the transaction that eventually is committed in a block
// It is derived from a group of MsgBid and MsgAsk's
message TradeSet {
Pair pair = 1; // i.e. EUR/USD
// the set of matched trades for that peer
repeated MatchedOrder matched_orders = 2;
}
message MatchedOrder {
OrderAsk order_ask = 1;
OrderBid order_bid = 2;
}

View File

@@ -27,6 +27,7 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
// PublicKey defines the keys available for use with Tendermint Validators
type PublicKey struct {
// Types that are valid to be assigned to Sum:
//
// *PublicKey_Ed25519
// *PublicKey_Secp256K1
Sum isPublicKey_Sum `protobuf_oneof:"sum"`

View File

@@ -68,6 +68,7 @@ func (m *Txs) GetTxs() [][]byte {
type Message struct {
// Types that are valid to be assigned to Sum:
//
// *Message_Txs
Sum isMessage_Sum `protobuf_oneof:"sum"`
}

View File

@@ -158,6 +158,7 @@ func (m *PacketMsg) GetData() []byte {
type Packet struct {
// Types that are valid to be assigned to Sum:
//
// *Packet_PacketPing
// *Packet_PacketPong
// *Packet_PacketMsg

View File

@@ -24,6 +24,7 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
type Message struct {
// Types that are valid to be assigned to Sum:
//
// *Message_SnapshotsRequest
// *Message_SnapshotsResponse
// *Message_ChunkRequest