1
1
use crate :: {
2
- backoff:: { Backoff , BackoffConfig } ,
2
+ backoff:: { Backoff , BackoffConfig , ErrorOrThrottle } ,
3
3
client:: error:: { Error , RequestContext , Result } ,
4
4
connection:: {
5
5
BrokerCache , BrokerConnection , BrokerConnector , MessengerTransport , MetadataLookupMode ,
@@ -19,6 +19,7 @@ use crate::{
19
19
record:: { Record as ProtocolRecord , * } ,
20
20
} ,
21
21
record:: { Record , RecordAndOffset } ,
22
+ throttle:: maybe_throttle,
22
23
validation:: ExactlyOne ,
23
24
} ;
24
25
use async_trait:: async_trait;
@@ -151,7 +152,7 @@ impl PartitionClient {
151
152
& * brokers,
152
153
"leader_detection" ,
153
154
|| async move {
154
- scope. get ( ) . await ?;
155
+ scope. get ( ) . await . map_err ( ErrorOrThrottle :: Error ) ?;
155
156
Ok ( ( ) )
156
157
} ,
157
158
)
@@ -190,9 +191,14 @@ impl PartitionClient {
190
191
self ,
191
192
"produce" ,
192
193
|| async move {
193
- let broker = self . get ( ) . await ?;
194
- let response = broker. request ( & request) . await ?;
194
+ let broker = self . get ( ) . await . map_err ( ErrorOrThrottle :: Error ) ?;
195
+ let response = broker
196
+ . request ( & request)
197
+ . await
198
+ . map_err ( |e| ErrorOrThrottle :: Error ( e. into ( ) ) ) ?;
199
+ maybe_throttle ( response. throttle_time_ms ) ?;
195
200
process_produce_response ( self . partition , & self . topic , n, response)
201
+ . map_err ( ErrorOrThrottle :: Error )
196
202
} ,
197
203
)
198
204
. await
@@ -221,8 +227,16 @@ impl PartitionClient {
221
227
self ,
222
228
"fetch_records" ,
223
229
|| async move {
224
- let response = self . get ( ) . await ?. request ( & request) . await ?;
230
+ let response = self
231
+ . get ( )
232
+ . await
233
+ . map_err ( ErrorOrThrottle :: Error ) ?
234
+ . request ( & request)
235
+ . await
236
+ . map_err ( |e| ErrorOrThrottle :: Error ( e. into ( ) ) ) ?;
237
+ maybe_throttle ( response. throttle_time_ms ) ?;
225
238
process_fetch_response ( self . partition , & self . topic , response, offset)
239
+ . map_err ( ErrorOrThrottle :: Error )
226
240
} ,
227
241
)
228
242
. await ?;
@@ -248,8 +262,16 @@ impl PartitionClient {
248
262
self ,
249
263
"get_offset" ,
250
264
|| async move {
251
- let response = self . get ( ) . await ?. request ( & request) . await ?;
265
+ let response = self
266
+ . get ( )
267
+ . await
268
+ . map_err ( ErrorOrThrottle :: Error ) ?
269
+ . request ( & request)
270
+ . await
271
+ . map_err ( |e| ErrorOrThrottle :: Error ( e. into ( ) ) ) ?;
272
+ maybe_throttle ( response. throttle_time_ms ) ?;
252
273
process_list_offsets_response ( self . partition , & self . topic , response)
274
+ . map_err ( ErrorOrThrottle :: Error )
253
275
} ,
254
276
)
255
277
. await ?;
@@ -272,8 +294,16 @@ impl PartitionClient {
272
294
self ,
273
295
"delete_records" ,
274
296
|| async move {
275
- let response = self . get ( ) . await ?. request ( & request) . await ?;
297
+ let response = self
298
+ . get ( )
299
+ . await
300
+ . map_err ( ErrorOrThrottle :: Error ) ?
301
+ . request ( & request)
302
+ . await
303
+ . map_err ( |e| ErrorOrThrottle :: Error ( e. into ( ) ) ) ?;
304
+ maybe_throttle ( Some ( response. throttle_time_ms ) ) ?;
276
305
process_delete_records_response ( & self . topic , self . partition , response)
306
+ . map_err ( ErrorOrThrottle :: Error )
277
307
} ,
278
308
)
279
309
. await ?;
@@ -462,15 +492,20 @@ async fn maybe_retry<B, R, F, T>(
462
492
where
463
493
B : BrokerCache ,
464
494
R : ( Fn ( ) -> F ) + Send + Sync ,
465
- F : std:: future:: Future < Output = Result < T > > + Send ,
495
+ F : std:: future:: Future < Output = Result < T , ErrorOrThrottle < Error > > > + Send ,
466
496
{
467
497
let mut backoff = Backoff :: new ( backoff_config) ;
468
498
469
499
backoff
470
500
. retry_with_backoff ( request_name, || async {
471
501
let error = match f ( ) . await {
472
- Ok ( v) => return ControlFlow :: Break ( Ok ( v) ) ,
473
- Err ( e) => e,
502
+ Ok ( v) => {
503
+ return ControlFlow :: Break ( Ok ( v) ) ;
504
+ }
505
+ Err ( ErrorOrThrottle :: Throttle ( throttle) ) => {
506
+ return ControlFlow :: Continue ( ErrorOrThrottle :: Throttle ( throttle) ) ;
507
+ }
508
+ Err ( ErrorOrThrottle :: Error ( e) ) => e,
474
509
} ;
475
510
476
511
let retry = match error {
@@ -504,7 +539,7 @@ where
504
539
} ;
505
540
506
541
if retry {
507
- ControlFlow :: Continue ( error)
542
+ ControlFlow :: Continue ( ErrorOrThrottle :: Error ( error) )
508
543
} else {
509
544
error ! (
510
545
e=%error,
0 commit comments