feat(metrics): dispatch internal engine state event from queuing/batching tasks

This commit is contained in:
Morgan Funtowicz 2025-02-27 22:43:20 +01:00
parent 1a9c5dec76
commit 712199c769
4 changed files with 117 additions and 23 deletions

View File

@ -11,22 +11,27 @@ use text_generation_router::infer::{
}; };
use text_generation_router::validation::ValidGenerateRequest; use text_generation_router::validation::ValidGenerateRequest;
use text_generation_router::{FinishReason, PrefillToken, Token}; use text_generation_router::{FinishReason, PrefillToken, Token};
use tokio::sync::broadcast::{channel, Receiver, Sender}; use tokio::sync::broadcast::{channel, Receiver as BroadcastReceiver, Sender as BroadcastSender};
use tokio::sync::mpsc::error::SendError; use tokio::sync::mpsc::error::SendError;
use tokio::sync::{mpsc, Notify}; use tokio::sync::{mpsc, Notify, RwLock};
use tokio::time::Instant; use tokio::time::Instant;
use tokio_stream::wrappers::UnboundedReceiverStream; use tokio_stream::wrappers::UnboundedReceiverStream;
use tracing::{info_span, instrument, Instrument, Span}; use tracing::{info_span, instrument, Instrument, Span};
pub struct BackendV3 { pub struct BackendV3 {
/// Events streaming channel /// Internal batching state exposing info for the proxy
events: (Sender<EngineState>, Receiver<EngineState>), state: Arc<RwLock<EngineState>>,
/// Request queue /// Request queue
queue: Queue, queue: Queue,
/// Events streaming channel
state_events: (BroadcastSender<EngineState>, BroadcastReceiver<EngineState>),
/// Notify batcher on queue appends /// Notify batcher on queue appends
batching_task_notifier: Arc<Notify>, batching_task_notifier: Arc<Notify>,
/// Client clone, used for health checks to skip the queue /// Client clone, used for health checks to skip the queue
client: ShardedClient, client: ShardedClient,
} }
@ -48,6 +53,12 @@ impl BackendV3 {
let block_size = shard_info.block_size; let block_size = shard_info.block_size;
let state_events = channel(1);
let state = Arc::new(RwLock::new(EngineState::new(
max_batch_total_tokens,
2 * max_batch_total_tokens,
)));
let queue = Queue::new( let queue = Queue::new(
shard_info.requires_padding, shard_info.requires_padding,
block_size, block_size,
@ -56,6 +67,8 @@ impl BackendV3 {
shard_info.speculate, shard_info.speculate,
max_batch_total_tokens, max_batch_total_tokens,
shard_info.support_chunking, shard_info.support_chunking,
Arc::clone(&state),
state_events.0.clone(),
); );
let batching_task_notifier = Arc::new(Notify::new()); let batching_task_notifier = Arc::new(Notify::new());
@ -69,11 +82,14 @@ impl BackendV3 {
max_batch_size, max_batch_size,
shard_info.support_chunking, shard_info.support_chunking,
queue.clone(), queue.clone(),
state.clone(),
state_events.0.clone(),
batching_task_notifier.clone(), batching_task_notifier.clone(),
)); ));
Self { Self {
events: channel(1), state,
state_events,
queue, queue,
batching_task_notifier, batching_task_notifier,
client, client,
@ -120,8 +136,8 @@ impl Backend for BackendV3 {
.is_ok() .is_ok()
} }
fn events(&self) -> Receiver<EngineState> { fn events(&self) -> BroadcastReceiver<EngineState> {
self.events.0.subscribe() self.state_events.0.subscribe()
} }
fn start_health(&self) -> bool { fn start_health(&self) -> bool {
@ -147,6 +163,8 @@ pub(crate) async fn batching_task(
max_batch_size: Option<usize>, max_batch_size: Option<usize>,
support_chunking: bool, support_chunking: bool,
queue: Queue, queue: Queue,
engine_state: Arc<RwLock<EngineState>>,
batch_events: BroadcastSender<EngineState>,
notifier: Arc<Notify>, notifier: Arc<Notify>,
) { ) {
// Infinite loop // Infinite loop
@ -182,6 +200,22 @@ pub(crate) async fn batching_task(
metrics::gauge!("tgi_batch_current_size").set(batch_size as f64); metrics::gauge!("tgi_batch_current_size").set(batch_size as f64);
metrics::gauge!("tgi_batch_current_max_tokens").set(batch_max_tokens as f64); metrics::gauge!("tgi_batch_current_max_tokens").set(batch_max_tokens as f64);
// Dispatch new state to the proxy
{
// Critical section, doing as little as possible here
{
let mut engine_state = engine_state.write().await;
engine_state.in_flight = batch_max_tokens;
}
// Send new state to the channel for broadcasting
if let Err(err) = batch_events.send(*engine_state.read().await) {
tracing::warn!(
"Failed to send BatchEvent::BatchChanged({batch_max_tokens}): {err}"
)
}
}
let token_budget = max_batch_total_tokens.saturating_sub(batch_max_tokens); let token_budget = max_batch_total_tokens.saturating_sub(batch_max_tokens);
let (min_size, max_size, prefill_token_budget) = if support_chunking { let (min_size, max_size, prefill_token_budget) = if support_chunking {

View File

@ -6,14 +6,17 @@ use crate::client::{
use nohash_hasher::{BuildNoHashHasher, IntMap}; use nohash_hasher::{BuildNoHashHasher, IntMap};
use std::cmp::max; use std::cmp::max;
use std::collections::VecDeque; use std::collections::VecDeque;
use text_generation_router::infer::InferError; use std::sync::Arc;
use text_generation_router::infer::InferStreamResponse; use text_generation_router::infer::InferStreamResponse;
use text_generation_router::infer::{EngineState, InferError};
use text_generation_router::validation::{ use text_generation_router::validation::{
Chunk, ChunksToString, ValidGenerateRequest, ValidGrammar, ValidParameters, Chunk, ChunksToString, ValidGenerateRequest, ValidGrammar, ValidParameters,
ValidStoppingParameters, ValidStoppingParameters,
}; };
use tokio::sync::{mpsc, oneshot}; use tokio::sync::broadcast::Sender as BroadcastSender;
use tokio::sync::{mpsc, oneshot, RwLock};
use tokio::time::Instant; use tokio::time::Instant;
use tracing::log::warn;
use tracing::{info_span, instrument, Instrument, Span}; use tracing::{info_span, instrument, Instrument, Span};
/// Queue entry /// Queue entry
@ -51,6 +54,8 @@ impl Queue {
speculate: u32, speculate: u32,
max_batch_total_tokens: u32, max_batch_total_tokens: u32,
support_chunking: bool, support_chunking: bool,
engine_state: Arc<RwLock<EngineState>>,
queue_events: BroadcastSender<EngineState>,
) -> Self { ) -> Self {
// Create channel // Create channel
let (queue_sender, queue_receiver) = mpsc::unbounded_channel(); let (queue_sender, queue_receiver) = mpsc::unbounded_channel();
@ -64,7 +69,9 @@ impl Queue {
speculate, speculate,
max_batch_total_tokens, max_batch_total_tokens,
support_chunking, support_chunking,
engine_state,
queue_receiver, queue_receiver,
queue_events,
)); ));
Self { queue_sender } Self { queue_sender }
@ -113,7 +120,7 @@ impl Queue {
} }
} }
// Background task responsible of the queue state // Background task responsible for the queue state
#[allow(clippy::too_many_arguments)] #[allow(clippy::too_many_arguments)]
async fn queue_task( async fn queue_task(
requires_padding: bool, requires_padding: bool,
@ -123,7 +130,9 @@ async fn queue_task(
speculate: u32, speculate: u32,
max_batch_total_tokens: u32, max_batch_total_tokens: u32,
support_chunking: bool, support_chunking: bool,
engine_state: Arc<RwLock<EngineState>>,
mut receiver: mpsc::UnboundedReceiver<QueueCommand>, mut receiver: mpsc::UnboundedReceiver<QueueCommand>,
queue_events: BroadcastSender<EngineState>,
) { ) {
let mut state = State::new( let mut state = State::new(
requires_padding, requires_padding,
@ -138,9 +147,29 @@ async fn queue_task(
while let Some(cmd) = receiver.recv().await { while let Some(cmd) = receiver.recv().await {
match cmd { match cmd {
QueueCommand::Append(entry, span) => { QueueCommand::Append(entry, span) => {
metrics::gauge!("tgi_queue_size").increment(1.0); let entry_num_tokens = entry.request.input_length;
metrics::gauge!("tgi_queue_size_tokens").increment(entry.request.input_length);
span.in_scope(|| state.append(*entry)); span.in_scope(|| state.append(*entry));
metrics::gauge!("tgi_queue_size").increment(1.0);
metrics::gauge!("tgi_queue_size_tokens").increment(entry_num_tokens);
// Dispatch new state to the proxy
{
// Lock free operation (read)
let num_queued_tokens = engine_state.read().await.in_queue;
{
// Critical section, doing as little as possible here
let mut engine_state = engine_state.write().await;
engine_state.in_queue = num_queued_tokens + entry_num_tokens;
}
// Send new state to the channel for broadcasting
if let Err(err) = queue_events.send(*engine_state.read().await) {
tracing::warn!(
"Failed to send BatchEvent::QueueChanged({}): {err}",
num_queued_tokens + entry_num_tokens
)
}
}
} }
QueueCommand::NextBatch { QueueCommand::NextBatch {
min_size, min_size,
@ -156,14 +185,32 @@ async fn queue_task(
.await; .await;
response_sender.send(next_batch).unwrap(); response_sender.send(next_batch).unwrap();
metrics::gauge!("tgi_queue_size").set(state.entries.len() as f64); {
metrics::gauge!("tgi_queue_size_tokens").set( let num_batch_tokens = state
state
.entries .entries
.iter() .iter()
.map(|(_, e)| e.request.input_length as f64) .map(|(_, e)| e.request.input_length)
.sum::<f64>(), .sum::<u32>();
); metrics::gauge!("tgi_queue_size").set(state.entries.len() as f64);
metrics::gauge!("tgi_queue_size_tokens").set(num_batch_tokens as f64);
// Dispatch new state to the proxy
{
// Critical section, doing as little as possible here
{
let mut engine_state = engine_state.write().await;
engine_state.in_queue = num_batch_tokens;
}
// Send new state to the channel for broadcasting
if let Err(err) = queue_events.send(*engine_state.read().await) {
tracing::warn!(
"Failed to send BatchEvent::QueueChanged({}): {err}",
num_batch_tokens
)
}
}
}
} }
} }
} }

View File

@ -31,16 +31,29 @@ use tracing::instrument;
#[derive(Debug, Copy, Clone, Serialize)] #[derive(Debug, Copy, Clone, Serialize)]
pub struct EngineState { pub struct EngineState {
/// Number of tokens currently participating in current batch /// Number of tokens currently participating in current batch
in_flight: u32, pub in_flight: u32,
/// Maximum number of tokens which can participate in a batch /// Maximum number of tokens which can participate in a batch
in_flight_max: u32, pub in_flight_max: u32,
/// Number of tokens currently waiting in the queue for future batching /// Number of tokens currently waiting in the queue for future batching
in_queue: u32, pub in_queue: u32,
/// Maximum number of tokens which can wait in the queue for future batching /// Maximum number of tokens which can wait in the queue for future batching
in_queue_max: u32, pub in_queue_max: u32,
}
#[cfg(feature = "engine-state")]
impl EngineState {
#[inline]
pub fn new(in_flight_max: u32, in_queue_max: u32) -> Self {
EngineState {
in_flight: 0,
in_flight_max,
in_queue: 0,
in_queue_max,
}
}
} }
#[async_trait] #[async_trait]

View File

@ -62,7 +62,6 @@ use tokio::select;
use tokio::signal; use tokio::signal;
use tokio::sync::oneshot; use tokio::sync::oneshot;
use tokio::time::Instant; use tokio::time::Instant;
use tokio_stream::wrappers::errors::BroadcastStreamRecvError;
use tokio_stream::wrappers::BroadcastStream; use tokio_stream::wrappers::BroadcastStream;
use tower_http::cors::{AllowOrigin, CorsLayer}; use tower_http::cors::{AllowOrigin, CorsLayer};
use tracing::{info_span, instrument, Instrument}; use tracing::{info_span, instrument, Instrument};
@ -2398,6 +2397,7 @@ async fn start(
.route("/health", get(health)) .route("/health", get(health))
.route("/ping", get(health)) .route("/ping", get(health))
.route("/metrics", get(metrics)) .route("/metrics", get(metrics))
.route("/state", get(state))
.route("/v1/models", get(openai_get_model_info)); .route("/v1/models", get(openai_get_model_info));
let compute_type = let compute_type =