Engine & Matching System - Technical Overview
TL;DR
Order Matching:
- Price-Time Priority - Best price wins, then earliest order at that price
- Uses BTreeMap data structures for O(log n) order book operations
- Single write lock during command execution ensures determinism
Hot Path Flow:
Redis Stream → Validation → Execution (write lock) → Settlement → Output EventsOrder Book Structure:
limit_asks: BTreeMap<Price, OrderPage>- Ascending (best ask = lowest price first)limit_bids: BTreeMap<Price, OrderPage>- Descending (best bid = highest price first)OrderPagecontainsBTreeMap<OrderId, LimitOrder>(time priority within price level)
Key Events:
OrderEvent::Accepted→FillEvent(0..n) →OrderEvent::Placed→DepthEvent→BalanceEvent
Communication:
- Input: Redis Streams (commands from API)
- Output: Redis Streams + PubSub (events to websockets, other services)
- Cluster: Gossip protocol for consensus across engine nodes
Multi-Node Architecture:
- Multiple nodes process same commands for fault tolerance (not load balancing)
- Writes don't scale horizontally - all nodes process everything
- Reads can be load balanced across any node
- Real scaling comes from sharding by market (BTC cluster, SOL cluster, etc.)
- Gossip protocol verifies all nodes have identical state via hash comparison
Table of Contents
- Order Matching Algorithm
- Order Book Structure
- Order Processing Hot Path
- Events & Messages
- Inter-Service Communication
- Multi-Node Architecture & Scaling
- Determinism & Reproducibility
- Engine Initialization
1. Order Matching Algorithm
Price-Time Priority (FIFO)
Backpack uses the standard price-time priority matching algorithm:
Price Priority - Better prices get matched first
- For asks: Lower price = better (you're selling cheaper)
- For bids: Higher price = better (you're buying higher)
Time Priority - At the same price, earlier orders get matched first
- Implemented via
BTreeMap<OrderId, LimitOrder>where OrderId is monotonically increasing
- Implemented via
Matching Flow
┌─────────────────────────────────────────────────────────────┐
│ OrderBook::execute() │
├──────────────────────────────────────────────────────────────┤
│ 1. Rescale order decimals to tick/step size │
│ 2. Emit OrderEvent::Accepted │
│ 3. Route to execute_limit() or execute_market() │
└─────────────────────────────────────────────────────────────┘
│
▼
┌─────────────────────────────────────────────────────────────┐
│ match_orders() │
├──────────────────────────────────────────────────────────────┤
│ For each price level (best to worst): │
│ - Check price band filters │
│ - Check slippage tolerance │
│ - Process orders at this level │
│ - Track depth changes │
│ - Remove empty pages │
│ Until: taker filled OR expired OR no more liquidity │
└─────────────────────────────────────────────────────────────┘
│
▼
┌─────────────────────────────────────────────────────────────┐
│ OrderPage::process_orders() │
├──────────────────────────────────────────────────────────────┤
│ For each maker at this price (time order): │
│ 1. Check self-trade prevention │
│ 2. Calculate fill quantity: │
│ trade_qty = min(maker_remaining, taker_remaining) │
│ trade_quote_qty = trade_qty × price │
│ 3. Update maker and taker with fill_quantity() │
│ 4. Calculate fees (maker_fee, taker_fee) │
│ 5. Emit FillEvent │
│ 6. Remove filled orders from page │
└─────────────────────────────────────────────────────────────┘Limit vs Market Order Execution
| Order Type | Matching Behavior | Unfilled Portion |
|---|---|---|
| Limit | Match up to limit price | Placed on book (unless IOC) |
| Market | Match at any available price | Expires (no resting) |
Limit Order Flow:
// execute_limit() pseudocode
fills = match_orders(taker_order)
if taker_order.is_unfilled() && !taker_order.is_ioc() {
add_to_book(taker_order) // Resting limit order
emit(OrderEvent::Placed)
}Market Order Flow:
// execute_market() pseudocode
fills = match_orders(taker_order)
// Market orders NEVER rest on book
// Any unfilled quantity expiresSelf-Trade Prevention
The engine prevents an account from trading with itself:
if taker.account_id == maker.account_id {
skip_this_maker() // Don't fill against yourself
}Fee Calculation
Fees differ by market type:
| Market | Maker Fee | Taker Fee | Fee Asset |
|---|---|---|---|
| Spot | Low (often rebate) | Higher | Base or Quote |
| Perpetual | Low | Higher | Always USDC |
| Prediction | Low | Higher | Always USDC |
// Simplified fee calculation
pub fn get_fees(trade_qty, trade_quote_qty) -> FillFee {
// Spot: fees in base/quote depending on side
// Perp/Prediction: fees always in quote (USDC), rounded up
FillFee { maker_fee, taker_fee }
}Key files:
/engine/src/models/order_book/mod.rs- Main matching logic (2,118 lines)/engine/src/models/order_book/order_page.rs- Per-price-level processing/engine/src/models/order_book/order_pair.rs- Fill and fee calculations
2. Order Book Structure
Core Data Structure
pub struct OrderBook {
// State
pub state: OrderBookState, // Closed, PostOnly, CancelOnly, Open
pub filters: OrderBookFilters, // Price/qty/leverage rules
// The actual book (price levels)
pub limit_asks: BTreeMap<Price, OrderPage>, // Ascending price
pub limit_bids: BTreeMap<Price, OrderPage>, // Descending price
// Indexes for fast lookup
pub client_order_index: HashMap<(AccountId, ClientOrderId), OrderId>,
pub limit_order_index: OrderIdMap<Price>,
// Trigger orders (stop-loss, take-profit)
pub trigger_orders: TriggerOrders,
pub trigger_order_index: AccountIdMap<OrderIdSet>,
// Trade history
pub recent_fills: CircularQueue<FillEvent>, // Last 1000 trades
pub last_trade_id: i64,
pub last_update_id: i64,
}Why BTreeMap? (vs HashMap)
A BTreeMap is a sorted dictionary - keys are always kept in order. A HashMap stores keys in random order (hashed).
| Operation | HashMap | BTreeMap |
|---|---|---|
| Get/Insert | O(1) | O(log n) |
| Iterate in order | O(n log n) - must sort | O(n) - already sorted |
| Find min/max | O(n) - scan all | O(log n) - it's at the edge |
| Memory | Lower | Higher |
Why BTreeMap wins for order books:
Matching needs to repeatedly find the best price:
- Best ask = lowest sell price
- Best bid = highest buy price
With HashMap:
"What's the lowest ask?" → Scan ALL prices, find minimum. Slow.
With BTreeMap:
"What's the lowest ask?" → First element. Fast.Also critical for determinism: BTreeMap iteration order is deterministic (sorted). HashMap iteration order can vary between runs, which would break multi-node consensus.
OrderPage (Price Level)
pub struct OrderPage {
pub orders: BTreeMap<OrderId, LimitOrder>, // Time-priority (OrderId is monotonic)
pub quantity: Quantity, // Total quantity at this level
}Each price level maintains:
- All orders at that price, keyed by OrderId (time priority)
- Total quantity for quick depth queries
Visual Representation
OrderBook (BTC_USDC)
═══════════════════════════════════════════════════════════════
ASKS (limit_asks - ascending BTreeMap)
┌─────────────────────────────────────────────────────────────┐
│ Price: $50,100 │ OrderPage │
│ │ ├─ OrderId: 1001 → 0.5 BTC (oldest) │
│ │ ├─ OrderId: 1005 → 1.0 BTC │
│ │ └─ OrderId: 1008 → 0.3 BTC (newest) │
│ │ Total: 1.8 BTC │
├─────────────────────────────────────────────────────────────┤
│ Price: $50,050 │ OrderPage │
│ │ ├─ OrderId: 1002 → 2.0 BTC │
│ │ Total: 2.0 BTC │
├─────────────────────────────────────────────────────────────┤
│ Price: $50,000 │ OrderPage ← BEST ASK │
│ │ ├─ OrderId: 1000 → 1.0 BTC │
│ │ Total: 1.0 BTC │
└─────────────────────────────────────────────────────────────┘
↕ Spread: $100
┌─────────────────────────────────────────────────────────────┐
│ Price: $49,900 │ OrderPage ← BEST BID │
│ │ ├─ OrderId: 999 → 0.8 BTC │
│ │ Total: 0.8 BTC │
├─────────────────────────────────────────────────────────────┤
│ Price: $49,850 │ OrderPage │
│ │ ├─ OrderId: 998 → 1.5 BTC │
│ │ Total: 1.5 BTC │
└─────────────────────────────────────────────────────────────┘
BIDS (limit_bids - descending BTreeMap)Order Book States
| State | New Orders | Taker Fills | Maker Rests |
|---|---|---|---|
| Open | Yes | Yes | Yes |
| PostOnly | Post-only only | No | Yes |
| CancelOnly | No | No | No |
| Closed | No | No | No |
Key files:
/engine/src/models/order_book/mod.rs- OrderBook struct/engine/src/models/order_book/order_page.rs- OrderPage struct/engine/src/models/market.rs- Market types (Spot, Perpetual, etc.)
3. Order Processing Hot Path
Three-Stage Pipeline
┌─────────────┐ ┌─────────────┐ ┌─────────────┐
│ VALIDATE │ → │ EXECUTE │ → │ SETTLE │
│ │ │ │ │ │
│ • Signature │ │ • Write lock│ │ • Balances │
│ • Auth │ │ • Matching │ │ • Positions │
│ • Format │ │ • Events │ │ • Fees │
└─────────────┘ └─────────────┘ └─────────────┘Stage 1: Validation
File: /engine/src/processor/command/stages/validation.rs
// What validation checks:
1. Message format is valid
2. Cryptographic signature is valid
3. Signer has permission for this action
4. Pre-validation of command parameters
// Output:
enum ValidatedMessage {
ValidCommand((ReceivedMessage, SignatureData)),
InvalidCommand((ReceivedMessage, ErrorResponse)),
Ignored(ReceivedMessage),
}Stage 2: Execution
File: /engine/src/processor/command/stages/execution.rs
This is the critical hot path where matching happens:
// Execution under single write lock
{
let mut engine = engine.write(); // Single write lock
for command in batch {
let events = engine.handle_command(command);
all_events.extend(events);
}
} // Lock released
// Key insight: Entire batch runs under ONE lock
// This ensures atomic, consistent state transitionsWhy single lock?
- Prevents race conditions
- Ensures deterministic execution order
- Simplifies reasoning about state
- Trades parallelism for correctness
Stage 3: Settlement
File: /engine/src/processor/command/stages/settlement.rs
After matching, settlement updates account state:
// Settlement processes events and applies state changes:
for event in events {
match event {
FillEvent { .. } => {
// Update buyer balance: -quote, +base
// Update seller balance: +quote, -base
// Deduct fees from both
}
DepositEvent { .. } => { /* Add to balance */ }
TransferEvent { .. } => { /* Move between accounts */ }
FundingEvent { .. } => { /* Apply funding payment */ }
// ... etc
}
}Command Loop Overview
┌─────────────────────────────────────────────────────────────┐
│ Command Processing Loop │
├─────────────────────────────────────────────────────────────┤
│ │
│ while running { │
│ // 1. Receive batch from Redis stream │
│ messages = redis.xread(count: write_max_batch_size) │
│ │
│ // 2. Validate all messages │
│ validated = messages.map(|m| validate(m)) │
│ │
│ // 3. Execute under write lock │
│ events = { │
│ let mut engine = engine.write(); │
│ validated │
│ .filter(|v| v.is_valid()) │
│ .flat_map(|v| engine.execute(v)) │
│ }; │
│ │
│ // 4. Settle and emit events │
│ output_events = settle(events); │
│ output_channel.send(output_events); │
│ } │
│ │
└─────────────────────────────────────────────────────────────┘Performance Characteristics
| Aspect | Design Choice | Why |
|---|---|---|
| Lock strategy | Single write lock per batch | Determinism > parallelism |
| Batch size | Configurable (default: 10,000) | Amortize lock overhead |
| External I/O | Zero during hot path | No DB/network calls |
| Output handling | Async unbounded channel | Non-blocking hot path |
| Data structures | BTreeMap for order book | Ordered iteration O(n) |
Key files:
/engine/src/processor/command/processor.rs- Main command loop/engine/src/processor/mod.rs- Processor initialization/engine/src/engine/order/commands.rs- Order command handling (1,589 lines)
4. Events & Messages
Event Types
pub enum EngineEvent {
// Trading
Order(OrderEvent), // Order lifecycle
Fill(FillEvent), // Trade execution
Depth(DepthEvent), // Book depth changes
// Account
Account(AccountEvent), // Deposit, Withdraw, Transfer
Balance(BalanceEvent), // Balance updates
// Positions
Position(PositionEvent), // Open/close futures positions
SettleEquity(SettleEquityEvent),
// Liquidation
Liquidation(LiquidationEvent),
// Funding & Interest
Funding(FundingEvent),
Interest(InterestPaymentEvent),
// Borrow/Lend
BorrowLend(BorrowLendEvent),
// Other
RequestForQuote(RequestForQuoteEvent),
Vault(VaultEvent),
Strategy(StrategyEvent),
}Order Event Lifecycle
┌─────────────────┐
│ Order Submitted │
└────────┬────────┘
│
▼
┌─────────────────┐
│ Accepted │ ← OrderEvent::Accepted
└────────┬────────┘
│
┌──────────────┼──────────────┐
│ │ │
▼ ▼ ▼
┌──────────┐ ┌──────────┐ ┌──────────┐
│ Fills │ │ Placed │ │ Expired │
│ (0..n) │ │ on book │ │ (IOC) │
└────┬─────┘ └────┬─────┘ └──────────┘
│ │ ↑
▼ │ │
┌──────────┐ │ │
│ Filled │←──────┴──────────────┘
│ or │ (later fill or cancel)
│ Partial │
└──────────┘FillEvent Structure
pub struct FillEvent {
pub trade_id: i64, // Unique trade identifier
pub taker: OrderType, // Taker order details
pub taker_fee: Decimal, // Fee charged to taker
pub maker: OrderType, // Maker order details
pub maker_fee: Decimal, // Fee charged to maker
pub quantity: Decimal, // Trade quantity
pub price: Price, // Execution price
pub timestamp: i64, // Engine timestamp (deterministic)
}Event Sequence for Order Execution
| Step | Event | Data |
|---|---|---|
| 1 | OrderEvent::Accepted | Order enters engine |
| 2 | FillEvent (0..n) | Each trade |
| 3 | OrderEvent::Placed | If limit order rests |
| 4 | DepthEvent | Book depth changes |
| 5 | BalanceEvent | Fee deductions |
| 6 | PositionEvent | Position updates (futures) |
Output Event Wrapper
pub struct EngineOutputEvent {
pub event: EngineEvent,
pub sequence: EngineEventSequence, // Global monotonic sequence
}Every event gets a globally unique, monotonically increasing sequence number for:
- Ordering guarantees
- Gap detection
- Replay capability
Key files:
- Event types defined in
bpx_buscrate /engine/src/clearing_house/settlement/mod.rs- Event handling
5. Inter-Service Communication
Architecture Overview
┌─────────────────────────────────────────────────────────────┐
│ INPUTS │
├─────────────────────────────────────────────────────────────┤
│ │
│ API Gateway ──┐ │
│ │ │
│ Liquidator ───┼──► Redis Stream ──► Engine │
│ │ (commands) │
│ Other Svcs ───┘ │
│ │
└─────────────────────────────────────────────────────────────┘
┌─────────────────────────────────────────────────────────────┐
│ OUTPUTS │
├─────────────────────────────────────────────────────────────┤
│ │
│ Engine ──┬──► Redis Stream ──► Event Consumers │
│ │ (events) (persistence, analytics) │
│ │ │
│ ├──► Redis PubSub ──► WebSocket Service │
│ │ (real-time) (client updates) │
│ │ │
│ └──► Redis PubSub ──► Requesting Client │
│ (responses) (API response) │
│ │
└─────────────────────────────────────────────────────────────┘Input: Redis Streams
// Commands arrive via Redis Stream
Stream Key: ENGINE_INPUT_COMMAND_STREAM_KEY
// Consumer configuration
StreamConsumer {
buffer_size: 1024 * 100, // 102,400 messages
xread_count: configurable,
}
// Message format
ReceivedMessage {
message_id: String,
payload: Vec<u8>,
signature: [u8; 64],
}Output: Multiple Channels
| Channel | Type | Purpose | Consumers |
|---|---|---|---|
| Events Stream | Redis Stream | Durable event log | Persistence, Analytics |
| PubSub | Redis PubSub | Real-time updates | WebSocket service |
| Responses | Redis PubSub | Request responses | API Gateway |
| Signed Events | Redis Stream | Verified events | Settlement service |
Output Handler Architecture
pub struct CommandOutputHandler {
redis_client: RedisClient, // Main Redis
event_redis_client: Option<RedisClient>, // Event sink
pubsub_redis_client: Option<RedisClient>,// PubSub sink
signing_key: SigningKey,
}Dual-Lane Event Processing:
| Lane | Concurrency | Delay | Event Types |
|---|---|---|---|
| Fast | Unbounded | None | WebSocket-relevant (fills, depth) |
| Slow | 3 | 100ms | Other events |
Gossip Protocol (Cluster Consensus)
For multi-node deployments, engines use gossip to ensure consistency:
┌─────────────────────────────────────────────────────────────┐
│ Gossip Protocol │
├─────────────────────────────────────────────────────────────┤
│ │
│ Engine A ◄────────► Engine B ◄────────► Engine C │
│ │ │ │ │
│ └───────────────────┴───────────────────┘ │
│ Redis PubSub │
│ (state broadcasts) │
│ │
├─────────────────────────────────────────────────────────────┤
│ │
│ EngineStateMessage { │
│ node_id: String, │
│ sequence: u64, // Command sequence │
│ event_sequence: u64, // Event sequence │
│ last_message_id: String, // Last processed msg │
│ timestamp_us: i64, │
│ } │
│ │
└─────────────────────────────────────────────────────────────┘Consensus Mechanism:
- Each node broadcasts its state periodically
- Nodes track history of other nodes' states
- Compare state hashes at intervals
- Form quorum when
quorum_countnodes agree - Handover phase waits for healthy consensus before accepting traffic
Key files:
/engine/src/processor/gossip/mod.rs- Gossip protocol/engine/src/processor/gossip/core.rs- Consensus implementation/engine/src/processor/output/mod.rs- Output handling/engine/src/processor/output/event_writer.rs- Event batching
6. Multi-Node Architecture & Scaling
Why Multiple Nodes?
Backpack runs multiple engine nodes processing the same commands. This is for fault tolerance, not load balancing:
┌─────────────┐
│ Redis │
│ (commands) │
└──────┬──────┘
│
┌───────────────┼───────────────┐
│ │ │
▼ ▼ ▼
┌─────────┐ ┌─────────┐ ┌─────────┐
│Engine A │ │Engine B │ │Engine C │
│(primary)│ │(standby)│ │(standby)│
└─────────┘ └─────────┘ └─────────┘
│ │ │
└───────All process everything──┘Benefits:
- High availability - Node crashes? Others continue instantly
- Disaster recovery - Data center fails? Other nodes have state
- Verification - Nodes check each other via gossip to catch bugs/corruption
Writes vs Reads: Different Scaling Strategies
| Path | Can Load Balance? | Why |
|---|---|---|
| Writes (orders) | No | All nodes must process all commands for determinism |
| Reads (queries) | Yes | Any node has correct state, distribute queries |
WRITES: Command → ALL nodes process → Redundancy, not speedup
READS: Query → Load balancer → ANY node responds → 3x throughputHow Exchanges Actually Scale Writes
The real scaling strategy is sharding by market:
Engine Cluster 1: BTC_USDC, ETH_USDC ← 3 redundant nodes
Engine Cluster 2: SOL_USDC, AVAX_USDC ← 3 redundant nodes
Engine Cluster 3: All other markets ← 3 redundant nodes
┌────────────────────────────────────────────────────────────┐
│ Order Router │
├────────────────────────────────────────────────────────────┤
│ BTC order? → Cluster 1 │
│ SOL order? → Cluster 2 │
│ DOGE order? → Cluster 3 │
└────────────────────────────────────────────────────────────┘Orders for different markets are independent - BTC orders don't need to wait for SOL orders.
Gossip Protocol: How Nodes Verify Each Other
Nodes continuously broadcast their state to detect drift:
Every few milliseconds:
Engine A: "Processed msg #5000, state hash = abc123"
Engine B: "Processed msg #5000, state hash = abc123"
Engine C: "Processed msg #5000, state hash = abc123"
All match? ✓ Consensus achieved, everything healthy.Gossip message structure:
EngineStateMessage {
node_id: String, // Who am I
sequence: u64, // Commands processed count
event_sequence: u64, // Events emitted count
last_message_id: String, // Last command seen
state_hash: [u8; 32], // Fingerprint of entire state
timestamp_us: i64, // When sent
}State hash = fingerprint of everything (balances, orders, positions). Same hash = identical state.
Quorum:
3 nodes, quorum = 2
2+ nodes agree → Healthy
1 node differs → That node is suspect, route traffic away
All disagree → Major problem, halt and investigateWhy This Architecture Works
1. Commands arrive via Redis (single ordered stream)
2. All nodes read SAME commands in SAME order
3. Deterministic processing (no randomness, no wall-clock time)
4. All nodes end up with IDENTICAL state
5. Gossip verifies this continuously
6. Drifted nodes detected and removed from service7. Determinism & Reproducibility
Why Determinism Matters
The engine must produce identical results given identical inputs. This enables:
- Multi-node consensus (nodes verify each other)
- State recovery via replay
- Audit and debugging
- Regulatory compliance
Determinism Guarantees
| Aspect | Implementation |
|---|---|
| Timestamps | Use message timestamp, not wall clock |
| Order processing | BTreeMap ensures consistent iteration order |
| Randomness | None - no random number generation |
| External data | Index prices are inputs, not fetched during processing |
| Floating point | Use Decimal type for exact arithmetic |
State Verification
// Each node computes state hash
state_hash = hash(engine.state)
// Gossip compares hashes
if node_a.state_hash == node_b.state_hash {
// Consensus achieved
}Checkpoint Validation
// Checkpoints allow validating replay
Checkpoint {
message_id: String, // Last processed message
state_hash: [u8; 32], // Engine state hash
event_sequence: u64, // Event counter
}
// Validation mode
engine --validate-checkpoints
// Replays from snapshot, verifies state matches at each checkpoint8. Engine Initialization
Startup Sequence
1. Parse CLI Arguments
└─► bootstrap | snapshot | validate modes
2. Load Configuration
└─► Environment variables, config files
3. Initialize Tracing
└─► Logging, metrics
4. Health Check Endpoints
└─► Port 8104, 9000
5. Load/Create Signing Key
└─► Ed25519 key for event signing
6. Snapshot/Recovery
├─► Bootstrap: Empty state
└─► Normal: Load latest snapshot
7. Create Engine
└─► Arc<RwLock<Engine>>
8. Initialize Processor
└─► Redis clients, gossip config
9. Start Command Consumer
└─► Redis stream reader
10. Recovery Phase
└─► Replay messages since snapshot
11. Handover Phase
└─► Wait for gossip consensus
12. Ready
└─► Bind port 9001, accept trafficOperating Modes
| Mode | Purpose | When Used |
|---|---|---|
| Normal | Continuous processing | Production |
| Bootstrap | Start with empty state | New deployment |
| Snapshot | Process to message, save state | Backup/migration |
| Validate | Verify checkpoint integrity | Audit/debugging |
Key Configuration
// Key configuration options
write_max_batch_size: 10_000, // Commands per batch
read_max_batch_size: 100, // Queries per batch
consumer_max_xread_count: ..., // Redis read batch
// Redis configuration
redis_client: ..., // Main Redis
redis_event_sink: ..., // Events (optional separate)
redis_pubsub_sink: ..., // PubSub (optional separate)Key files:
/engine/src/bin/main.rs- Entry point (449 lines)/engine/src/processor/mod.rs- Processor initialization
Key File Reference
| Component | Location | Lines |
|---|---|---|
| Entry point | /engine/src/bin/main.rs | 449 |
| Processor | /engine/src/processor/mod.rs | 306 |
| Command loop | /engine/src/processor/command/processor.rs | 321 |
| Validation stage | /engine/src/processor/command/stages/validation.rs | ~100 |
| Execution stage | /engine/src/processor/command/stages/execution.rs | ~200 |
| Settlement stage | /engine/src/processor/command/stages/settlement.rs | ~60 |
| Order book | /engine/src/models/order_book/mod.rs | 2,118 |
| Order page | /engine/src/models/order_book/order_page.rs | 242 |
| Order pair | /engine/src/models/order_book/order_pair.rs | 115 |
| Order commands | /engine/src/engine/order/commands.rs | 1,589 |
| Order helpers | /engine/src/engine/order/helpers.rs | 303 |
| Account model | /engine/src/models/account/mod.rs | ~300 |
| Market types | /engine/src/models/market.rs | ~300 |
| Settlement | /engine/src/clearing_house/settlement/mod.rs | 405 |
| Gossip protocol | /engine/src/processor/gossip/mod.rs | ~100 |
| Gossip core | /engine/src/processor/gossip/core.rs | ~300 |
| Output handler | /engine/src/processor/output/mod.rs | ~200 |
| Event writer | /engine/src/processor/output/event_writer.rs | ~200 |
Quick Reference: Order Book Operations
| Operation | Method | Complexity |
|---|---|---|
| Execute order | OrderBook::execute() | O(m log n) |
| Match orders | match_orders() | O(m × k) |
| Add to book | add_limit_ask/bid() | O(log n) |
| Cancel order | cancel_order() | O(log n) |
| Get best bid/ask | Iterator first element | O(log n) |
| Get depth | Iterate price levels | O(d) |
Where:
- n = number of price levels
- m = number of fills
- k = orders at price level
- d = depth levels requested