parent
fa9a088467
commit
4c693e6524
21
README.md
21
README.md
|
@ -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
|
||||
|
||||
|
|
|
@ -2,21 +2,35 @@ syntax = "proto3";
|
|||
|
||||
package generate.v1;
|
||||
|
||||
service TextGeneration {
|
||||
service TextGenerationService {
|
||||
/// Service discovery
|
||||
rpc ServiceDiscovery(Empty) returns (ServiceDiscoveryResponse) {}
|
||||
rpc ServiceDiscovery (ServiceDiscoveryRequest) returns (ServiceDiscoveryResponse) {}
|
||||
/// Empties batch cache
|
||||
rpc ClearCache(Empty) returns (Empty);
|
||||
/// Generate tokens for a batch without cache
|
||||
rpc Generate(Batch) returns (Response);
|
||||
/// Generate tokens for a batch with cache
|
||||
rpc GenerateWithCache(BatchCached) returns (Response);
|
||||
rpc ClearCache (ClearCacheRequest) returns (ClearCacheResponse);
|
||||
/// Generate tokens for a batch
|
||||
rpc Generate (GenerateRequest) returns (GenerateResponse);
|
||||
/// Generate tokens for a list of cached batches
|
||||
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 {
|
||||
/// Other shards urls
|
||||
repeated string urls = 1;
|
||||
}
|
||||
|
||||
/// Empty request
|
||||
message ClearCacheRequest {}
|
||||
|
||||
/// Empty response
|
||||
message ClearCacheResponse {}
|
||||
|
||||
message LogitsWarperParameters {
|
||||
float temperature = 1;
|
||||
uint32 top_k = 2;
|
||||
|
@ -29,10 +43,12 @@ message Request {
|
|||
uint64 id = 1;
|
||||
/// The generation context
|
||||
string inputs = 2;
|
||||
/// The number of tokens inside inputs
|
||||
uint32 input_length = 3;
|
||||
/// Logits Warper Parameters
|
||||
LogitsWarperParameters parameters = 3;
|
||||
LogitsWarperParameters parameters = 4;
|
||||
/// Stopping criteria
|
||||
uint32 max_new_tokens = 4;
|
||||
uint32 max_new_tokens = 5;
|
||||
}
|
||||
|
||||
message Batch {
|
||||
|
@ -40,44 +56,63 @@ message Batch {
|
|||
uint64 id = 1;
|
||||
/// Individual requests
|
||||
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 {
|
||||
/// Batch ID
|
||||
uint64 id = 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;
|
||||
message GeneratedText {
|
||||
/// Request
|
||||
Request request = 1;
|
||||
/// Output
|
||||
string output = 2;
|
||||
}
|
||||
|
||||
message CacheEntry {
|
||||
/// Cache ID; same as batch ID
|
||||
uint64 id = 1;
|
||||
/// Requests present in cache entry
|
||||
repeated uint64 request_ids = 2;
|
||||
/// Sequence length
|
||||
uint32 sequence_length = 3;
|
||||
message GenerateRequest {
|
||||
/// Batch
|
||||
Batch batch = 1;
|
||||
}
|
||||
|
||||
message Response {
|
||||
/// Finished requests (optional)
|
||||
repeated FinishedGeneration finished = 1;
|
||||
/// Cache entry (optional)
|
||||
optional CacheEntry cache_entry = 2;
|
||||
message GenerateResponse {
|
||||
/// Finished requests
|
||||
repeated GeneratedText generated_texts = 1;
|
||||
/// Next batch (cached)
|
||||
optional Batch batch = 2;
|
||||
}
|
||||
|
||||
message GenerateWithCacheRequest {
|
||||
/// Cached batches
|
||||
repeated Batch batches = 1;
|
||||
}
|
||||
|
||||
// Represent an empty message.
|
||||
message Empty {}
|
||||
message GenerateWithCacheResponse {
|
||||
/// 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;
|
||||
}
|
||||
|
|
|
@ -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::Result;
|
||||
use std::time::Duration;
|
||||
|
@ -9,7 +9,7 @@ use tracing::*;
|
|||
/// BLOOM Inference gRPC client
|
||||
#[derive(Clone)]
|
||||
pub struct Client {
|
||||
stub: TextGenerationClient<Timeout<Channel>>,
|
||||
stub: TextGenerationServiceClient<Timeout<Channel>>,
|
||||
}
|
||||
|
||||
impl Client {
|
||||
|
@ -22,13 +22,13 @@ impl Client {
|
|||
let timeout_channel = Timeout::new(channel, timeout);
|
||||
|
||||
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.
|
||||
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()
|
||||
.connect_with_connector(tower::service_fn(move |_: Uri| {
|
||||
tokio::net::UnixStream::connect(path.clone())
|
||||
|
@ -38,13 +38,13 @@ impl Client {
|
|||
let timeout_channel = Timeout::new(channel, timeout);
|
||||
|
||||
Self {
|
||||
stub: TextGenerationClient::new(timeout_channel),
|
||||
stub: TextGenerationServiceClient::new(timeout_channel),
|
||||
}
|
||||
}
|
||||
|
||||
#[instrument(skip(self))]
|
||||
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
|
||||
.stub
|
||||
.service_discovery(request)
|
||||
|
@ -64,7 +64,7 @@ impl Client {
|
|||
|
||||
#[instrument(skip(self))]
|
||||
pub async fn clear_cache(&mut self) -> Result<()> {
|
||||
let request = tonic::Request::new(Empty {});
|
||||
let request = tonic::Request::new(ClearCacheRequest {});
|
||||
self.stub
|
||||
.clear_cache(request)
|
||||
.instrument(info_span!("clear_cache"))
|
||||
|
@ -73,32 +73,59 @@ impl Client {
|
|||
}
|
||||
|
||||
#[instrument(skip(self))]
|
||||
pub async fn generate(
|
||||
&mut self,
|
||||
request: Batch,
|
||||
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> {
|
||||
let request = tonic::Request::new(request);
|
||||
pub async fn generate(&mut self, batch: Batch) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
|
||||
let request = tonic::Request::new(GenerateRequest { batch: Some(batch) });
|
||||
let response = self
|
||||
.stub
|
||||
.generate(request)
|
||||
.instrument(info_span!("generate"))
|
||||
.await?
|
||||
.into_inner();
|
||||
Ok((response.finished, response.cache_entry))
|
||||
Ok((response.generated_texts, response.batch))
|
||||
}
|
||||
|
||||
#[instrument(skip(self))]
|
||||
pub async fn generate_with_cache(
|
||||
&mut self,
|
||||
request: BatchCached,
|
||||
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> {
|
||||
let request = tonic::Request::new(request);
|
||||
batches: Vec<Batch>,
|
||||
) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
|
||||
let request = tonic::Request::new(GenerateWithCacheRequest { batches });
|
||||
let response = self
|
||||
.stub
|
||||
.generate_with_cache(request)
|
||||
.instrument(info_span!("generate_with_cache"))
|
||||
.await?
|
||||
.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))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,9 +5,7 @@ mod pb;
|
|||
mod sharded_client;
|
||||
|
||||
pub use client::Client;
|
||||
pub use pb::generate::v1::{
|
||||
Batch, BatchCached, CacheEntry, FinishedGeneration, LogitsWarperParameters, Request,
|
||||
};
|
||||
pub use pb::generate::v1::{Batch, GeneratedText, LogitsWarperParameters, Request};
|
||||
pub use sharded_client::ShardedClient;
|
||||
use thiserror::Error;
|
||||
pub use tonic::transport::Uri;
|
||||
|
|
|
@ -1,5 +1,5 @@
|
|||
use crate::Result;
|
||||
use crate::{Batch, BatchCached, CacheEntry, Client, FinishedGeneration};
|
||||
use crate::{Batch, Client, GeneratedText};
|
||||
use futures::future::join_all;
|
||||
use std::time::Duration;
|
||||
use tokio::sync::{broadcast, mpsc};
|
||||
|
@ -9,11 +9,19 @@ use tonic::transport::Uri;
|
|||
enum Command {
|
||||
Generate(
|
||||
Batch,
|
||||
mpsc::Sender<Result<(Vec<FinishedGeneration>, Option<CacheEntry>)>>,
|
||||
mpsc::Sender<Result<(Vec<GeneratedText>, Option<Batch>)>>,
|
||||
),
|
||||
GenerateWithCache(
|
||||
BatchCached,
|
||||
mpsc::Sender<Result<(Vec<FinishedGeneration>, Option<CacheEntry>)>>,
|
||||
Vec<Batch>,
|
||||
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<()>>),
|
||||
}
|
||||
|
@ -25,8 +33,16 @@ async fn client_task(mut client: Client, mut request_subscriber: broadcast::Rece
|
|||
let result = client.generate(batch).await;
|
||||
response_tx.try_send(result).unwrap_or(());
|
||||
}
|
||||
Command::GenerateWithCache(batch_cached, response_tx) => {
|
||||
let result = client.generate_with_cache(batch_cached).await;
|
||||
Command::GenerateWithCache(batches, response_tx) => {
|
||||
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(());
|
||||
}
|
||||
Command::ClearCache(response_tx) => {
|
||||
|
@ -74,10 +90,7 @@ impl ShardedClient {
|
|||
Self::from_master_client(master_client).await
|
||||
}
|
||||
|
||||
pub async fn generate(
|
||||
&self,
|
||||
batch: Batch,
|
||||
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> {
|
||||
pub async fn generate(&self, batch: Batch) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
|
||||
let (response_tx, mut response_rx) = mpsc::channel(1);
|
||||
self.request_tx
|
||||
.send(Command::Generate(batch, response_tx))
|
||||
|
@ -87,11 +100,36 @@ impl ShardedClient {
|
|||
|
||||
pub async fn generate_with_cache(
|
||||
&self,
|
||||
batch_cached: BatchCached,
|
||||
) -> Result<(Vec<FinishedGeneration>, Option<CacheEntry>)> {
|
||||
batches: Vec<Batch>,
|
||||
) -> Result<(Vec<GeneratedText>, Option<Batch>)> {
|
||||
let (response_tx, mut response_rx) = mpsc::channel(1);
|
||||
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();
|
||||
response_rx.recv().await.unwrap()
|
||||
}
|
||||
|
|
|
@ -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::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;
|
||||
|
||||
|
@ -32,12 +31,16 @@ impl Batcher {
|
|||
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 {
|
||||
return Err(InferError {});
|
||||
}
|
||||
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();
|
||||
match request_rx.await.unwrap() {
|
||||
Ok(output) => Ok(output),
|
||||
|
@ -51,76 +54,57 @@ async fn batching_task(client: ShardedClient, db: Db, shared: Arc<Shared>) {
|
|||
shared.batching_task.notified().await;
|
||||
|
||||
if let Some(batch) = db.next_batch(32) {
|
||||
let mut cache_entry = infer_batch(batch, &client, &db).await;
|
||||
|
||||
loop {
|
||||
if let Some(entry) = cache_entry {
|
||||
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;
|
||||
let request_ids = batch.requests.iter().map(|req| req.id).collect();
|
||||
let mut cached_batch = match batch.size {
|
||||
size if size > 16 => {
|
||||
wrap_future(client.generate_until_finished(batch), request_ids, &db).await
|
||||
}
|
||||
_ => 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(
|
||||
batch: BatchCached,
|
||||
client: &ShardedClient,
|
||||
async fn wrap_future(
|
||||
future: impl Future<Output = Result<(Vec<GeneratedText>, Option<Batch>), ClientError>>,
|
||||
request_ids: Vec<u64>,
|
||||
db: &Db,
|
||||
) -> Option<CacheEntry> {
|
||||
match client.generate_with_cache(batch.clone()).await {
|
||||
Ok((finished, cache_entry)) => {
|
||||
send_finished(finished, db);
|
||||
cache_entry
|
||||
) -> Option<Batch> {
|
||||
match future.await {
|
||||
Ok((generated_texts, next_batch)) => {
|
||||
send_generated(generated_texts, db);
|
||||
next_batch
|
||||
}
|
||||
Err(err) => {
|
||||
println!("{:?}", err);
|
||||
send_error(err, batch.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,
|
||||
);
|
||||
send_error(err, request_ids, db);
|
||||
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| {
|
||||
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(());
|
||||
});
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ impl Db {
|
|||
|
||||
pub(crate) fn append(
|
||||
&self,
|
||||
input_length: usize,
|
||||
request: GenerateRequest,
|
||||
sender: Sender<Result<String, ClientError>>,
|
||||
) {
|
||||
|
@ -63,6 +64,7 @@ impl Db {
|
|||
let request = Request {
|
||||
id,
|
||||
inputs: request.inputs,
|
||||
input_length: input_length as u32,
|
||||
parameters,
|
||||
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> {
|
||||
if let Some((last_id, requests)) = self.next_requests(max_size) {
|
||||
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 {
|
||||
id: state.next_batch_id,
|
||||
requests,
|
||||
size: size as u32,
|
||||
max_sequence_length,
|
||||
};
|
||||
state.next_batch_start_id = last_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 requests.len() >= min_size {
|
||||
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 {
|
||||
id: state.next_batch_id,
|
||||
requests,
|
||||
size: size as u32,
|
||||
max_sequence_length,
|
||||
};
|
||||
state.next_batch_start_id = last_id + 1;
|
||||
state.next_batch_id += 1;
|
||||
|
|
|
@ -1,31 +1,45 @@
|
|||
use bloom_inference_client::ShardedClient;
|
||||
use poem;
|
||||
use poem::listener::TcpListener;
|
||||
use std::time::Duration;
|
||||
use tokenizers::Tokenizer;
|
||||
|
||||
mod server;
|
||||
mod validation;
|
||||
|
||||
use validation::Validation;
|
||||
|
||||
mod db;
|
||||
|
||||
use db::Db;
|
||||
|
||||
mod batcher;
|
||||
|
||||
use batcher::Batcher;
|
||||
|
||||
#[tokio::main]
|
||||
async fn main() -> Result<(), std::io::Error> {
|
||||
tracing_subscriber::fmt::init();
|
||||
fn main() -> Result<(), std::io::Error> {
|
||||
let tokenizer = Tokenizer::from_pretrained("bigscience/bloom", None).unwrap();
|
||||
|
||||
let sharded_client =
|
||||
ShardedClient::connect_uds("/tmp/bloom-inference-0".to_string(), Duration::from_secs(5))
|
||||
tokio::runtime::Builder::new_multi_thread()
|
||||
.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;
|
||||
sharded_client
|
||||
.clear_cache()
|
||||
.await
|
||||
.expect("Unable to clear cache");
|
||||
tracing::info!("Connected");
|
||||
sharded_client
|
||||
.clear_cache()
|
||||
.await
|
||||
.expect("Unable to clear cache");
|
||||
tracing::info!("Connected");
|
||||
|
||||
let addr = "127.0.0.1:3000".to_string();
|
||||
let listener = TcpListener::bind(addr);
|
||||
let addr = "127.0.0.1:3000".to_string();
|
||||
let listener = TcpListener::bind(addr);
|
||||
|
||||
server::run(sharded_client, listener).await
|
||||
server::run(sharded_client, tokenizer, listener).await
|
||||
})
|
||||
}
|
||||
|
|
|
@ -1,12 +1,13 @@
|
|||
use poem::{EndpointExt, handler, post, Route, Server};
|
||||
use crate::{Batcher, ShardedClient, Validation};
|
||||
use poem::http::StatusCode;
|
||||
use poem::listener::TcpListener;
|
||||
use poem::middleware::AddData;
|
||||
use poem::web::{Data, Json};
|
||||
use tokio::time::Instant;
|
||||
use crate::{Batcher, ShardedClient};
|
||||
use tracing::instrument;
|
||||
use poem::{handler, post, EndpointExt, Route, Server};
|
||||
use serde::Deserialize;
|
||||
use tokenizers::Tokenizer;
|
||||
use tokio::time::Instant;
|
||||
use tracing::instrument;
|
||||
|
||||
#[derive(Clone, Debug, Deserialize)]
|
||||
pub(crate) struct GenerateParameters {
|
||||
|
@ -59,21 +60,24 @@ pub(crate) struct GenerateRequest {
|
|||
pub parameters: GenerateParameters,
|
||||
}
|
||||
|
||||
|
||||
#[handler]
|
||||
#[instrument(skip(infer), fields(time, time_per_token))]
|
||||
#[instrument(skip(validation, infer), fields(time, time_per_token))]
|
||||
async fn generate(
|
||||
validation: Data<&Validation>,
|
||||
infer: Data<&Batcher>,
|
||||
req: Json<GenerateRequest>,
|
||||
) -> poem::Result<Json<serde_json::Value>> {
|
||||
let start = Instant::now();
|
||||
|
||||
let output = infer
|
||||
.infer(GenerateRequest {
|
||||
let (input_length, validated_request) = validation
|
||||
.validate(GenerateRequest {
|
||||
inputs: req.inputs.clone(),
|
||||
parameters: req.parameters.clone(),
|
||||
})
|
||||
.await;
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let output = infer.infer(input_length, validated_request).await;
|
||||
|
||||
match output {
|
||||
Ok(generated_text) => {
|
||||
|
@ -92,20 +96,22 @@ async fn generate(
|
|||
}
|
||||
}
|
||||
|
||||
pub async fn run(client: ShardedClient, listener: TcpListener<String>) -> Result<(), std::io::Error> {
|
||||
client
|
||||
.clear_cache()
|
||||
.await
|
||||
.expect("Unable to clear cache");
|
||||
pub async fn run(
|
||||
client: ShardedClient,
|
||||
tokenizer: Tokenizer,
|
||||
listener: TcpListener<String>,
|
||||
) -> Result<(), std::io::Error> {
|
||||
client.clear_cache().await.expect("Unable to clear cache");
|
||||
tracing::info!("Connected");
|
||||
|
||||
let infer = Batcher::new(client);
|
||||
|
||||
let validation = Validation::new(tokenizer);
|
||||
|
||||
let app = Route::new()
|
||||
.at("/generate", post(generate))
|
||||
.with(AddData::new(validation))
|
||||
.with(AddData::new(infer));
|
||||
|
||||
Server::new(listener)
|
||||
.run(app)
|
||||
.await
|
||||
}
|
||||
Server::new(listener).run(app).await
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
|
@ -1,44 +1,19 @@
|
|||
import torch
|
||||
|
||||
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),
|
||||
)
|
||||
from bloom_inference.model import Batch
|
||||
from typing import Dict, Optional
|
||||
|
||||
|
||||
class Cache:
|
||||
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)
|
||||
|
||||
def set(self, entry: CacheEntry):
|
||||
def set(self, entry: Batch):
|
||||
if entry is not None:
|
||||
self.cache[entry.batch_id] = entry
|
||||
|
||||
def delete(self, batch_id: str):
|
||||
def delete(self, batch_id: int):
|
||||
del self.cache[batch_id]
|
||||
|
||||
def clear(self):
|
||||
|
|
|
@ -8,7 +8,6 @@ from typing import List, Tuple, Optional, Dict
|
|||
from transformers import AutoTokenizer, AutoModelForCausalLM, AutoConfig
|
||||
from transformers.modeling_utils import no_init_weights
|
||||
|
||||
from bloom_inference.cache import CacheEntry
|
||||
from bloom_inference.pb import generate_pb2
|
||||
from bloom_inference.shard_model import shard_model, match_suffix
|
||||
from bloom_inference.utils import (
|
||||
|
@ -24,25 +23,35 @@ torch.manual_seed(0)
|
|||
@dataclass
|
||||
class Batch:
|
||||
batch_id: int
|
||||
request_ids: List[int]
|
||||
requests: List[generate_pb2.Request]
|
||||
input_ids: Dict[str, torch.Tensor]
|
||||
all_input_ids: List[torch.Tensor]
|
||||
next_token_choosers: List[NextTokenChooser]
|
||||
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
|
||||
def from_batch_pb(
|
||||
cls, pb: generate_pb2.Batch, tokenizer: AutoTokenizer, device: torch.device
|
||||
def from_pb(
|
||||
cls, pb: generate_pb2.Batch, tokenizer: AutoTokenizer, device: torch.device
|
||||
) -> "Batch":
|
||||
request_ids = []
|
||||
inputs = []
|
||||
next_token_choosers = []
|
||||
stopping_criterias = []
|
||||
input_lengths = []
|
||||
|
||||
# Parse batch
|
||||
for r in pb.requests:
|
||||
request_ids.append(r.id)
|
||||
inputs.append(r.inputs)
|
||||
input_lengths.append(r.input_length)
|
||||
next_token_choosers.append(
|
||||
NextTokenChooser(
|
||||
temperature=r.parameters.temperature,
|
||||
|
@ -54,94 +63,93 @@ class Batch:
|
|||
stopping_criterias.append(StoppingCriteria(max_new_tokens=r.max_new_tokens))
|
||||
|
||||
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(
|
||||
pb.id,
|
||||
request_ids,
|
||||
input_ids,
|
||||
all_input_ids,
|
||||
next_token_choosers,
|
||||
stopping_criterias,
|
||||
batch_id=pb.id,
|
||||
requests=pb.requests,
|
||||
input_ids=input_ids,
|
||||
all_input_ids=all_input_ids,
|
||||
next_token_choosers=next_token_choosers,
|
||||
stopping_criterias=stopping_criterias,
|
||||
size=pb.size,
|
||||
max_sequence_length=pb.max_sequence_length,
|
||||
)
|
||||
|
||||
@classmethod
|
||||
def from_cache_entry(cls, cache_entry: CacheEntry) -> "Batch":
|
||||
return cls(
|
||||
cache_entry.batch_id,
|
||||
cache_entry.request_ids,
|
||||
cache_entry.input_ids,
|
||||
cache_entry.all_input_ids,
|
||||
cache_entry.next_token_choosers,
|
||||
cache_entry.stopping_criterias,
|
||||
)
|
||||
def concatenate(cls, batches: List["Batch"]) -> "Batch":
|
||||
# Used for padding
|
||||
total_batch_size = sum(batch.size for batch in batches)
|
||||
max_sequence_length = max(batch.max_sequence_length for batch in batches)
|
||||
|
||||
@classmethod
|
||||
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
|
||||
# Batch attributes
|
||||
input_ids = {"input_ids": None, "attention_mask": None, "past_key_values": []}
|
||||
request_ids = []
|
||||
requests = []
|
||||
all_input_ids = []
|
||||
next_token_choosers = []
|
||||
stopping_criterias = []
|
||||
|
||||
# Used for slicing correctly inside the tensors
|
||||
# Equivalent to a cumsum on batch sizes
|
||||
start_index = 0
|
||||
for i, batch_id in enumerate(pb.batch_cached_ids):
|
||||
cache_entry = cache.pop(batch_id)
|
||||
if cache_entry is None:
|
||||
raise ValueError(f"Batch ID {batch_id} not found in cache")
|
||||
request_ids.extend(cache_entry.request_ids)
|
||||
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)
|
||||
for i, batch in enumerate(batches):
|
||||
requests.extend(batch.requests)
|
||||
all_input_ids.extend(batch.all_input_ids)
|
||||
next_token_choosers.extend(batch.next_token_choosers)
|
||||
stopping_criterias.extend(batch.stopping_criterias)
|
||||
|
||||
batch_size = len(cache_entry.request_ids)
|
||||
end_index = start_index + batch_size
|
||||
sequence_length = max(len(entry) for entry in cache_entry.all_input_ids)
|
||||
# Slicing end index for this batch
|
||||
end_index = start_index + batch.size
|
||||
|
||||
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(
|
||||
(total_batch_size, 1),
|
||||
dtype=cache_entry.input_ids["input_ids"].dtype,
|
||||
device=cache_entry.input_ids["input_ids"].device,
|
||||
dtype=batch.input_ids["input_ids"].dtype,
|
||||
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(
|
||||
(total_batch_size, max_sequence_length),
|
||||
dtype=cache_entry.input_ids["attention_mask"].dtype,
|
||||
device=cache_entry.input_ids["attention_mask"].device,
|
||||
dtype=batch.input_ids["attention_mask"].dtype,
|
||||
device=batch.input_ids["attention_mask"].device,
|
||||
)
|
||||
|
||||
input_ids["attention_mask"][
|
||||
start_index:end_index, -sequence_length:
|
||||
] = cache_entry.input_ids["attention_mask"][:, -sequence_length:]
|
||||
# input_ids["input_ids"] is always of shape [batch_size, 1]
|
||||
# We do not need to pad it
|
||||
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"]):
|
||||
# TODO: this could be done without the views by using indices
|
||||
# We need to slice the attention mask to remove padding from previous steps
|
||||
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_values = past[1]
|
||||
|
||||
_, head_dim, padded_sequence_length = past_keys.shape
|
||||
|
||||
# Reshape the tensors to make slicing easier
|
||||
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(
|
||||
batch_size, -1, padded_sequence_length, head_dim
|
||||
batch.size, -1, padded_sequence_length, head_dim
|
||||
)
|
||||
num_heads = past_keys.shape[1]
|
||||
|
||||
# Initialize tensors
|
||||
# This will run only once per layer
|
||||
if j == len(input_ids["past_key_values"]):
|
||||
padded_past_keys = torch.zeros(
|
||||
(
|
||||
|
@ -167,15 +175,17 @@ class Batch:
|
|||
[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][
|
||||
start_index:end_index, :, :, -(sequence_length - 1):
|
||||
] = past_keys[:, :, :, -(sequence_length - 1):]
|
||||
start_index:end_index, :, :, -(batch.max_sequence_length - 1) :
|
||||
] = past_keys[:, :, :, -(batch.max_sequence_length - 1) :]
|
||||
|
||||
input_ids["past_key_values"][j][1][
|
||||
start_index:end_index, :, -(sequence_length - 1):, :
|
||||
] = past_values[:, :, -(sequence_length - 1):, :]
|
||||
start_index:end_index, :, -(batch.max_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"][
|
||||
j
|
||||
][0].view(total_batch_size * num_heads, head_dim, -1)
|
||||
|
@ -183,27 +193,27 @@ class Batch:
|
|||
j
|
||||
][1].view(total_batch_size * num_heads, -1, head_dim)
|
||||
|
||||
start_index += batch_size
|
||||
|
||||
assert pb.request_ids == request_ids
|
||||
start_index += batch.size
|
||||
|
||||
return cls(
|
||||
pb.id,
|
||||
request_ids,
|
||||
input_ids,
|
||||
all_input_ids,
|
||||
next_token_choosers,
|
||||
stopping_criterias,
|
||||
batch_id=batches[0].batch_id,
|
||||
requests=requests,
|
||||
input_ids=input_ids,
|
||||
all_input_ids=all_input_ids,
|
||||
next_token_choosers=next_token_choosers,
|
||||
stopping_criterias=stopping_criterias,
|
||||
size=total_batch_size,
|
||||
max_sequence_length=max_sequence_length,
|
||||
)
|
||||
|
||||
|
||||
@dataclass
|
||||
class FinishedGeneration:
|
||||
request_id: str
|
||||
class GeneratedText:
|
||||
request: generate_pb2.Request
|
||||
output: str
|
||||
|
||||
def to_pb(self) -> generate_pb2.FinishedGeneration:
|
||||
return generate_pb2.FinishedGeneration(id=self.request_id, output=self.output)
|
||||
def to_pb(self) -> generate_pb2.GeneratedText:
|
||||
return generate_pb2.GeneratedText(request=self.request, output=self.output)
|
||||
|
||||
|
||||
class BLOOM:
|
||||
|
@ -229,25 +239,28 @@ class BLOOM:
|
|||
)
|
||||
|
||||
def generate_token(
|
||||
self, batch: Batch
|
||||
) -> Tuple[List[FinishedGeneration], Optional[CacheEntry]]:
|
||||
self, batch: Batch
|
||||
) -> Tuple[List[GeneratedText], Optional[Batch]]:
|
||||
with torch.no_grad():
|
||||
outputs = self.forward(**batch.input_ids)
|
||||
|
||||
# List of indices to cache
|
||||
cache_indices = []
|
||||
cache_past_indices = []
|
||||
next_batch_keep_indices = []
|
||||
next_batch_past_keep_indices = []
|
||||
|
||||
# New input_ids for next forward; keep in cache
|
||||
cache_next_input_ids = []
|
||||
cache_all_input_ids = []
|
||||
# New input_ids for next forward
|
||||
next_batch_input_ids = []
|
||||
next_batch_all_input_ids = []
|
||||
|
||||
next_batch_size = 0
|
||||
next_batch_max_sequence_length = 0
|
||||
|
||||
# Finished requests
|
||||
finished_generations: List[FinishedGeneration] = []
|
||||
generated_texts: List[GeneratedText] = []
|
||||
|
||||
# Zipped iterator
|
||||
iterator = zip(
|
||||
batch.request_ids,
|
||||
batch.requests,
|
||||
outputs.logits,
|
||||
batch.next_token_choosers,
|
||||
batch.stopping_criterias,
|
||||
|
@ -256,11 +269,11 @@ class BLOOM:
|
|||
|
||||
# For each member of the batch
|
||||
for i, (
|
||||
request_id,
|
||||
logits,
|
||||
next_token_chooser,
|
||||
stopping_criteria,
|
||||
all_tokens,
|
||||
request,
|
||||
logits,
|
||||
next_token_chooser,
|
||||
stopping_criteria,
|
||||
all_tokens,
|
||||
) in enumerate(iterator):
|
||||
# Select next token
|
||||
next_token = next_token_chooser(all_tokens, logits.unsqueeze(0)[:, -1])
|
||||
|
@ -274,64 +287,75 @@ class BLOOM:
|
|||
output = self.tokenizer.decode(
|
||||
all_tokens.squeeze(-1), skip_special_tokens=True
|
||||
)
|
||||
# Add to the list of finished generations with the original request id
|
||||
finished_generations.append(FinishedGeneration(request_id, output))
|
||||
# must be added to the cache
|
||||
# Add to the list of finished generations with the original request
|
||||
generated_texts.append(GeneratedText(request, output))
|
||||
# add to the next batch
|
||||
else:
|
||||
cache_indices.append(i)
|
||||
cache_past_indices.extend([j for j in range(i * self.num_heads, (i + 1) * self.num_heads)])
|
||||
cache_next_input_ids.append(next_token)
|
||||
cache_all_input_ids.append(all_tokens)
|
||||
next_batch_keep_indices.append(i)
|
||||
# past_key_values is of shape [batch_size * num_heads, ...]
|
||||
# so we need to take into account the `num_heads` stride here
|
||||
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
|
||||
if not cache_indices:
|
||||
return finished_generations, None
|
||||
# We finished all generations in the batch; there is no next batch
|
||||
if not next_batch_keep_indices:
|
||||
return generated_texts, None
|
||||
|
||||
# If we finished at least one generation
|
||||
cache_input_ids = {"input_ids": torch.cat(cache_next_input_ids, dim=0)}
|
||||
if finished_generations:
|
||||
next_batch_input_ids = {"input_ids": torch.cat(next_batch_input_ids, dim=0)}
|
||||
if generated_texts:
|
||||
# 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"][
|
||||
cache_indices
|
||||
next_batch_input_ids["attention_mask"] = batch.input_ids["attention_mask"][
|
||||
next_batch_keep_indices
|
||||
]
|
||||
cache_input_ids["past_key_values"] = [
|
||||
(keys[cache_past_indices], values[cache_past_indices])
|
||||
next_batch_input_ids["past_key_values"] = [
|
||||
(
|
||||
keys[next_batch_past_keep_indices],
|
||||
values[next_batch_past_keep_indices],
|
||||
)
|
||||
for keys, values in outputs["past_key_values"]
|
||||
]
|
||||
cache_request_ids = [batch.request_ids[i] for i in cache_indices]
|
||||
cache_next_token_choosers = [
|
||||
batch.next_token_choosers[i] for i in cache_indices
|
||||
next_batch_requests = [batch.requests[i] for i in next_batch_keep_indices]
|
||||
next_batch_next_token_choosers = [
|
||||
batch.next_token_choosers[i] for i in next_batch_keep_indices
|
||||
]
|
||||
cache_stopping_criterias = [
|
||||
batch.stopping_criterias[i] for i in cache_indices
|
||||
next_batch_stopping_criterias = [
|
||||
batch.stopping_criterias[i] for i in next_batch_keep_indices
|
||||
]
|
||||
else:
|
||||
cache_input_ids["attention_mask"] = batch.input_ids["attention_mask"]
|
||||
cache_input_ids["past_key_values"] = outputs["past_key_values"]
|
||||
cache_request_ids = batch.request_ids
|
||||
cache_next_token_choosers = batch.next_token_choosers
|
||||
cache_stopping_criterias = batch.stopping_criterias
|
||||
next_batch_input_ids["attention_mask"] = batch.input_ids["attention_mask"]
|
||||
next_batch_input_ids["past_key_values"] = outputs["past_key_values"]
|
||||
next_batch_requests = batch.requests
|
||||
next_batch_next_token_choosers = batch.next_token_choosers
|
||||
next_batch_stopping_criterias = batch.stopping_criterias
|
||||
|
||||
# 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"],
|
||||
torch.ones((cache_input_ids["attention_mask"].shape[0], 1)).to(
|
||||
cache_input_ids["attention_mask"].device
|
||||
),
|
||||
next_batch_input_ids["attention_mask"],
|
||||
torch.ones((next_batch_size, 1)).to(self.device),
|
||||
],
|
||||
dim=1,
|
||||
)
|
||||
|
||||
cache_entry = CacheEntry(
|
||||
batch.batch_id,
|
||||
cache_request_ids,
|
||||
cache_input_ids,
|
||||
cache_all_input_ids,
|
||||
cache_next_token_choosers,
|
||||
cache_stopping_criterias,
|
||||
next_batch = Batch(
|
||||
batch_id=batch.batch_id,
|
||||
requests=next_batch_requests,
|
||||
input_ids=next_batch_input_ids,
|
||||
all_input_ids=next_batch_all_input_ids,
|
||||
next_token_choosers=next_batch_next_token_choosers,
|
||||
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):
|
||||
|
|
|
@ -10,7 +10,7 @@ from bloom_inference.model import BLOOM, Batch, BLOOMSharded
|
|||
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]):
|
||||
self.cache = cache
|
||||
self.model = model
|
||||
|
@ -21,32 +21,90 @@ class TextGeneration(generate_pb2_grpc.TextGenerationServicer):
|
|||
|
||||
async def ClearCache(self, request, context):
|
||||
self.cache.clear()
|
||||
return generate_pb2.Empty()
|
||||
return generate_pb2.ClearCacheResponse()
|
||||
|
||||
async def Generate(self, request, context):
|
||||
batch = Batch.from_batch_pb(request, self.model.tokenizer, self.model.device)
|
||||
finished_generations, cache_entry = self.model.generate_token(batch)
|
||||
self.cache.set(cache_entry)
|
||||
batch = Batch.from_pb(request.batch, self.model.tokenizer, self.model.device)
|
||||
|
||||
return generate_pb2.Response(
|
||||
finished=[
|
||||
finished_generation.to_pb()
|
||||
for finished_generation in finished_generations
|
||||
generated_texts, next_batch = self.model.generate_token(batch)
|
||||
self.cache.set(next_batch)
|
||||
|
||||
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):
|
||||
batch = Batch.from_batch_cached_pb(request, self.cache)
|
||||
finished_generations, cache_entry = self.model.generate_token(batch)
|
||||
self.cache.set(cache_entry)
|
||||
if len(request.batches) == 0:
|
||||
raise ValueError("Must provide at least one batch")
|
||||
|
||||
return generate_pb2.Response(
|
||||
finished=[
|
||||
finished_generation.to_pb()
|
||||
for finished_generation in finished_generations
|
||||
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, 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 = aio.server()
|
||||
generate_pb2_grpc.add_TextGenerationServicer_to_server(
|
||||
TextGeneration(model, Cache(), server_urls), server
|
||||
generate_pb2_grpc.add_TextGenerationServiceServicer_to_server(
|
||||
TextGenerationService(model, Cache(), server_urls), server
|
||||
)
|
||||
SERVICE_NAMES = (
|
||||
generate_pb2.DESCRIPTOR.services_by_name["TextGeneration"].full_name,
|
||||
generate_pb2.DESCRIPTOR.services_by_name["TextGenerationService"].full_name,
|
||||
reflection.SERVICE_NAME,
|
||||
)
|
||||
reflection.enable_server_reflection(SERVICE_NAMES, server)
|
||||
|
|
Loading…
Reference in New Issue