-
Notifications
You must be signed in to change notification settings - Fork 613
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
feat(source): throttling source based on storage stats #8049
Changes from 1 commit
43ab0e4
ae34ce4
c167be8
86af573
ce3c36a
3adb86e
1d9ee28
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 |
---|---|---|
|
@@ -27,14 +27,15 @@ use risingwave_connector::source::{ | |
use risingwave_source::source_desc::{FsSourceDesc, SourceDescBuilder}; | ||
use risingwave_storage::StateStore; | ||
use tokio::sync::mpsc::UnboundedReceiver; | ||
use tokio::time::Instant; | ||
|
||
use super::executor_core::StreamSourceCore; | ||
use crate::error::StreamResult; | ||
use crate::executor::error::StreamExecutorError; | ||
use crate::executor::monitor::StreamingMetrics; | ||
use crate::executor::source::reader::SourceReaderStream; | ||
use crate::executor::throttler::SourceThrottlerImpl; | ||
use crate::executor::*; | ||
|
||
/// [`FsSourceExecutor`] is a streaming source, fir external file systems | ||
/// such as s3. | ||
pub struct FsSourceExecutor<S: StateStore> { | ||
|
@@ -55,8 +56,7 @@ pub struct FsSourceExecutor<S: StateStore> { | |
/// Receiver of barrier channel. | ||
barrier_receiver: Option<UnboundedReceiver<Barrier>>, | ||
|
||
/// Expected barrier latency | ||
expected_barrier_latency_ms: u64, | ||
throttlers: Vec<SourceThrottlerImpl>, | ||
} | ||
|
||
impl<S: StateStore> FsSourceExecutor<S> { | ||
|
@@ -68,7 +68,7 @@ impl<S: StateStore> FsSourceExecutor<S> { | |
stream_source_core: StreamSourceCore<S>, | ||
metrics: Arc<StreamingMetrics>, | ||
barrier_receiver: UnboundedReceiver<Barrier>, | ||
expected_barrier_latency_ms: u64, | ||
throttlers: Vec<SourceThrottlerImpl>, | ||
executor_id: u64, | ||
) -> StreamResult<Self> { | ||
Ok(Self { | ||
|
@@ -79,7 +79,7 @@ impl<S: StateStore> FsSourceExecutor<S> { | |
stream_source_core, | ||
metrics, | ||
barrier_receiver: Some(barrier_receiver), | ||
expected_barrier_latency_ms, | ||
throttlers, | ||
}) | ||
} | ||
|
||
|
@@ -336,18 +336,16 @@ impl<S: StateStore> FsSourceExecutor<S> { | |
|
||
yield Message::Barrier(barrier); | ||
|
||
// We allow data to flow for 5 * `expected_barrier_latency_ms` milliseconds, considering | ||
// some other latencies like network and cost in Meta. | ||
let max_wait_barrier_time_ms = self.expected_barrier_latency_ms as u128 * 5; | ||
let mut last_barrier_time = Instant::now(); | ||
let mut self_paused = false; | ||
let mut metric_row_per_barrier: u64 = 0; | ||
while let Some(msg) = stream.next().await { | ||
match msg? { | ||
// This branch will be preferred. | ||
Either::Left(barrier) => { | ||
last_barrier_time = Instant::now(); | ||
if self_paused { | ||
for throttler in &mut self.throttlers { | ||
throttler.on_barrier(); | ||
} | ||
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. why do we need multiple throttler in source exec |
||
if self_paused && self.throttlers.iter().all(|t| !t.should_pause()) { | ||
stream.resume_source(); | ||
self_paused = false; | ||
} | ||
|
@@ -386,10 +384,7 @@ impl<S: StateStore> FsSourceExecutor<S> { | |
chunk, | ||
split_offset_mapping, | ||
}) => { | ||
if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { | ||
// Exceeds the max wait barrier time, the source will be paused. Currently | ||
// we can guarantee the source is not paused since it received stream | ||
// chunks. | ||
if !self_paused && self.throttlers.iter().any(|t| t.should_pause()) { | ||
self_paused = true; | ||
stream.pause_source(); | ||
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. Please add |
||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
// Copyright 2023 RisingWave Labs | ||
// | ||
// Licensed under the Apache License, Version 2.0 (the "License"); | ||
// you may not use this file except in compliance with the License. | ||
// You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, software | ||
// distributed under the License is distributed on an "AS IS" BASIS, | ||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
// See the License for the specific language governing permissions and | ||
// limitations under the License. | ||
|
||
use risingwave_storage::StateStoreImpl; | ||
use tokio::time::Instant; | ||
|
||
pub enum SourceThrottlerImpl { | ||
MaxWaitBarrier(MaxWaitBarrierThrottler), | ||
StateStore(StateStoreImpl), | ||
} | ||
|
||
/// A constant to multiply when calculating the maximum time to wait for a barrier. This is due to | ||
/// some latencies in network and cost in meta. | ||
const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; | ||
|
||
pub struct MaxWaitBarrierThrottler { | ||
max_wait_barrier_time_ms: u128, | ||
last_barrier_time: Instant, | ||
} | ||
|
||
impl MaxWaitBarrierThrottler { | ||
pub fn new(barrier_interval_ms: u128) -> Self { | ||
Self { | ||
max_wait_barrier_time_ms: barrier_interval_ms * WAIT_BARRIER_MULTIPLE_TIMES, | ||
last_barrier_time: Instant::now(), | ||
} | ||
} | ||
|
||
fn should_pause(&self) -> bool { | ||
// We allow data to flow for `WAIT_BARRIER_MULTIPLE_TIMES` * | ||
// `expected_barrier_latency_ms` milliseconds, considering some | ||
// other latencies like network and cost in Meta. | ||
self.last_barrier_time.elapsed().as_millis() > self.max_wait_barrier_time_ms | ||
} | ||
} | ||
|
||
impl SourceThrottlerImpl { | ||
pub fn should_pause(&self) -> bool { | ||
match self { | ||
SourceThrottlerImpl::MaxWaitBarrier(inner) => inner.should_pause(), | ||
SourceThrottlerImpl::StateStore(inner) => { | ||
if let Some(hummock) = inner.as_hummock() { | ||
return hummock.need_write_throttling(); | ||
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. any hint on when hummock needs throttle? 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.
when |
||
} | ||
false | ||
} | ||
} | ||
} | ||
|
||
pub fn on_barrier(&mut self) { | ||
#[allow(clippy::single_match)] | ||
match self { | ||
SourceThrottlerImpl::MaxWaitBarrier(inner) => { | ||
inner.last_barrier_time = Instant::now(); | ||
} | ||
_ => {} | ||
} | ||
} | ||
} |
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.
Do we need to distinguish between different groups of l0 ? , I think their write paths are independent