Refactored gRPC interface

Added validation logic
This commit is contained in:
Olivier Dehaene 2022-10-11 16:50:54 +02:00
parent fa9a088467
commit 4c693e6524
13 changed files with 612 additions and 361 deletions

View File

@ -1,6 +1,23 @@
# BLOOM Inference # Text Generation Inference
A Rust and gRPC server for BLOOM Inference. A Rust and gRPC server for text generation inference.
## Load Tests
See `k6/load_test.js`
We send the default examples with a 1 second delay between each request.
Stages:
- Ramp up to 50 concurrent requests per second in 1min
- Ramp up from 50 to 100 concurrent requests per second in 2min
- Ramp down to 0 concurrent requests per second in 1min
| | avg | min | med | max | p(90) | p(95) | RPS |
|------------------------|-----------|-----------|-----------|------------|-----------|-----------|----------|
| Original code | 8.9s | 1s | 9.12s | 16.69s | 13.7s | 14.26s | 5.9 |
| ISO with original code | 8.88s | 959.53ms | 8.89s | 17.08s | 13.34s | 14.12s | 5.94 |
| New batching logic | **5.44s** | **1.27s** | **5.28s** | **13.12s** | **7.78s** | **8.92s** | **9.08** |
## Install ## Install

View File

@ -2,21 +2,35 @@ syntax = "proto3";
package generate.v1; package generate.v1;
service TextGeneration { service TextGenerationService {
/// Service discovery /// Service discovery
rpc ServiceDiscovery(Empty) returns (ServiceDiscoveryResponse) {} rpc ServiceDiscovery (ServiceDiscoveryRequest) returns (ServiceDiscoveryResponse) {}
/// Empties batch cache /// Empties batch cache
rpc ClearCache(Empty) returns (Empty); rpc ClearCache (ClearCacheRequest) returns (ClearCacheResponse);
/// Generate tokens for a batch without cache /// Generate tokens for a batch
rpc Generate(Batch) returns (Response); rpc Generate (GenerateRequest) returns (GenerateResponse);
/// Generate tokens for a batch with cache /// Generate tokens for a list of cached batches
rpc GenerateWithCache(BatchCached) returns (Response); rpc GenerateWithCache (GenerateWithCacheRequest) returns (GenerateWithCacheResponse);
/// Generate tokens until the text of at least one request of the batch is generated
rpc GenerateUntilFinished (GenerateUntilFinishedRequest) returns (GenerateUntilFinishedResponse);
/// Generate tokens until the text of at least one request of the cached batches i finished
rpc GenerateUntilFinishedWithCache (GenerateUntilFinishedWithCacheRequest) returns (GenerateUntilFinishedWithCacheResponse);
} }
/// Empty request
message ServiceDiscoveryRequest {}
message ServiceDiscoveryResponse { message ServiceDiscoveryResponse {
/// Other shards urls
repeated string urls = 1; repeated string urls = 1;
} }
/// Empty request
message ClearCacheRequest {}
/// Empty response
message ClearCacheResponse {}
message LogitsWarperParameters { message LogitsWarperParameters {
float temperature = 1; float temperature = 1;
uint32 top_k = 2; uint32 top_k = 2;
@ -29,10 +43,12 @@ message Request {
uint64 id = 1; uint64 id = 1;
/// The generation context /// The generation context
string inputs = 2; string inputs = 2;
/// The number of tokens inside inputs
uint32 input_length = 3;
/// Logits Warper Parameters /// Logits Warper Parameters
LogitsWarperParameters parameters = 3; LogitsWarperParameters parameters = 4;
/// Stopping criteria /// Stopping criteria
uint32 max_new_tokens = 4; uint32 max_new_tokens = 5;
} }
message Batch { message Batch {
@ -40,44 +56,63 @@ message Batch {
uint64 id = 1; uint64 id = 1;
/// Individual requests /// Individual requests
repeated Request requests = 2; repeated Request requests = 2;
/// Batch size (==len(requests))
uint32 size = 3;
/// Length of the longest sequence within the batch (used for padding)
uint32 max_sequence_length = 4;
} }
message BatchCached { message GeneratedText {
/// Batch ID /// Request
uint64 id = 1; Request request = 1;
/// Request ids within cache
repeated uint64 request_ids = 2;
/// Cache IDs
repeated uint64 batch_cached_ids = 3;
/// Batch size (sum of all batch sizes)
uint32 total_batch_size = 4;
/// Max sequence length
uint32 max_sequence_length = 5;
}
message FinishedGeneration {
/// ID of the original request
uint64 id = 1;
/// Output /// Output
string output = 2; string output = 2;
} }
message CacheEntry { message GenerateRequest {
/// Cache ID; same as batch ID /// Batch
uint64 id = 1; Batch batch = 1;
/// Requests present in cache entry
repeated uint64 request_ids = 2;
/// Sequence length
uint32 sequence_length = 3;
} }
message Response { message GenerateResponse {
/// Finished requests (optional) /// Finished requests
repeated FinishedGeneration finished = 1; repeated GeneratedText generated_texts = 1;
/// Cache entry (optional) /// Next batch (cached)
optional CacheEntry cache_entry = 2; optional Batch batch = 2;
} }
message GenerateWithCacheRequest {
/// Cached batches
repeated Batch batches = 1;
}
// Represent an empty message. message GenerateWithCacheResponse {
message Empty {} /// Finished requests
repeated GeneratedText generated_texts = 1;
/// Next batch (cached)
optional Batch batch = 2;
}
message GenerateUntilFinishedRequest {
/// Batch
Batch batch = 1;
}
message GenerateUntilFinishedResponse {
/// Finished requests
repeated GeneratedText generated_texts = 1;
/// Next batch (cached)
optional Batch batch = 2;
}
message GenerateUntilFinishedWithCacheRequest {
/// Cached batches
repeated Batch batches = 1;
}
message GenerateUntilFinishedWithCacheResponse {
/// Finished requests
repeated GeneratedText generated_texts = 1;
/// Next batch (cached)
optional Batch batch = 2;
}

View File

@ -1,4 +1,4 @@
use crate::pb::generate::v1::text_generation_client::TextGenerationClient; use crate::pb::generate::v1::text_generation_service_client::TextGenerationServiceClient;
use crate::pb::generate::v1::*; use crate::pb::generate::v1::*;
use crate::Result; use crate::Result;
use std::time::Duration; use std::time::Duration;
@ -9,7 +9,7 @@ use tracing::*;
/// BLOOM Inference gRPC client /// BLOOM Inference gRPC client
#[derive(Clone)] #[derive(Clone)]
pub struct Client { pub struct Client {
stub: TextGenerationClient<Timeout<Channel>>, stub: TextGenerationServiceClient<Timeout<Channel>>,
} }
impl Client { impl Client {
@ -22,13 +22,13 @@ impl Client {
let timeout_channel = Timeout::new(channel, timeout); let timeout_channel = Timeout::new(channel, timeout);
Self { Self {
stub: TextGenerationClient::new(timeout_channel), stub: TextGenerationServiceClient::new(timeout_channel),
} }
} }
/// Returns a client connected to the given unix socket. Requests exceeding timeout will fail. /// Returns a client connected to the given unix socket. Requests exceeding timeout will fail.
pub async fn connect_uds(path: String, timeout: Duration) -> Self { pub async fn connect_uds(path: String, timeout: Duration) -> Self {
let channel = Channel::from_shared(format!("http://[::]:50051")) let channel = Channel::from_shared("http://[::]:50051".to_string())
.unwrap() .unwrap()
.connect_with_connector(tower::service_fn(move |_: Uri| { .connect_with_connector(tower::service_fn(move |_: Uri| {
tokio::net::UnixStream::connect(path.clone()) tokio::net::UnixStream::connect(path.clone())
@ -38,13 +38,13 @@ impl Client {
let timeout_channel = Timeout::new(channel, timeout); let timeout_channel = Timeout::new(channel, timeout);
Self { Self {
stub: TextGenerationClient::new(timeout_channel), stub: TextGenerationServiceClient::new(timeout_channel),
} }
} }
#[instrument(skip(self))] #[instrument(skip(self))]
pub async fn service_discovery(&mut self) -> Result<Vec<String>> { pub async fn service_discovery(&mut self) -> Result<Vec<String>> {
let request = tonic::Request::new(Empty {}); let request = tonic::Request::new(ServiceDiscoveryRequest {});
let response = self let response = self
.stub .stub
.service_discovery(request) .service_discovery(request)
@ -64,7 +64,7 @@ impl Client {
#[instrument(skip(self))] #[instrument(skip(self))]
pub async fn clear_cache(&mut self) -> Result<()> { pub async fn clear_cache(&mut self) -> Result<()> {
let request = tonic::Request::new(Empty {}); let request = tonic::Request::new(ClearCacheRequest {});
self.stub self.stub
.clear_cache(request) .clear_cache(request)
.instrument(info_span!("clear_cache")) .instrument(info_span!("clear_cache"))
@ -73,32 +73,59 @@ impl Client {
} }
#[instrument(skip(self))] #[instrument(skip(self))]
pub async fn generate( pub async fn generate(&mut self, batch: Batch) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
&mut self, let request = tonic::Request::new(GenerateRequest { batch: Some(batch) });
request: Batch,
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> {
let request = tonic::Request::new(request);
let response = self let response = self
.stub .stub
.generate(request) .generate(request)
.instrument(info_span!("generate")) .instrument(info_span!("generate"))
.await? .await?
.into_inner(); .into_inner();
Ok((response.finished, response.cache_entry)) Ok((response.generated_texts, response.batch))
} }
#[instrument(skip(self))] #[instrument(skip(self))]
pub async fn generate_with_cache( pub async fn generate_with_cache(
&mut self, &mut self,
request: BatchCached, batches: Vec<Batch>,
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> { ) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
let request = tonic::Request::new(request); let request = tonic::Request::new(GenerateWithCacheRequest { batches });
let response = self let response = self
.stub .stub
.generate_with_cache(request) .generate_with_cache(request)
.instrument(info_span!("generate_with_cache")) .instrument(info_span!("generate_with_cache"))
.await? .await?
.into_inner(); .into_inner();
Ok((response.finished, response.cache_entry)) Ok((response.generated_texts, response.batch))
}
#[instrument(skip(self))]
pub async fn generate_until_finished(
&mut self,
batch: Batch,
) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
let request = tonic::Request::new(GenerateUntilFinishedRequest { batch: Some(batch) });
let response = self
.stub
.generate_until_finished(request)
.instrument(info_span!("generate_until_finished"))
.await?
.into_inner();
Ok((response.generated_texts, response.batch))
}
#[instrument(skip(self))]
pub async fn generate_until_finished_with_cache(
&mut self,
batches: Vec<Batch>,
) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
let request = tonic::Request::new(GenerateUntilFinishedWithCacheRequest { batches });
let response = self
.stub
.generate_until_finished_with_cache(request)
.instrument(info_span!("generate_until_finished_with_cache"))
.await?
.into_inner();
Ok((response.generated_texts, response.batch))
} }
} }

View File

@ -5,9 +5,7 @@ mod pb;
mod sharded_client; mod sharded_client;
pub use client::Client; pub use client::Client;
pub use pb::generate::v1::{ pub use pb::generate::v1::{Batch, GeneratedText, LogitsWarperParameters, Request};
Batch, BatchCached, CacheEntry, FinishedGeneration, LogitsWarperParameters, Request,
};
pub use sharded_client::ShardedClient; pub use sharded_client::ShardedClient;
use thiserror::Error; use thiserror::Error;
pub use tonic::transport::Uri; pub use tonic::transport::Uri;

View File

@ -1,5 +1,5 @@
use crate::Result; use crate::Result;
use crate::{Batch, BatchCached, CacheEntry, Client, FinishedGeneration}; use crate::{Batch, Client, GeneratedText};
use futures::future::join_all; use futures::future::join_all;
use std::time::Duration; use std::time::Duration;
use tokio::sync::{broadcast, mpsc}; use tokio::sync::{broadcast, mpsc};
@ -9,11 +9,19 @@ use tonic::transport::Uri;
enum Command { enum Command {
Generate( Generate(
Batch, Batch,
mpsc::Sender<Result<(Vec<FinishedGeneration>, Option<CacheEntry>)>>, mpsc::Sender<Result<(Vec<GeneratedText>, Option<Batch>)>>,
), ),
GenerateWithCache( GenerateWithCache(
BatchCached, Vec<Batch>,
mpsc::Sender<Result<(Vec<FinishedGeneration>, Option<CacheEntry>)>>, mpsc::Sender<Result<(Vec<GeneratedText>, Option<Batch>)>>,
),
GenerateUntilFinished(
Batch,
mpsc::Sender<Result<(Vec<GeneratedText>, Option<Batch>)>>,
),
GenerateUntilFinishedWithCache(
Vec<Batch>,
mpsc::Sender<Result<(Vec<GeneratedText>, Option<Batch>)>>,
), ),
ClearCache(mpsc::Sender<Result<()>>), ClearCache(mpsc::Sender<Result<()>>),
} }
@ -25,8 +33,16 @@ async fn client_task(mut client: Client, mut request_subscriber: broadcast::Rece
let result = client.generate(batch).await; let result = client.generate(batch).await;
response_tx.try_send(result).unwrap_or(()); response_tx.try_send(result).unwrap_or(());
} }
Command::GenerateWithCache(batch_cached, response_tx) => { Command::GenerateWithCache(batches, response_tx) => {
let result = client.generate_with_cache(batch_cached).await; let result = client.generate_with_cache(batches).await;
response_tx.try_send(result).unwrap_or(());
}
Command::GenerateUntilFinished(batch, response_tx) => {
let result = client.generate_until_finished(batch).await;
response_tx.try_send(result).unwrap_or(());
}
Command::GenerateUntilFinishedWithCache(batches, response_tx) => {
let result = client.generate_until_finished_with_cache(batches).await;
response_tx.try_send(result).unwrap_or(()); response_tx.try_send(result).unwrap_or(());
} }
Command::ClearCache(response_tx) => { Command::ClearCache(response_tx) => {
@ -74,10 +90,7 @@ impl ShardedClient {
Self::from_master_client(master_client).await Self::from_master_client(master_client).await
} }
pub async fn generate( pub async fn generate(&self, batch: Batch) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
&self,
batch: Batch,
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> {
let (response_tx, mut response_rx) = mpsc::channel(1); let (response_tx, mut response_rx) = mpsc::channel(1);
self.request_tx self.request_tx
.send(Command::Generate(batch, response_tx)) .send(Command::Generate(batch, response_tx))
@ -87,11 +100,36 @@ impl ShardedClient {
pub async fn generate_with_cache( pub async fn generate_with_cache(
&self, &self,
batch_cached: BatchCached, batches: Vec<Batch>,
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> { ) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
let (response_tx, mut response_rx) = mpsc::channel(1); let (response_tx, mut response_rx) = mpsc::channel(1);
self.request_tx self.request_tx
.send(Command::GenerateWithCache(batch_cached, response_tx)) .send(Command::GenerateWithCache(batches, response_tx))
.unwrap();
response_rx.recv().await.unwrap()
}
pub async fn generate_until_finished(
&self,
batch: Batch,
) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
let (response_tx, mut response_rx) = mpsc::channel(1);
self.request_tx
.send(Command::GenerateUntilFinished(batch, response_tx))
.unwrap();
response_rx.recv().await.unwrap()
}
pub async fn generate_until_finished_with_cache(
&self,
batches: Vec<Batch>,
) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
let (response_tx, mut response_rx) = mpsc::channel(1);
self.request_tx
.send(Command::GenerateUntilFinishedWithCache(
batches,
response_tx,
))
.unwrap(); .unwrap();
response_rx.recv().await.unwrap() response_rx.recv().await.unwrap()
} }

View File

@ -1,10 +1,9 @@
use crate::Db;
use bloom_inference_client::{
Batch, BatchCached, CacheEntry, ClientError, FinishedGeneration, ShardedClient,
};
use std::sync::Arc;
use tokio::sync::{Notify, oneshot};
use crate::server::GenerateRequest; use crate::server::GenerateRequest;
use crate::Db;
use bloom_inference_client::{Batch, ClientError, GeneratedText, ShardedClient};
use std::future::Future;
use std::sync::Arc;
use tokio::sync::{oneshot, Notify};
const MAX_LENGTH: usize = 128; const MAX_LENGTH: usize = 128;
@ -32,12 +31,16 @@ impl Batcher {
Self { db, shared } Self { db, shared }
} }
pub(crate) async fn infer(&self, request: GenerateRequest) -> Result<String, InferError> { pub(crate) async fn infer(
&self,
input_length: usize,
request: GenerateRequest,
) -> Result<String, InferError> {
if self.db.len() > MAX_LENGTH { if self.db.len() > MAX_LENGTH {
return Err(InferError {}); return Err(InferError {});
} }
let (request_tx, request_rx) = oneshot::channel(); let (request_tx, request_rx) = oneshot::channel();
self.db.append(request, request_tx); self.db.append(input_length, request, request_tx);
self.shared.batching_task.notify_waiters(); self.shared.batching_task.notify_waiters();
match request_rx.await.unwrap() { match request_rx.await.unwrap() {
Ok(output) => Ok(output), Ok(output) => Ok(output),
@ -51,76 +54,57 @@ async fn batching_task(client: ShardedClient, db: Db, shared: Arc<Shared>) {
shared.batching_task.notified().await; shared.batching_task.notified().await;
if let Some(batch) = db.next_batch(32) { if let Some(batch) = db.next_batch(32) {
let mut cache_entry = infer_batch(batch, &client, &db).await; let request_ids = batch.requests.iter().map(|req| req.id).collect();
let mut cached_batch = match batch.size {
loop { size if size > 16 => {
if let Some(entry) = cache_entry { wrap_future(client.generate_until_finished(batch), request_ids, &db).await
let mut batch_cached_ids = vec![entry.id];
let mut total_batch_size = entry.request_ids.len();
let mut max_sequence_length = entry.sequence_length;
let mut request_ids = entry.request_ids;
// if total_batch_size <= 16 {
// if let Some(batch) = db.next_batch_minimum_size(16, 48) {
// let other_cache_entry = infer_batch(batch, &client, &db).await;
//
// if let Some(entry) = other_cache_entry {
// batch_cached_ids.push(entry.id);
// total_batch_size += entry.request_ids.len();
// max_sequence_length =
// max_sequence_length.max(entry.sequence_length);
// request_ids.extend(entry.request_ids.into_iter());
// }
// }
// }
let batch_cached = BatchCached {
id: entry.id,
batch_cached_ids,
total_batch_size: total_batch_size as u32,
max_sequence_length,
request_ids,
};
cache_entry = infer_batch_cached(batch_cached, &client, &db).await;
} else {
break;
} }
_ => wrap_future(client.generate(batch), request_ids, &db).await,
};
while let Some(batch) = cached_batch {
let batch_size = batch.size;
let mut request_ids: Vec<u64> = batch.requests.iter().map(|req| req.id).collect();
let mut batches = vec![batch];
if batch_size <= 16 {
if let Some(new_batch) = db.next_batch_minimum_size(16, 48) {
let new_batch_request_ids =
new_batch.requests.iter().map(|req| req.id).collect();
let new_cached_batch =
wrap_future(client.generate(new_batch), new_batch_request_ids, &db)
.await;
if let Some(new_cached_batch) = new_cached_batch {
request_ids.extend(new_cached_batch.requests.iter().map(|req| req.id));
batches.push(new_cached_batch);
}
}
}
cached_batch = match batch_size {
size if size > 16 => {
wrap_future(client.generate_until_finished_with_cache(batches), request_ids, &db).await
}
_ => wrap_future(client.generate_with_cache(batches), request_ids, &db).await,
};
} }
} }
} }
} }
async fn infer_batch_cached( async fn wrap_future(
batch: BatchCached, future: impl Future<Output = Result<(Vec<GeneratedText>, Option<Batch>), ClientError>>,
client: &ShardedClient, request_ids: Vec<u64>,
db: &Db, db: &Db,
) -> Option<CacheEntry> { ) -> Option<Batch> {
match client.generate_with_cache(batch.clone()).await { match future.await {
Ok((finished, cache_entry)) => { Ok((generated_texts, next_batch)) => {
send_finished(finished, db); send_generated(generated_texts, db);
cache_entry next_batch
} }
Err(err) => { Err(err) => {
println!("{:?}", err); println!("{:?}", err);
send_error(err, batch.request_ids, &db); send_error(err, request_ids, db);
None
}
}
}
async fn infer_batch(batch: Batch, client: &ShardedClient, db: &Db) -> Option<CacheEntry> {
match client.generate(batch.clone()).await {
Ok((finished, cache_entry)) => {
send_finished(finished, db);
cache_entry
}
Err(err) => {
println!("{:?}", err);
send_error(
err,
batch.requests.into_iter().map(|req| req.id).collect(),
&db,
);
None None
} }
} }
@ -133,9 +117,9 @@ fn send_error(error: ClientError, request_ids: Vec<u64>, db: &Db) {
}); });
} }
fn send_finished(finished: Vec<FinishedGeneration>, db: &Db) { fn send_generated(finished: Vec<GeneratedText>, db: &Db) {
finished.into_iter().for_each(|output| { finished.into_iter().for_each(|output| {
let (_, response_tx) = db.remove(&output.id).unwrap(); let (_, response_tx) = db.remove(&output.request.unwrap().id).unwrap();
response_tx.send(Ok(output.output)).unwrap_or(()); response_tx.send(Ok(output.output)).unwrap_or(());
}); });
} }

View File

@ -46,6 +46,7 @@ impl Db {
pub(crate) fn append( pub(crate) fn append(
&self, &self,
input_length: usize,
request: GenerateRequest, request: GenerateRequest,
sender: Sender<Result<String, ClientError>>, sender: Sender<Result<String, ClientError>>,
) { ) {
@ -63,6 +64,7 @@ impl Db {
let request = Request { let request = Request {
id, id,
inputs: request.inputs, inputs: request.inputs,
input_length: input_length as u32,
parameters, parameters,
max_new_tokens: request.parameters.max_new_tokens, max_new_tokens: request.parameters.max_new_tokens,
}; };
@ -103,9 +105,13 @@ impl Db {
pub(crate) fn next_batch(&self, max_size: usize) -> Option<Batch> { pub(crate) fn next_batch(&self, max_size: usize) -> Option<Batch> {
if let Some((last_id, requests)) = self.next_requests(max_size) { if let Some((last_id, requests)) = self.next_requests(max_size) {
let mut state = self.shared.state.write(); let mut state = self.shared.state.write();
let size = requests.len();
let max_sequence_length = requests.iter().map(|r| r.input_length).max().unwrap();
let batch = Batch { let batch = Batch {
id: state.next_batch_id, id: state.next_batch_id,
requests, requests,
size: size as u32,
max_sequence_length,
}; };
state.next_batch_start_id = last_id + 1; state.next_batch_start_id = last_id + 1;
state.next_batch_id += 1; state.next_batch_id += 1;
@ -122,9 +128,13 @@ impl Db {
if let Some((last_id, requests)) = self.next_requests(max_size) { if let Some((last_id, requests)) = self.next_requests(max_size) {
if requests.len() >= min_size { if requests.len() >= min_size {
let mut state = self.shared.state.write(); let mut state = self.shared.state.write();
let size = requests.len();
let max_sequence_length = requests.iter().map(|r| r.input_length).max().unwrap();
let batch = Batch { let batch = Batch {
id: state.next_batch_id, id: state.next_batch_id,
requests, requests,
size: size as u32,
max_sequence_length,
}; };
state.next_batch_start_id = last_id + 1; state.next_batch_start_id = last_id + 1;
state.next_batch_id += 1; state.next_batch_id += 1;

View File

@ -1,31 +1,45 @@
use bloom_inference_client::ShardedClient; use bloom_inference_client::ShardedClient;
use poem;
use poem::listener::TcpListener; use poem::listener::TcpListener;
use std::time::Duration; use std::time::Duration;
use tokenizers::Tokenizer;
mod server; mod server;
mod validation;
use validation::Validation;
mod db; mod db;
use db::Db; use db::Db;
mod batcher; mod batcher;
use batcher::Batcher; use batcher::Batcher;
#[tokio::main] fn main() -> Result<(), std::io::Error> {
async fn main() -> Result<(), std::io::Error> { let tokenizer = Tokenizer::from_pretrained("bigscience/bloom", None).unwrap();
tracing_subscriber::fmt::init();
let sharded_client = tokio::runtime::Builder::new_multi_thread()
ShardedClient::connect_uds("/tmp/bloom-inference-0".to_string(), Duration::from_secs(5)) .enable_all()
.build()
.unwrap()
.block_on(async {
tracing_subscriber::fmt::init();
let sharded_client = ShardedClient::connect_uds(
"/tmp/bloom-inference-0".to_string(),
Duration::from_secs(5),
)
.await; .await;
sharded_client sharded_client
.clear_cache() .clear_cache()
.await .await
.expect("Unable to clear cache"); .expect("Unable to clear cache");
tracing::info!("Connected"); tracing::info!("Connected");
let addr = "127.0.0.1:3000".to_string(); let addr = "127.0.0.1:3000".to_string();
let listener = TcpListener::bind(addr); let listener = TcpListener::bind(addr);
server::run(sharded_client, listener).await server::run(sharded_client, tokenizer, listener).await
})
} }

View File

@ -1,12 +1,13 @@
use poem::{EndpointExt, handler, post, Route, Server}; use crate::{Batcher, ShardedClient, Validation};
use poem::http::StatusCode; use poem::http::StatusCode;
use poem::listener::TcpListener; use poem::listener::TcpListener;
use poem::middleware::AddData; use poem::middleware::AddData;
use poem::web::{Data, Json}; use poem::web::{Data, Json};
use tokio::time::Instant; use poem::{handler, post, EndpointExt, Route, Server};
use crate::{Batcher, ShardedClient};
use tracing::instrument;
use serde::Deserialize; use serde::Deserialize;
use tokenizers::Tokenizer;
use tokio::time::Instant;
use tracing::instrument;
#[derive(Clone, Debug, Deserialize)] #[derive(Clone, Debug, Deserialize)]
pub(crate) struct GenerateParameters { pub(crate) struct GenerateParameters {
@ -59,21 +60,24 @@ pub(crate) struct GenerateRequest {
pub parameters: GenerateParameters, pub parameters: GenerateParameters,
} }
#[handler] #[handler]
#[instrument(skip(infer), fields(time, time_per_token))] #[instrument(skip(validation, infer), fields(time, time_per_token))]
async fn generate( async fn generate(
validation: Data<&Validation>,
infer: Data<&Batcher>, infer: Data<&Batcher>,
req: Json<GenerateRequest>, req: Json<GenerateRequest>,
) -> poem::Result<Json<serde_json::Value>> { ) -> poem::Result<Json<serde_json::Value>> {
let start = Instant::now(); let start = Instant::now();
let output = infer let (input_length, validated_request) = validation
.infer(GenerateRequest { .validate(GenerateRequest {
inputs: req.inputs.clone(), inputs: req.inputs.clone(),
parameters: req.parameters.clone(), parameters: req.parameters.clone(),
}) })
.await; .await
.unwrap();
let output = infer.infer(input_length, validated_request).await;
match output { match output {
Ok(generated_text) => { Ok(generated_text) => {
@ -92,20 +96,22 @@ async fn generate(
} }
} }
pub async fn run(client: ShardedClient, listener: TcpListener<String>) -> Result<(), std::io::Error> { pub async fn run(
client client: ShardedClient,
.clear_cache() tokenizer: Tokenizer,
.await listener: TcpListener<String>,
.expect("Unable to clear cache"); ) -> Result<(), std::io::Error> {
client.clear_cache().await.expect("Unable to clear cache");
tracing::info!("Connected"); tracing::info!("Connected");
let infer = Batcher::new(client); let infer = Batcher::new(client);
let validation = Validation::new(tokenizer);
let app = Route::new() let app = Route::new()
.at("/generate", post(generate)) .at("/generate", post(generate))
.with(AddData::new(validation))
.with(AddData::new(infer)); .with(AddData::new(infer));
Server::new(listener) Server::new(listener).run(app).await
.run(app) }
.await
}

65
router/src/validation.rs Normal file
View File

@ -0,0 +1,65 @@
use crate::server::GenerateRequest;
use tokenizers::tokenizer::Tokenizer;
use tokio::sync::{mpsc, oneshot};
#[derive(Debug)]
pub struct ValidationError {}
type ValidationRequest = (
GenerateRequest,
oneshot::Sender<Result<(usize, GenerateRequest), ValidationError>>,
);
#[derive(Debug, Clone)]
pub(crate) struct Validation {
sender: mpsc::Sender<ValidationRequest>,
}
impl Validation {
pub(crate) fn new(tokenizer: Tokenizer) -> Self {
let (validation_sender, validation_receiver) = mpsc::channel(128);
tokio::spawn(validation_task(tokenizer, validation_receiver));
Self {
sender: validation_sender,
}
}
pub(crate) async fn validate(
&self,
request: GenerateRequest,
) -> Result<(usize, GenerateRequest), ValidationError> {
let (sender, receiver) = oneshot::channel();
self.sender.send((request, sender)).await.unwrap();
receiver.await.unwrap()
}
}
async fn validation_task(tokenizer: Tokenizer, mut receiver: mpsc::Receiver<ValidationRequest>) {
while let Some((request, response_tx)) = receiver.recv().await {
if request.parameters.temperature < 0.0 {
response_tx.send(Err(ValidationError {})).unwrap_or(());
continue;
}
if request.parameters.top_p <= 0.0 || request.parameters.top_p > 1.0 {
response_tx.send(Err(ValidationError {})).unwrap_or(());
continue;
}
if request.parameters.max_new_tokens > 512 {
response_tx.send(Err(ValidationError {})).unwrap_or(());
continue;
}
let inputs = tokenizer.encode(request.inputs.clone(), false).unwrap();
let input_length = inputs.len();
if input_length > 512 {
response_tx.send(Err(ValidationError {})).unwrap_or(());
continue;
}
response_tx.send(Ok((input_length, request))).unwrap_or(());
}
println!("drop here");
}

View File

@ -1,44 +1,19 @@
import torch from bloom_inference.model import Batch
from typing import Dict, Optional
from dataclasses import dataclass
from typing import Dict, Optional, List
from bloom_inference.pb import generate_pb2
from bloom_inference.utils import NextTokenChooser, StoppingCriteria
@dataclass
class CacheEntry:
batch_id: int
request_ids: List[int]
input_ids: Dict[str, torch.Tensor]
all_input_ids: List[torch.Tensor]
next_token_choosers: List[NextTokenChooser]
stopping_criterias: List[StoppingCriteria]
def __len__(self):
return len(self.request_ids)
def to_pb(self):
return generate_pb2.CacheEntry(
id=self.batch_id,
request_ids=self.request_ids,
sequence_length=max(len(entry) for entry in self.all_input_ids),
)
class Cache: class Cache:
def __init__(self): def __init__(self):
self.cache: Dict[str, CacheEntry] = {} self.cache: Dict[int, Batch] = {}
def pop(self, batch_id: str) -> Optional[CacheEntry]: def pop(self, batch_id: int) -> Optional[Batch]:
return self.cache.pop(batch_id, None) return self.cache.pop(batch_id, None)
def set(self, entry: CacheEntry): def set(self, entry: Batch):
if entry is not None: if entry is not None:
self.cache[entry.batch_id] = entry self.cache[entry.batch_id] = entry
def delete(self, batch_id: str): def delete(self, batch_id: int):
del self.cache[batch_id] del self.cache[batch_id]
def clear(self): def clear(self):

View File

@ -8,7 +8,6 @@ from typing import List, Tuple, Optional, Dict
from transformers import AutoTokenizer, AutoModelForCausalLM, AutoConfig from transformers import AutoTokenizer, AutoModelForCausalLM, AutoConfig
from transformers.modeling_utils import no_init_weights from transformers.modeling_utils import no_init_weights
from bloom_inference.cache import CacheEntry
from bloom_inference.pb import generate_pb2 from bloom_inference.pb import generate_pb2
from bloom_inference.shard_model import shard_model, match_suffix from bloom_inference.shard_model import shard_model, match_suffix
from bloom_inference.utils import ( from bloom_inference.utils import (
@ -24,25 +23,35 @@ torch.manual_seed(0)
@dataclass @dataclass
class Batch: class Batch:
batch_id: int batch_id: int
request_ids: List[int] requests: List[generate_pb2.Request]
input_ids: Dict[str, torch.Tensor] input_ids: Dict[str, torch.Tensor]
all_input_ids: List[torch.Tensor] all_input_ids: List[torch.Tensor]
next_token_choosers: List[NextTokenChooser] next_token_choosers: List[NextTokenChooser]
stopping_criterias: List[StoppingCriteria] stopping_criterias: List[StoppingCriteria]
size: int
max_sequence_length: int
def to_pb(self):
return generate_pb2.Batch(
id=self.batch_id,
requests=self.requests,
size=self.size,
max_sequence_length=self.max_sequence_length,
)
@classmethod @classmethod
def from_batch_pb( def from_pb(
cls, pb: generate_pb2.Batch, tokenizer: AutoTokenizer, device: torch.device cls, pb: generate_pb2.Batch, tokenizer: AutoTokenizer, device: torch.device
) -> "Batch": ) -> "Batch":
request_ids = []
inputs = [] inputs = []
next_token_choosers = [] next_token_choosers = []
stopping_criterias = [] stopping_criterias = []
input_lengths = []
# Parse batch # Parse batch
for r in pb.requests: for r in pb.requests:
request_ids.append(r.id)
inputs.append(r.inputs) inputs.append(r.inputs)
input_lengths.append(r.input_length)
next_token_choosers.append( next_token_choosers.append(
NextTokenChooser( NextTokenChooser(
temperature=r.parameters.temperature, temperature=r.parameters.temperature,
@ -54,94 +63,93 @@ class Batch:
stopping_criterias.append(StoppingCriteria(max_new_tokens=r.max_new_tokens)) stopping_criterias.append(StoppingCriteria(max_new_tokens=r.max_new_tokens))
input_ids = tokenizer(inputs, return_tensors="pt", padding=True).to(device) input_ids = tokenizer(inputs, return_tensors="pt", padding=True).to(device)
all_input_ids = input_ids["input_ids"].unsqueeze(-1) # Remove padding from all_input_ids
all_input_ids = [
input_ids.squeeze(0)[-length:].unsqueeze(-1)
for length, input_ids in zip(
input_lengths, input_ids["input_ids"].split(1, dim=0)
)
]
return cls( return cls(
pb.id, batch_id=pb.id,
request_ids, requests=pb.requests,
input_ids, input_ids=input_ids,
all_input_ids, all_input_ids=all_input_ids,
next_token_choosers, next_token_choosers=next_token_choosers,
stopping_criterias, stopping_criterias=stopping_criterias,
size=pb.size,
max_sequence_length=pb.max_sequence_length,
) )
@classmethod @classmethod
def from_cache_entry(cls, cache_entry: CacheEntry) -> "Batch": def concatenate(cls, batches: List["Batch"]) -> "Batch":
return cls( # Used for padding
cache_entry.batch_id, total_batch_size = sum(batch.size for batch in batches)
cache_entry.request_ids, max_sequence_length = max(batch.max_sequence_length for batch in batches)
cache_entry.input_ids,
cache_entry.all_input_ids,
cache_entry.next_token_choosers,
cache_entry.stopping_criterias,
)
@classmethod # Batch attributes
def from_batch_cached_pb(cls, pb: generate_pb2.BatchCached, cache) -> "Batch":
if len(pb.batch_cached_ids) == 1:
cache_entry = cache.pop(pb.batch_cached_ids[0])
if cache_entry is None:
raise ValueError(f"Batch ID {pb.batch_id} not found in cache")
return cls.from_cache_entry(cache_entry)
total_batch_size = pb.total_batch_size
max_sequence_length = pb.max_sequence_length
input_ids = {"input_ids": None, "attention_mask": None, "past_key_values": []} input_ids = {"input_ids": None, "attention_mask": None, "past_key_values": []}
request_ids = [] requests = []
all_input_ids = [] all_input_ids = []
next_token_choosers = [] next_token_choosers = []
stopping_criterias = [] stopping_criterias = []
# Used for slicing correctly inside the tensors
# Equivalent to a cumsum on batch sizes
start_index = 0 start_index = 0
for i, batch_id in enumerate(pb.batch_cached_ids): for i, batch in enumerate(batches):
cache_entry = cache.pop(batch_id) requests.extend(batch.requests)
if cache_entry is None: all_input_ids.extend(batch.all_input_ids)
raise ValueError(f"Batch ID {batch_id} not found in cache") next_token_choosers.extend(batch.next_token_choosers)
request_ids.extend(cache_entry.request_ids) stopping_criterias.extend(batch.stopping_criterias)
all_input_ids.extend(cache_entry.all_input_ids)
next_token_choosers.extend(cache_entry.next_token_choosers)
stopping_criterias.extend(cache_entry.stopping_criterias)
batch_size = len(cache_entry.request_ids) # Slicing end index for this batch
end_index = start_index + batch_size end_index = start_index + batch.size
sequence_length = max(len(entry) for entry in cache_entry.all_input_ids)
if input_ids["input_ids"] is None: # We only concatenate batches that did at least one step
if batch.input_ids["input_ids"].shape[1] > 1:
raise ValueError("Batch input_ids should be of shape (batch_size, 1)")
# Initialize tensors
if i == 0:
input_ids["input_ids"] = torch.empty( input_ids["input_ids"] = torch.empty(
(total_batch_size, 1), (total_batch_size, 1),
dtype=cache_entry.input_ids["input_ids"].dtype, dtype=batch.input_ids["input_ids"].dtype,
device=cache_entry.input_ids["input_ids"].device, device=batch.input_ids["input_ids"].device,
) )
input_ids["input_ids"][start_index:end_index] = cache_entry.input_ids[
"input_ids"
]
if input_ids["attention_mask"] is None:
input_ids["attention_mask"] = torch.zeros( input_ids["attention_mask"] = torch.zeros(
(total_batch_size, max_sequence_length), (total_batch_size, max_sequence_length),
dtype=cache_entry.input_ids["attention_mask"].dtype, dtype=batch.input_ids["attention_mask"].dtype,
device=cache_entry.input_ids["attention_mask"].device, device=batch.input_ids["attention_mask"].device,
) )
input_ids["attention_mask"][ # input_ids["input_ids"] is always of shape [batch_size, 1]
start_index:end_index, -sequence_length: # We do not need to pad it
] = cache_entry.input_ids["attention_mask"][:, -sequence_length:] input_ids["input_ids"][start_index:end_index] = batch.input_ids["input_ids"]
for j, past in enumerate(cache_entry.input_ids["past_key_values"]): # We need to slice the attention mask to remove padding from previous steps
# TODO: this could be done without the views by using indices input_ids["attention_mask"][
start_index:end_index, -batch.max_sequence_length :
] = batch.input_ids["attention_mask"][:, -batch.max_sequence_length :]
for j, past in enumerate(batch.input_ids["past_key_values"]):
past_keys = past[0] past_keys = past[0]
past_values = past[1] past_values = past[1]
_, head_dim, padded_sequence_length = past_keys.shape _, head_dim, padded_sequence_length = past_keys.shape
# Reshape the tensors to make slicing easier
past_keys = past_keys.view( past_keys = past_keys.view(
batch_size, -1, head_dim, padded_sequence_length batch.size, -1, head_dim, padded_sequence_length
) )
past_values = past_values.view( past_values = past_values.view(
batch_size, -1, padded_sequence_length, head_dim batch.size, -1, padded_sequence_length, head_dim
) )
num_heads = past_keys.shape[1] num_heads = past_keys.shape[1]
# Initialize tensors
# This will run only once per layer
if j == len(input_ids["past_key_values"]): if j == len(input_ids["past_key_values"]):
padded_past_keys = torch.zeros( padded_past_keys = torch.zeros(
( (
@ -167,15 +175,17 @@ class Batch:
[padded_past_keys, padded_past_values] [padded_past_keys, padded_past_values]
) )
# We slice the past keys and values to remove the padding from previous batches
input_ids["past_key_values"][j][0][ input_ids["past_key_values"][j][0][
start_index:end_index, :, :, -(sequence_length - 1): start_index:end_index, :, :, -(batch.max_sequence_length - 1) :
] = past_keys[:, :, :, -(sequence_length - 1):] ] = past_keys[:, :, :, -(batch.max_sequence_length - 1) :]
input_ids["past_key_values"][j][1][ input_ids["past_key_values"][j][1][
start_index:end_index, :, -(sequence_length - 1):, : start_index:end_index, :, -(batch.max_sequence_length - 1) :, :
] = past_values[:, :, -(sequence_length - 1):, :] ] = past_values[:, :, -(batch.max_sequence_length - 1) :, :]
if (i + 1) == len(pb.batch_cached_ids): # If we are on the last batch, we need to reshape the tensors
if (i + 1) == len(batches):
input_ids["past_key_values"][j][0] = input_ids["past_key_values"][ input_ids["past_key_values"][j][0] = input_ids["past_key_values"][
j j
][0].view(total_batch_size * num_heads, head_dim, -1) ][0].view(total_batch_size * num_heads, head_dim, -1)
@ -183,27 +193,27 @@ class Batch:
j j
][1].view(total_batch_size * num_heads, -1, head_dim) ][1].view(total_batch_size * num_heads, -1, head_dim)
start_index += batch_size start_index += batch.size
assert pb.request_ids == request_ids
return cls( return cls(
pb.id, batch_id=batches[0].batch_id,
request_ids, requests=requests,
input_ids, input_ids=input_ids,
all_input_ids, all_input_ids=all_input_ids,
next_token_choosers, next_token_choosers=next_token_choosers,
stopping_criterias, stopping_criterias=stopping_criterias,
size=total_batch_size,
max_sequence_length=max_sequence_length,
) )
@dataclass @dataclass
class FinishedGeneration: class GeneratedText:
request_id: str request: generate_pb2.Request
output: str output: str
def to_pb(self) -> generate_pb2.FinishedGeneration: def to_pb(self) -> generate_pb2.GeneratedText:
return generate_pb2.FinishedGeneration(id=self.request_id, output=self.output) return generate_pb2.GeneratedText(request=self.request, output=self.output)
class BLOOM: class BLOOM:
@ -229,25 +239,28 @@ class BLOOM:
) )
def generate_token( def generate_token(
self, batch: Batch self, batch: Batch
) -> Tuple[List[FinishedGeneration], Optional[CacheEntry]]: ) -> Tuple[List[GeneratedText], Optional[Batch]]:
with torch.no_grad(): with torch.no_grad():
outputs = self.forward(**batch.input_ids) outputs = self.forward(**batch.input_ids)
# List of indices to cache # List of indices to cache
cache_indices = [] next_batch_keep_indices = []
cache_past_indices = [] next_batch_past_keep_indices = []
# New input_ids for next forward; keep in cache # New input_ids for next forward
cache_next_input_ids = [] next_batch_input_ids = []
cache_all_input_ids = [] next_batch_all_input_ids = []
next_batch_size = 0
next_batch_max_sequence_length = 0
# Finished requests # Finished requests
finished_generations: List[FinishedGeneration] = [] generated_texts: List[GeneratedText] = []
# Zipped iterator # Zipped iterator
iterator = zip( iterator = zip(
batch.request_ids, batch.requests,
outputs.logits, outputs.logits,
batch.next_token_choosers, batch.next_token_choosers,
batch.stopping_criterias, batch.stopping_criterias,
@ -256,11 +269,11 @@ class BLOOM:
# For each member of the batch # For each member of the batch
for i, ( for i, (
request_id, request,
logits, logits,
next_token_chooser, next_token_chooser,
stopping_criteria, stopping_criteria,
all_tokens, all_tokens,
) in enumerate(iterator): ) in enumerate(iterator):
# Select next token # Select next token
next_token = next_token_chooser(all_tokens, logits.unsqueeze(0)[:, -1]) next_token = next_token_chooser(all_tokens, logits.unsqueeze(0)[:, -1])
@ -274,64 +287,75 @@ class BLOOM:
output = self.tokenizer.decode( output = self.tokenizer.decode(
all_tokens.squeeze(-1), skip_special_tokens=True all_tokens.squeeze(-1), skip_special_tokens=True
) )
# Add to the list of finished generations with the original request id # Add to the list of finished generations with the original request
finished_generations.append(FinishedGeneration(request_id, output)) generated_texts.append(GeneratedText(request, output))
# must be added to the cache # add to the next batch
else: else:
cache_indices.append(i) next_batch_keep_indices.append(i)
cache_past_indices.extend([j for j in range(i * self.num_heads, (i + 1) * self.num_heads)]) # past_key_values is of shape [batch_size * num_heads, ...]
cache_next_input_ids.append(next_token) # so we need to take into account the `num_heads` stride here
cache_all_input_ids.append(all_tokens) next_batch_past_keep_indices.extend(
[j for j in range(i * self.num_heads, (i + 1) * self.num_heads)]
)
next_batch_input_ids.append(next_token)
next_batch_all_input_ids.append(all_tokens)
next_batch_size += 1
next_batch_max_sequence_length = max(
next_batch_max_sequence_length, len(all_tokens)
)
# No cache is needed, we finished all generations in the batch # We finished all generations in the batch; there is no next batch
if not cache_indices: if not next_batch_keep_indices:
return finished_generations, None return generated_texts, None
# If we finished at least one generation # If we finished at least one generation
cache_input_ids = {"input_ids": torch.cat(cache_next_input_ids, dim=0)} next_batch_input_ids = {"input_ids": torch.cat(next_batch_input_ids, dim=0)}
if finished_generations: if generated_texts:
# Apply indices to attention mask, past key values and other items that need to be cached # Apply indices to attention mask, past key values and other items that need to be cached
cache_input_ids["attention_mask"] = batch.input_ids["attention_mask"][ next_batch_input_ids["attention_mask"] = batch.input_ids["attention_mask"][
cache_indices next_batch_keep_indices
] ]
cache_input_ids["past_key_values"] = [ next_batch_input_ids["past_key_values"] = [
(keys[cache_past_indices], values[cache_past_indices]) (
keys[next_batch_past_keep_indices],
values[next_batch_past_keep_indices],
)
for keys, values in outputs["past_key_values"] for keys, values in outputs["past_key_values"]
] ]
cache_request_ids = [batch.request_ids[i] for i in cache_indices] next_batch_requests = [batch.requests[i] for i in next_batch_keep_indices]
cache_next_token_choosers = [ next_batch_next_token_choosers = [
batch.next_token_choosers[i] for i in cache_indices batch.next_token_choosers[i] for i in next_batch_keep_indices
] ]
cache_stopping_criterias = [ next_batch_stopping_criterias = [
batch.stopping_criterias[i] for i in cache_indices batch.stopping_criterias[i] for i in next_batch_keep_indices
] ]
else: else:
cache_input_ids["attention_mask"] = batch.input_ids["attention_mask"] next_batch_input_ids["attention_mask"] = batch.input_ids["attention_mask"]
cache_input_ids["past_key_values"] = outputs["past_key_values"] next_batch_input_ids["past_key_values"] = outputs["past_key_values"]
cache_request_ids = batch.request_ids next_batch_requests = batch.requests
cache_next_token_choosers = batch.next_token_choosers next_batch_next_token_choosers = batch.next_token_choosers
cache_stopping_criterias = batch.stopping_criterias next_batch_stopping_criterias = batch.stopping_criterias
# Update attention_mask with padding as we added a new token to input_ids # Update attention_mask with padding as we added a new token to input_ids
cache_input_ids["attention_mask"] = torch.cat( next_batch_input_ids["attention_mask"] = torch.cat(
[ [
cache_input_ids["attention_mask"], next_batch_input_ids["attention_mask"],
torch.ones((cache_input_ids["attention_mask"].shape[0], 1)).to( torch.ones((next_batch_size, 1)).to(self.device),
cache_input_ids["attention_mask"].device
),
], ],
dim=1, dim=1,
) )
cache_entry = CacheEntry( next_batch = Batch(
batch.batch_id, batch_id=batch.batch_id,
cache_request_ids, requests=next_batch_requests,
cache_input_ids, input_ids=next_batch_input_ids,
cache_all_input_ids, all_input_ids=next_batch_all_input_ids,
cache_next_token_choosers, next_token_choosers=next_batch_next_token_choosers,
cache_stopping_criterias, stopping_criterias=next_batch_stopping_criterias,
size=next_batch_size,
max_sequence_length=next_batch_max_sequence_length,
) )
return finished_generations, cache_entry return generated_texts, next_batch
class BLOOMSharded(BLOOM): class BLOOMSharded(BLOOM):

View File

@ -10,7 +10,7 @@ from bloom_inference.model import BLOOM, Batch, BLOOMSharded
from bloom_inference.pb import generate_pb2_grpc, generate_pb2 from bloom_inference.pb import generate_pb2_grpc, generate_pb2
class TextGeneration(generate_pb2_grpc.TextGenerationServicer): class TextGenerationService(generate_pb2_grpc.TextGenerationServiceServicer):
def __init__(self, model: BLOOM, cache: Cache, server_urls: List[str]): def __init__(self, model: BLOOM, cache: Cache, server_urls: List[str]):
self.cache = cache self.cache = cache
self.model = model self.model = model
@ -21,32 +21,90 @@ class TextGeneration(generate_pb2_grpc.TextGenerationServicer):
async def ClearCache(self, request, context): async def ClearCache(self, request, context):
self.cache.clear() self.cache.clear()
return generate_pb2.Empty() return generate_pb2.ClearCacheResponse()
async def Generate(self, request, context): async def Generate(self, request, context):
batch = Batch.from_batch_pb(request, self.model.tokenizer, self.model.device) batch = Batch.from_pb(request.batch, self.model.tokenizer, self.model.device)
finished_generations, cache_entry = self.model.generate_token(batch)
self.cache.set(cache_entry)
return generate_pb2.Response( generated_texts, next_batch = self.model.generate_token(batch)
finished=[ self.cache.set(next_batch)
finished_generation.to_pb()
for finished_generation in finished_generations return generate_pb2.GenerateResponse(
generated_texts=[
generated_text.to_pb() for generated_text in generated_texts
], ],
cache_entry=cache_entry.to_pb() if cache_entry else None, batch=next_batch.to_pb() if next_batch else None,
) )
async def GenerateWithCache(self, request, context): async def GenerateWithCache(self, request, context):
batch = Batch.from_batch_cached_pb(request, self.cache) if len(request.batches) == 0:
finished_generations, cache_entry = self.model.generate_token(batch) raise ValueError("Must provide at least one batch")
self.cache.set(cache_entry)
return generate_pb2.Response( batches = []
finished=[ for batch_pb in request.batches:
finished_generation.to_pb() batch = self.cache.pop(batch_pb.id)
for finished_generation in finished_generations if batch is None:
raise ValueError(f"Batch ID {batch_pb.id} not found in cache.")
batches.append(batch)
if len(batches) > 1:
batch = Batch.concatenate(batches)
else:
batch = batches[0]
generated_texts, next_batch = self.model.generate_token(batch)
self.cache.set(next_batch)
return generate_pb2.GenerateWithCacheResponse(
generated_texts=[
generated_text.to_pb() for generated_text in generated_texts
], ],
cache_entry=cache_entry.to_pb() if cache_entry else None, batch=next_batch.to_pb() if next_batch else None,
)
async def GenerateUntilFinished(self, request, context):
batch = Batch.from_pb(request.batch, self.model.tokenizer, self.model.device)
generated_texts = []
while not generated_texts:
generated_texts, next_batch = self.model.generate_token(batch)
batch = next_batch
self.cache.set(next_batch)
return generate_pb2.GenerateUntilFinishedResponse(
generated_texts=[
generated_text.to_pb() for generated_text in generated_texts
],
batch=next_batch.to_pb() if next_batch else None,
)
async def GenerateUntilFinishedWithCache(self, request, context):
if len(request.batches) == 0:
raise ValueError("Must provide at least one batch")
batches = []
for batch_pb in request.batches:
batch = self.cache.pop(batch_pb.id)
if batch is None:
raise ValueError(f"Batch ID {batch_pb.id} not found in cache.")
batches.append(batch)
if len(batches) > 1:
batch = Batch.concatenate(batches)
else:
batch = batches[0]
generated_texts = []
while not generated_texts:
generated_texts, next_batch = self.model.generate_token(batch)
batch = next_batch
self.cache.set(next_batch)
return generate_pb2.GenerateUntilFinishedWithCacheResponse(
generated_texts=[
generated_text.to_pb() for generated_text in generated_texts
],
batch=next_batch.to_pb() if next_batch else None,
) )
@ -71,11 +129,11 @@ def serve(model_name, sharded, shard_directory):
server_urls = [local_url] server_urls = [local_url]
server = aio.server() server = aio.server()
generate_pb2_grpc.add_TextGenerationServicer_to_server( generate_pb2_grpc.add_TextGenerationServiceServicer_to_server(
TextGeneration(model, Cache(), server_urls), server TextGenerationService(model, Cache(), server_urls), server
) )
SERVICE_NAMES = ( SERVICE_NAMES = (
generate_pb2.DESCRIPTOR.services_by_name["TextGeneration"].full_name, generate_pb2.DESCRIPTOR.services_by_name["TextGenerationService"].full_name,
reflection.SERVICE_NAME, reflection.SERVICE_NAME,
) )
reflection.enable_server_reflection(SERVICE_NAMES, server) reflection.enable_server_reflection(SERVICE_NAMES, server)