-
Notifications
You must be signed in to change notification settings - Fork 44
feat: add ProduceResult
with encoded request size metadata
#276
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||
---|---|---|---|---|---|---|---|---|
|
@@ -300,6 +300,16 @@ impl BatchProducerBuilder { | |||||||
} | ||||||||
} | ||||||||
|
||||||||
/// The result of the produce call. | ||||||||
#[derive(Debug, Default)] | ||||||||
pub struct ProduceResult { | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
makes it easier to extend this later w/o introducing breaking changes. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added in a12992f |
||||||||
/// The offsets of the produced records. | ||||||||
pub offsets: Vec<i64>, | ||||||||
|
||||||||
/// The size of the request encoded in bytes. | ||||||||
pub encoded_request_size: usize, | ||||||||
} | ||||||||
|
||||||||
/// The [`ProducerClient`] provides an abstraction over a Kafka client than can | ||||||||
/// produce a record. | ||||||||
/// | ||||||||
|
@@ -316,15 +326,15 @@ pub trait ProducerClient: std::fmt::Debug + Send + Sync { | |||||||
&self, | ||||||||
records: Vec<Record>, | ||||||||
compression: Compression, | ||||||||
) -> BoxFuture<'_, Result<Vec<i64>, ClientError>>; | ||||||||
) -> BoxFuture<'_, Result<ProduceResult, ClientError>>; | ||||||||
} | ||||||||
|
||||||||
impl ProducerClient for PartitionClient { | ||||||||
fn produce( | ||||||||
&self, | ||||||||
records: Vec<Record>, | ||||||||
compression: Compression, | ||||||||
) -> BoxFuture<'_, Result<Vec<i64>, ClientError>> { | ||||||||
) -> BoxFuture<'_, Result<ProduceResult, ClientError>> { | ||||||||
Box::pin(self.produce(records, compression)) | ||||||||
} | ||||||||
} | ||||||||
|
@@ -703,7 +713,7 @@ mod tests { | |||||||
&self, | ||||||||
records: Vec<Record>, | ||||||||
_compression: Compression, | ||||||||
) -> BoxFuture<'_, Result<Vec<i64>, ClientError>> { | ||||||||
) -> BoxFuture<'_, Result<ProduceResult, ClientError>> { | ||||||||
Box::pin(async move { | ||||||||
tokio::time::sleep(self.delay).await; | ||||||||
|
||||||||
|
@@ -727,7 +737,12 @@ mod tests { | |||||||
.map(|x| (x + offset_base) as i64) | ||||||||
.collect(); | ||||||||
batch_sizes.push(records.len()); | ||||||||
Ok(offsets) | ||||||||
let record_size = records.iter().map(|r| r.approximate_size()).sum::<usize>(); | ||||||||
Ok(ProduceResult { | ||||||||
offsets, | ||||||||
// Uses the approximate size of the records to estimate the size of the request. | ||||||||
encoded_request_size: record_size, | ||||||||
}) | ||||||||
}) | ||||||||
} | ||||||||
} | ||||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,7 +27,10 @@ use tokio::{ | |
}; | ||
use tracing::{debug, info, warn}; | ||
|
||
use crate::protocol::{messages::ApiVersionsRequest, traits::ReadType}; | ||
use crate::protocol::{ | ||
messages::{ApiVersionsRequest, ResponseBodyWithMetadata}, | ||
traits::ReadType, | ||
}; | ||
use crate::{ | ||
backoff::ErrorOrThrottle, | ||
protocol::{ | ||
|
@@ -315,6 +318,19 @@ where | |
} | ||
|
||
pub async fn request<R>(&self, msg: R) -> Result<R::ResponseBody, RequestError> | ||
|
||
where | ||
R: RequestBody + Send + WriteVersionedType<Vec<u8>>, | ||
R::ResponseBody: ReadVersionedType<Cursor<Vec<u8>>>, | ||
{ | ||
self.request_with_version_ranges(msg, &self.version_ranges) | ||
.await | ||
.map(|body| body.response) | ||
} | ||
|
||
pub async fn request_with_metadata<R>( | ||
&self, | ||
msg: R, | ||
) -> Result<ResponseBodyWithMetadata<R::ResponseBody>, RequestError> | ||
where | ||
R: RequestBody + Send + WriteVersionedType<Vec<u8>>, | ||
R::ResponseBody: ReadVersionedType<Cursor<Vec<u8>>>, | ||
|
@@ -327,7 +343,7 @@ where | |
&self, | ||
msg: R, | ||
version_ranges: &HashMap<ApiKey, ApiVersionRange>, | ||
) -> Result<R::ResponseBody, RequestError> | ||
) -> Result<ResponseBodyWithMetadata<R::ResponseBody>, RequestError> | ||
where | ||
R: RequestBody + Send + WriteVersionedType<Vec<u8>>, | ||
R::ResponseBody: ReadVersionedType<Cursor<Vec<u8>>>, | ||
|
@@ -371,6 +387,7 @@ where | |
.write_versioned(&mut buf, header_version) | ||
.expect("Writing header to buffer should always work"); | ||
msg.write_versioned(&mut buf, body_api_version)?; | ||
let encoded_size = buf.len(); | ||
|
||
let (tx, rx) = channel(); | ||
|
||
|
@@ -412,7 +429,10 @@ where | |
}); | ||
} | ||
|
||
Ok(body) | ||
Ok(ResponseBodyWithMetadata { | ||
response: body, | ||
encoded_request_size: encoded_size, | ||
}) | ||
} | ||
|
||
async fn send_message(&self, msg: Vec<u8>) -> Result<(), RequestError> { | ||
|
@@ -468,7 +488,7 @@ where | |
.request_with_version_ranges(&body, &version_ranges) | ||
.await | ||
{ | ||
Ok(response) => { | ||
Ok(ResponseBodyWithMetadata { response, .. }) => { | ||
if let Err(ErrorOrThrottle::Throttle(throttle)) = | ||
maybe_throttle::<SyncVersionsError>(response.throttle_time_ms) | ||
{ | ||
|
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -86,6 +86,16 @@ impl<W: Write, T: WriteVersionedType<W>> WriteVersionedType<W> for &T { | |||||
} | ||||||
} | ||||||
|
||||||
/// A response body with metadata about the request. | ||||||
|
/// A response body with metadata about the request. | |
/// A response body with metadata about the request & response. |
I think we can make this a bit more generic for future additions.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated in a12992f
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the end2end test should assert this new field as well. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added in b1b5c99 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This makes it clearer which API you're referencing here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated in a12992f