Skip to content

Commit 1b01bc0

Browse files
Merge pull request #187 from influxdata/crepererum/issue62b
fix: (potential) broker cache invalidation race
2 parents 821c23b + 4e765e8 commit 1b01bc0

File tree

5 files changed

+206
-91
lines changed

5 files changed

+206
-91
lines changed

src/backoff.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ pub type BackoffResult<T> = Result<T, BackoffError>;
3333
#[derive(Debug)]
3434
pub enum ErrorOrThrottle<E>
3535
where
36-
E: std::error::Error + Send,
36+
E: Send,
3737
{
3838
Error(E),
3939
Throttle(Duration),

src/client/controller.rs

Lines changed: 60 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -2,13 +2,14 @@ use async_trait::async_trait;
22
use std::ops::ControlFlow;
33
use std::sync::Arc;
44
use tokio::sync::Mutex;
5-
use tracing::{error, info};
5+
use tracing::{debug, error, info};
66

77
use crate::{
88
backoff::{Backoff, BackoffConfig, ErrorOrThrottle},
99
client::{Error, Result},
1010
connection::{
11-
BrokerCache, BrokerConnection, BrokerConnector, MessengerTransport, MetadataLookupMode,
11+
BrokerCache, BrokerCacheGeneration, BrokerConnection, BrokerConnector, MessengerTransport,
12+
MetadataLookupMode,
1213
},
1314
messenger::RequestError,
1415
protocol::{
@@ -29,15 +30,15 @@ pub struct ControllerClient {
2930
backoff_config: BackoffConfig,
3031

3132
/// Current broker connection if any
32-
current_broker: Mutex<Option<BrokerConnection>>,
33+
current_broker: Mutex<(Option<BrokerConnection>, BrokerCacheGeneration)>,
3334
}
3435

3536
impl ControllerClient {
3637
pub(super) fn new(brokers: Arc<BrokerConnector>) -> Self {
3738
Self {
3839
brokers,
3940
backoff_config: Default::default(),
40-
current_broker: Mutex::new(None),
41+
current_broker: Mutex::new((None, BrokerCacheGeneration::START)),
4142
}
4243
}
4344

@@ -64,28 +65,34 @@ impl ControllerClient {
6465
};
6566

6667
maybe_retry(&self.backoff_config, self, "create_topic", || async move {
67-
let broker = self.get().await.map_err(ErrorOrThrottle::Error)?;
68+
let (broker, gen) = self
69+
.get()
70+
.await
71+
.map_err(|e| ErrorOrThrottle::Error((e, None)))?;
6872
let response = broker
6973
.request(request)
7074
.await
71-
.map_err(|e| ErrorOrThrottle::Error(e.into()))?;
75+
.map_err(|e| ErrorOrThrottle::Error((e.into(), Some(gen))))?;
7276

7377
maybe_throttle(response.throttle_time_ms)?;
7478

7579
let topic = response
7680
.topics
7781
.exactly_one()
78-
.map_err(|e| ErrorOrThrottle::Error(Error::exactly_one_topic(e)))?;
82+
.map_err(|e| ErrorOrThrottle::Error((Error::exactly_one_topic(e), Some(gen))))?;
7983

8084
match topic.error {
8185
None => Ok(()),
82-
Some(protocol_error) => Err(ErrorOrThrottle::Error(Error::ServerError {
83-
protocol_error,
84-
error_message: topic.error_message.and_then(|s| s.0),
85-
request: RequestContext::Topic(topic.name.0),
86-
response: None,
87-
is_virtual: false,
88-
})),
86+
Some(protocol_error) => Err(ErrorOrThrottle::Error((
87+
Error::ServerError {
88+
protocol_error,
89+
error_message: topic.error_message.and_then(|s| s.0),
90+
request: RequestContext::Topic(topic.name.0),
91+
response: None,
92+
is_virtual: false,
93+
},
94+
Some(gen),
95+
))),
8996
}
9097
})
9198
.await?;
@@ -119,10 +126,10 @@ impl BrokerCache for &ControllerClient {
119126
type R = MessengerTransport;
120127
type E = Error;
121128

122-
async fn get(&self) -> Result<Arc<Self::R>> {
129+
async fn get(&self) -> Result<(Arc<Self::R>, BrokerCacheGeneration)> {
123130
let mut current_broker = self.current_broker.lock().await;
124-
if let Some(broker) = &*current_broker {
125-
return Ok(Arc::clone(broker));
131+
if let Some(broker) = &current_broker.0 {
132+
return Ok((Arc::clone(broker), current_broker.1));
126133
}
127134

128135
info!("Creating new controller broker connection",);
@@ -135,13 +142,28 @@ impl BrokerCache for &ControllerClient {
135142
))
136143
})?;
137144

138-
*current_broker = Some(Arc::clone(&broker));
139-
Ok(broker)
145+
current_broker.0 = Some(Arc::clone(&broker));
146+
current_broker.1.bump();
147+
148+
Ok((broker, current_broker.1))
140149
}
141150

142-
async fn invalidate(&self, reason: &'static str) {
151+
async fn invalidate(&self, reason: &'static str, gen: BrokerCacheGeneration) {
152+
let mut guard = self.current_broker.lock().await;
153+
154+
if guard.1 != gen {
155+
// stale request
156+
debug!(
157+
reason,
158+
current_gen = guard.1.get(),
159+
request_gen = gen.get(),
160+
"stale invalidation request for arbitrary broker cache",
161+
);
162+
return;
163+
}
164+
143165
info!(reason, "Invalidating cached controller broker",);
144-
self.current_broker.lock().await.take();
166+
guard.0.take();
145167
}
146168
}
147169

@@ -156,13 +178,15 @@ async fn maybe_retry<B, R, F, T>(
156178
where
157179
B: BrokerCache,
158180
R: (Fn() -> F) + Send + Sync,
159-
F: std::future::Future<Output = Result<T, ErrorOrThrottle<Error>>> + Send,
181+
F: std::future::Future<
182+
Output = Result<T, ErrorOrThrottle<(Error, Option<BrokerCacheGeneration>)>>,
183+
> + Send,
160184
{
161185
let mut backoff = Backoff::new(backoff_config);
162186

163187
backoff
164188
.retry_with_backoff(request_name, || async {
165-
let error = match f().await {
189+
let (error, cache_gen) = match f().await {
166190
Ok(v) => {
167191
return ControlFlow::Break(Ok(v));
168192
}
@@ -176,19 +200,26 @@ where
176200
// broken connection
177201
Error::Request(RequestError::Poisoned(_) | RequestError::IO(_))
178202
| Error::Connection(_) => {
179-
broker_cache
180-
.invalidate("controller client: connection broken")
181-
.await
203+
if let Some(cache_gen) = cache_gen {
204+
broker_cache
205+
.invalidate("controller client: connection broken", cache_gen)
206+
.await
207+
}
182208
}
183209

184210
// our broker is actually not the controller
185211
Error::ServerError {
186212
protocol_error: ProtocolError::NotController,
187213
..
188214
} => {
189-
broker_cache
190-
.invalidate("controller client: server error: not controller")
191-
.await;
215+
if let Some(cache_gen) = cache_gen {
216+
broker_cache
217+
.invalidate(
218+
"controller client: server error: not controller",
219+
cache_gen,
220+
)
221+
.await;
222+
}
192223
}
193224

194225
// fatal

0 commit comments

Comments
 (0)