risingwave_stream/executor/source/fetch_executor.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431
// Copyright 2024 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 std::marker::PhantomData;
use std::ops::Bound;
use either::Either;
use futures::{stream, StreamExt, TryStreamExt};
use futures_async_stream::try_stream;
use risingwave_common::catalog::{ColumnId, TableId};
use risingwave_common::hash::VnodeBitmapExt;
use risingwave_common::types::ScalarRef;
use risingwave_connector::parser::parquet_parser::get_total_row_nums_for_parquet_file;
use risingwave_connector::parser::EncodingProperties;
use risingwave_connector::source::filesystem::opendal_source::{
OpendalAzblob, OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource,
};
use risingwave_connector::source::filesystem::OpendalFsSplit;
use risingwave_connector::source::reader::desc::SourceDesc;
use risingwave_connector::source::{
BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData,
};
use risingwave_storage::store::PrefetchOptions;
use thiserror_ext::AsReport;
use super::{
apply_rate_limit, get_split_offset_col_idx, get_split_offset_mapping_from_chunk,
prune_additional_cols, SourceStateTableHandler, StreamSourceCore,
};
use crate::common::rate_limit::limited_chunk_size;
use crate::executor::prelude::*;
use crate::executor::stream_reader::StreamReaderWithPause;
const SPLIT_BATCH_SIZE: usize = 1000;
type SplitBatch = Option<Vec<SplitImpl>>;
pub struct FsFetchExecutor<S: StateStore, Src: OpendalSource> {
actor_ctx: ActorContextRef,
/// Streaming source for external
stream_source_core: Option<StreamSourceCore<S>>,
/// Upstream list executor.
upstream: Option<Executor>,
/// Rate limit in rows/s.
rate_limit_rps: Option<u32>,
_marker: PhantomData<Src>,
}
impl<S: StateStore, Src: OpendalSource> FsFetchExecutor<S, Src> {
pub fn new(
actor_ctx: ActorContextRef,
stream_source_core: StreamSourceCore<S>,
upstream: Executor,
rate_limit_rps: Option<u32>,
) -> Self {
Self {
actor_ctx,
stream_source_core: Some(stream_source_core),
upstream: Some(upstream),
rate_limit_rps,
_marker: PhantomData,
}
}
async fn replace_with_new_batch_reader<const BIASED: bool>(
splits_on_fetch: &mut usize,
state_store_handler: &SourceStateTableHandler<S>,
column_ids: Vec<ColumnId>,
source_ctx: SourceContext,
source_desc: &SourceDesc,
stream: &mut StreamReaderWithPause<BIASED, StreamChunk>,
rate_limit_rps: Option<u32>,
) -> StreamExecutorResult<()> {
let mut batch = Vec::with_capacity(SPLIT_BATCH_SIZE);
'vnodes: for vnode in state_store_handler.state_table.vnodes().iter_vnodes() {
let table_iter = state_store_handler
.state_table
.iter_with_vnode(
vnode,
&(Bound::<OwnedRow>::Unbounded, Bound::<OwnedRow>::Unbounded),
// This usage is similar with `backfill`. So we only need to fetch a large data rather than establish a connection for a whole object.
PrefetchOptions::prefetch_for_small_range_scan(),
)
.await?;
pin_mut!(table_iter);
let properties = source_desc.source.config.clone();
while let Some(item) = table_iter.next().await {
let row = item?;
let split = match row.datum_at(1) {
Some(ScalarRefImpl::Jsonb(jsonb_ref)) => match properties {
risingwave_connector::source::ConnectorProperties::Gcs(_) => {
let split: OpendalFsSplit<OpendalGcs> =
OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?;
SplitImpl::from(split)
}
risingwave_connector::source::ConnectorProperties::OpendalS3(_) => {
let split: OpendalFsSplit<OpendalS3> =
OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?;
SplitImpl::from(split)
}
risingwave_connector::source::ConnectorProperties::Azblob(_) => {
let split: OpendalFsSplit<OpendalAzblob> =
OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?;
SplitImpl::from(split)
}
risingwave_connector::source::ConnectorProperties::PosixFs(_) => {
let split: OpendalFsSplit<OpendalPosixFs> =
OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?;
SplitImpl::from(split)
}
_ => unreachable!(),
},
_ => unreachable!(),
};
batch.push(split);
if batch.len() >= SPLIT_BATCH_SIZE {
break 'vnodes;
}
}
}
if batch.is_empty() {
stream.replace_data_stream(stream::pending().boxed());
} else {
*splits_on_fetch += batch.len();
let batch_reader = Self::build_batched_stream_reader(
column_ids,
source_ctx,
source_desc,
Some(batch),
rate_limit_rps,
)
.await?
.map_err(StreamExecutorError::connector_error);
stream.replace_data_stream(batch_reader);
}
Ok(())
}
async fn build_batched_stream_reader(
column_ids: Vec<ColumnId>,
source_ctx: SourceContext,
source_desc: &SourceDesc,
batch: SplitBatch,
rate_limit_rps: Option<u32>,
) -> StreamExecutorResult<BoxChunkSourceStream> {
let (stream, _) = source_desc
.source
.build_stream(batch, column_ids, Arc::new(source_ctx), false)
.await
.map_err(StreamExecutorError::connector_error)?;
Ok(apply_rate_limit(stream, rate_limit_rps).boxed())
}
fn build_source_ctx(
&self,
source_desc: &SourceDesc,
source_id: TableId,
source_name: &str,
) -> SourceContext {
SourceContext::new(
self.actor_ctx.id,
source_id,
self.actor_ctx.fragment_id,
source_name.to_owned(),
source_desc.metrics.clone(),
SourceCtrlOpts {
chunk_size: limited_chunk_size(self.rate_limit_rps),
rate_limit: self.rate_limit_rps,
},
source_desc.source.config.clone(),
None,
)
}
#[try_stream(ok = Message, error = StreamExecutorError)]
async fn into_stream(mut self) {
let mut upstream = self.upstream.take().unwrap().execute();
let barrier = expect_first_barrier(&mut upstream).await?;
let first_epoch = barrier.epoch;
let is_pause_on_startup = barrier.is_pause_on_startup();
yield Message::Barrier(barrier);
let mut core = self.stream_source_core.take().unwrap();
let mut state_store_handler = core.split_state_store;
// Build source description from the builder.
let source_desc_builder = core.source_desc_builder.take().unwrap();
let source_desc = source_desc_builder
.build()
.map_err(StreamExecutorError::connector_error)?;
let (Some(split_idx), Some(offset_idx)) = get_split_offset_col_idx(&source_desc.columns)
else {
unreachable!("Partition and offset columns must be set.");
};
// Initialize state table.
state_store_handler.init_epoch(first_epoch).await?;
let mut splits_on_fetch: usize = 0;
let mut stream =
StreamReaderWithPause::<true, StreamChunk>::new(upstream, stream::pending().boxed());
if is_pause_on_startup {
stream.pause_stream();
}
// If it is a recovery startup,
// there can be file assignments in the state table.
// Hence we try building a reader first.
Self::replace_with_new_batch_reader(
&mut splits_on_fetch,
&state_store_handler, // move into the function
core.column_ids.clone(),
self.build_source_ctx(&source_desc, core.source_id, &core.source_name),
&source_desc,
&mut stream,
self.rate_limit_rps,
)
.await?;
while let Some(msg) = stream.next().await {
match msg {
Err(e) => {
tracing::error!(error = %e.as_report(), "Fetch Error");
splits_on_fetch = 0;
}
Ok(msg) => {
match msg {
// This branch will be preferred.
Either::Left(msg) => {
match &msg {
Message::Barrier(barrier) => {
let mut need_rebuild_reader = false;
if let Some(mutation) = barrier.mutation.as_deref() {
match mutation {
Mutation::Pause => stream.pause_stream(),
Mutation::Resume => stream.resume_stream(),
Mutation::Throttle(actor_to_apply) => {
if let Some(new_rate_limit) =
actor_to_apply.get(&self.actor_ctx.id)
&& *new_rate_limit != self.rate_limit_rps
{
tracing::debug!(
"updating rate limit from {:?} to {:?}",
self.rate_limit_rps,
*new_rate_limit
);
self.rate_limit_rps = *new_rate_limit;
need_rebuild_reader = true;
}
}
_ => (),
}
}
state_store_handler
.state_table
.commit(barrier.epoch)
.await?;
if let Some(vnode_bitmap) =
barrier.as_update_vnode_bitmap(self.actor_ctx.id)
{
// if _cache_may_stale, we must rebuild the stream to adjust vnode mappings
let (_prev_vnode_bitmap, cache_may_stale) =
state_store_handler
.state_table
.update_vnode_bitmap(vnode_bitmap);
if cache_may_stale {
splits_on_fetch = 0;
}
}
if splits_on_fetch == 0 || need_rebuild_reader {
Self::replace_with_new_batch_reader(
&mut splits_on_fetch,
&state_store_handler,
core.column_ids.clone(),
self.build_source_ctx(
&source_desc,
core.source_id,
&core.source_name,
),
&source_desc,
&mut stream,
self.rate_limit_rps,
)
.await?;
}
// Propagate the barrier.
yield msg;
}
// Receiving file assignments from upstream list executor,
// store into state table.
Message::Chunk(chunk) => {
// For Parquet encoding, the offset indicates the current row being read.
// Therefore, to determine if the end of a Parquet file has been reached, we need to compare its offset with the total number of rows.
// We directly obtain the total row count and set the size in `OpendalFsSplit` to this value.
let file_assignment = if let EncodingProperties::Parquet =
source_desc.source.parser_config.encoding_config
{
let filename_list: Vec<_> = chunk
.data_chunk()
.rows()
.map(|row| {
let filename = row.datum_at(0).unwrap().into_utf8();
filename.to_string()
})
.collect();
let mut parquet_file_assignment = vec![];
for filename in &filename_list {
let total_row_num =
get_total_row_nums_for_parquet_file(
filename,
source_desc.clone(),
)
.await?;
parquet_file_assignment.push(
OpendalFsSplit::<Src>::new(
filename.to_owned(),
0,
total_row_num - 1, // -1 because offset start from 0.
),
)
}
parquet_file_assignment
} else {
chunk
.data_chunk()
.rows()
.map(|row| {
let filename = row.datum_at(0).unwrap().into_utf8();
let size = row.datum_at(2).unwrap().into_int64();
OpendalFsSplit::<Src>::new(
filename.to_owned(),
0,
size as usize,
)
})
.collect()
};
state_store_handler.set_states(file_assignment).await?;
state_store_handler.state_table.try_flush().await?;
}
_ => unreachable!(),
}
}
// StreamChunk from FsSourceReader, and the reader reads only one file.
Either::Right(chunk) => {
let mapping =
get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx)
.unwrap();
debug_assert_eq!(mapping.len(), 1);
if let Some((split_id, offset)) = mapping.into_iter().next() {
let row = state_store_handler
.get(split_id.clone())
.await?
.expect("The fs_split should be in the state table.");
let fs_split = match row.datum_at(1) {
Some(ScalarRefImpl::Jsonb(jsonb_ref)) => {
OpendalFsSplit::<Src>::restore_from_json(
jsonb_ref.to_owned_scalar(),
)?
}
_ => unreachable!(),
};
if offset.parse::<usize>().unwrap() >= fs_split.size {
splits_on_fetch -= 1;
state_store_handler.delete(split_id).await?;
} else {
state_store_handler
.set(split_id, fs_split.encode_to_json())
.await?;
}
}
let chunk = prune_additional_cols(
&chunk,
split_idx,
offset_idx,
&source_desc.columns,
);
yield Message::Chunk(chunk);
}
}
}
}
}
}
}
impl<S: StateStore, Src: OpendalSource> Execute for FsFetchExecutor<S, Src> {
fn execute(self: Box<Self>) -> BoxedMessageStream {
self.into_stream().boxed()
}
}
impl<S: StateStore, Src: OpendalSource> Debug for FsFetchExecutor<S, Src> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
if let Some(core) = &self.stream_source_core {
f.debug_struct("FsFetchExecutor")
.field("source_id", &core.source_id)
.field("column_ids", &core.column_ids)
.finish()
} else {
f.debug_struct("FsFetchExecutor").finish()
}
}
}