feat: add @tag model override support and remove Qdrant dependencies
- Add model-tags parser for @Tag syntax in chat messages - Support Anthropic models (Sonnet, Haiku, Opus) via @tag - Remove Qdrant vector database from infrastructure and configs - Simplify license model config to use null fallbacks - Add greeting stream after model switch via @tag - Fix protobuf field names to camelCase for v7 compatibility - Add 429 rate limit retry logic with exponential backoff - Remove RAG references from agent harness documentation
This commit is contained in:
@@ -25,7 +25,12 @@ import org.slf4j.LoggerFactory;
|
||||
* - Closed bar (isClosed=true): emitted once when a window boundary is crossed.
|
||||
* Topic: "{ticker}|ohlc:{period}" — consumed by strategies/triggers.
|
||||
*
|
||||
* Accumulator layout (long[7]):
|
||||
* Replay protection: ticks whose trade timestamp predates a period's current window start
|
||||
* are discarded (prevents Kafka replay from contaminating current bars). Open bars are
|
||||
* additionally suppressed until the first live tick (within LIVE_TICK_THRESHOLD_MS of now)
|
||||
* is processed, so Kafka catch-up produces a single bar rather than a flood.
|
||||
*
|
||||
* Accumulator layout (long[8]):
|
||||
* [0] open
|
||||
* [1] high
|
||||
* [2] low
|
||||
@@ -33,13 +38,18 @@ import org.slf4j.LoggerFactory;
|
||||
* [4] volume (sum of base amount)
|
||||
* [5] windowStartMs (epoch ms)
|
||||
* [6] tickCount
|
||||
* [7] valid (1 = seeded or fresh window, 0 = mid-window cold start — open bars suppressed)
|
||||
*/
|
||||
public class RealtimeBarFunction extends RichFlatMapFunction<TickWrapper, RealtimeBar> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(RealtimeBarFunction.class);
|
||||
private static final long serialVersionUID = 1L;
|
||||
// Ticks within this many ms of wall-clock time are considered live (vs. Kafka catch-up).
|
||||
private static final long LIVE_TICK_THRESHOLD_MS = 30_000L;
|
||||
|
||||
private final int[] periods;
|
||||
private transient MapState<Integer, long[]> accumState;
|
||||
// Suppresses open bar emissions during Kafka catch-up; set to true on first live tick.
|
||||
private transient boolean caughtUp = false;
|
||||
|
||||
/**
|
||||
* @param periods Period lengths in seconds (e.g., 60, 300, 900, 3600)
|
||||
@@ -63,6 +73,20 @@ public class RealtimeBarFunction extends RichFlatMapFunction<TickWrapper, Realti
|
||||
if (tick == null) return;
|
||||
|
||||
long nowMs = System.currentTimeMillis();
|
||||
// Seeds use Long.MAX_VALUE so they always pass the per-period timestamp gate below.
|
||||
long tickTimestampMs = tick.isSeed() ? Long.MAX_VALUE : (tick.getTimestamp() / 1_000_000L);
|
||||
|
||||
if (tick.isSeed()) {
|
||||
LOG.info("Seed tick received: ticker={}, seedPeriod={}, seedWindowStart={}, seedHigh={}, nowMs={}",
|
||||
tick.getTicker(), tick.getSeedPeriodSeconds(), tick.getSeedWindowStartMs(),
|
||||
tick.getSeedHigh(), nowMs);
|
||||
}
|
||||
|
||||
// Advance catch-up flag on the first live tick (within threshold of wall-clock time).
|
||||
if (!caughtUp && !tick.isSeed() && (nowMs - tickTimestampMs) < LIVE_TICK_THRESHOLD_MS) {
|
||||
caughtUp = true;
|
||||
LOG.info("Caught up to live data: ticker={}", tick.getTicker());
|
||||
}
|
||||
|
||||
for (int period : periods) {
|
||||
long periodMs = period * 1000L;
|
||||
@@ -70,45 +94,83 @@ public class RealtimeBarFunction extends RichFlatMapFunction<TickWrapper, Realti
|
||||
|
||||
long[] accum = accumState.get(period);
|
||||
|
||||
// Seed ticks pre-populate the accumulator from historical OHLC.
|
||||
// Only apply when the accumulator is absent and the seed targets this period's current window.
|
||||
if (tick.isSeed()) {
|
||||
if (tick.getSeedPeriodSeconds() == period && accum == null
|
||||
&& tick.getSeedWindowStartMs() == windowStart) {
|
||||
long[] seeded = {
|
||||
tick.getPrice(), // open
|
||||
tick.getSeedHigh(), // high
|
||||
tick.getSeedLow(), // low
|
||||
tick.getSeedClose(), // close
|
||||
tick.getAmount(), // volume
|
||||
windowStart,
|
||||
0L, // tickCount (no live ticks yet)
|
||||
1L // valid
|
||||
};
|
||||
accumState.put(period, seeded);
|
||||
LOG.info("Applied seed: ticker={}, period={}s, windowStart={}", tick.getTicker(), period, windowStart);
|
||||
} else if (tick.getSeedPeriodSeconds() == period) {
|
||||
// Period matched but seed was not applied — log the mismatch reason
|
||||
LOG.info("Seed not applied: ticker={}, period={}s, accumNull={}, seedWindow={}, currentWindow={}",
|
||||
tick.getTicker(), period, accum == null, tick.getSeedWindowStartMs(), windowStart);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
// Discard ticks whose trade timestamp predates this period's current window.
|
||||
// Prevents Kafka replay of historical trades from contaminating current bars.
|
||||
if (tickTimestampMs < windowStart) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (accum == null) {
|
||||
// First tick for this period — open a new window and emit the first open bar
|
||||
long[] newAccum = openWindow(tick, windowStart);
|
||||
// First live tick for this period, no seed — open mid-window, suppress open bars
|
||||
long[] newAccum = openWindow(tick, windowStart, false);
|
||||
accumState.put(period, newAccum);
|
||||
out.collect(toBar(tick.getTicker(), period, newAccum, false));
|
||||
LOG.info("Cold-start (no seed): ticker={}, period={}s, valid=0, open bars suppressed", tick.getTicker(), period);
|
||||
|
||||
} else if (accum[5] != windowStart) {
|
||||
// Window boundary crossed — emit the closed bar, then start a fresh window
|
||||
// Window boundary crossed — emit closed bar, then start a fresh valid window
|
||||
if (accum[6] > 0) {
|
||||
out.collect(toBar(tick.getTicker(), period, accum, true));
|
||||
LOG.debug("Emitted closed bar: ticker={}, period={}s, windowStart={}, ticks={}",
|
||||
tick.getTicker(), period, accum[5], accum[6]);
|
||||
}
|
||||
long[] newAccum = openWindow(tick, windowStart);
|
||||
long[] newAccum = openWindow(tick, windowStart, true);
|
||||
accumState.put(period, newAccum);
|
||||
out.collect(toBar(tick.getTicker(), period, newAccum, false));
|
||||
if (caughtUp) {
|
||||
out.collect(toBar(tick.getTicker(), period, newAccum, false));
|
||||
}
|
||||
|
||||
} else {
|
||||
// Same window — update accumulator and emit current open bar
|
||||
// Same window — update accumulator
|
||||
accum[1] = Math.max(accum[1], tick.getPrice()); // high
|
||||
accum[2] = Math.min(accum[2], tick.getPrice()); // low
|
||||
accum[3] = tick.getPrice(); // close
|
||||
accum[4] += tick.getAmount(); // volume
|
||||
accum[6]++; // tick count
|
||||
accumState.put(period, accum);
|
||||
out.collect(toBar(tick.getTicker(), period, accum, false));
|
||||
if (accum[7] == 1 && caughtUp) {
|
||||
out.collect(toBar(tick.getTicker(), period, accum, false));
|
||||
} else if (accum[7] == 0 && caughtUp) {
|
||||
LOG.debug("Open bar suppressed (valid=0, no seed): ticker={}, period={}s", tick.getTicker(), period);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static long[] openWindow(TickWrapper tick, long windowStart) {
|
||||
private static long[] openWindow(TickWrapper tick, long windowStart, boolean valid) {
|
||||
return new long[]{
|
||||
tick.getPrice(), // open
|
||||
tick.getPrice(), // high
|
||||
tick.getPrice(), // low
|
||||
tick.getPrice(), // close
|
||||
tick.getPrice(), // open
|
||||
tick.getPrice(), // high
|
||||
tick.getPrice(), // low
|
||||
tick.getPrice(), // close
|
||||
tick.getAmount(), // volume
|
||||
windowStart,
|
||||
1L // tickCount
|
||||
1L, // tickCount
|
||||
valid ? 1L : 0L // valid flag
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@@ -40,7 +40,7 @@ public class TickDeserializer implements DeserializationSchema<TickWrapper> {
|
||||
|
||||
Tick tick = Tick.parseFrom(payload);
|
||||
|
||||
return new TickWrapper(
|
||||
TickWrapper tw = new TickWrapper(
|
||||
tick.getTicker(),
|
||||
tick.getTradeId(),
|
||||
tick.getTimestamp(),
|
||||
@@ -49,6 +49,15 @@ public class TickDeserializer implements DeserializationSchema<TickWrapper> {
|
||||
tick.getQuoteAmount(),
|
||||
tick.getTakerBuy()
|
||||
);
|
||||
if (tick.hasIsSeed() && tick.getIsSeed()) {
|
||||
tw.setIsSeed(true);
|
||||
tw.setSeedHigh(tick.getSeedHigh());
|
||||
tw.setSeedLow(tick.getSeedLow());
|
||||
tw.setSeedClose(tick.getSeedClose());
|
||||
tw.setSeedWindowStartMs(tick.getSeedWindowStartMs());
|
||||
tw.setSeedPeriodSeconds(tick.getSeedPeriodSeconds());
|
||||
}
|
||||
return tw;
|
||||
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Failed to deserialize Tick, skipping: {}", e.getMessage());
|
||||
|
||||
@@ -20,6 +20,12 @@ public class TickWrapper implements Serializable {
|
||||
/** Quote amount as scaled integer */
|
||||
private long quoteAmount;
|
||||
private boolean takerBuy;
|
||||
private boolean isSeed;
|
||||
private long seedHigh;
|
||||
private long seedLow;
|
||||
private long seedClose;
|
||||
private long seedWindowStartMs;
|
||||
private int seedPeriodSeconds;
|
||||
|
||||
public TickWrapper() {}
|
||||
|
||||
@@ -41,6 +47,12 @@ public class TickWrapper implements Serializable {
|
||||
public long getAmount() { return amount; }
|
||||
public long getQuoteAmount() { return quoteAmount; }
|
||||
public boolean isTakerBuy() { return takerBuy; }
|
||||
public boolean isSeed() { return isSeed; }
|
||||
public long getSeedHigh() { return seedHigh; }
|
||||
public long getSeedLow() { return seedLow; }
|
||||
public long getSeedClose() { return seedClose; }
|
||||
public long getSeedWindowStartMs() { return seedWindowStartMs; }
|
||||
public int getSeedPeriodSeconds() { return seedPeriodSeconds; }
|
||||
|
||||
public void setTicker(String ticker) { this.ticker = ticker; }
|
||||
public void setTradeId(String tradeId) { this.tradeId = tradeId; }
|
||||
@@ -49,6 +61,12 @@ public class TickWrapper implements Serializable {
|
||||
public void setAmount(long amount) { this.amount = amount; }
|
||||
public void setQuoteAmount(long quoteAmount) { this.quoteAmount = quoteAmount; }
|
||||
public void setTakerBuy(boolean takerBuy) { this.takerBuy = takerBuy; }
|
||||
public void setIsSeed(boolean isSeed) { this.isSeed = isSeed; }
|
||||
public void setSeedHigh(long seedHigh) { this.seedHigh = seedHigh; }
|
||||
public void setSeedLow(long seedLow) { this.seedLow = seedLow; }
|
||||
public void setSeedClose(long seedClose) { this.seedClose = seedClose; }
|
||||
public void setSeedWindowStartMs(long seedWindowStartMs) { this.seedWindowStartMs = seedWindowStartMs; }
|
||||
public void setSeedPeriodSeconds(int seedPeriodSeconds) { this.seedPeriodSeconds = seedPeriodSeconds; }
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
|
||||
Reference in New Issue
Block a user