risingwave_connector/source/iceberg/
parquet_file_handler.rs1use std::collections::HashMap;
16use std::future::IntoFuture;
17use std::ops::Range;
18use std::pin::Pin;
19use std::sync::Arc;
20
21use anyhow::{Context, bail};
22use bytes::Bytes;
23use futures::future::BoxFuture;
24use futures::{FutureExt, Stream, TryFutureExt};
25use iceberg::io::{
26 FileIOBuilder, FileMetadata, FileRead, S3_ACCESS_KEY_ID, S3_REGION, S3_SECRET_ACCESS_KEY,
27};
28use itertools::Itertools;
29use opendal::Operator;
30use opendal::layers::{LoggingLayer, RetryLayer};
31use opendal::services::{Azblob, Gcs, S3};
32use parquet::arrow::async_reader::AsyncFileReader;
33use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask, parquet_to_arrow_schema};
34use parquet::file::metadata::{FileMetaData, ParquetMetaData, ParquetMetaDataReader};
35use prometheus::core::GenericCounter;
36use risingwave_common::array::StreamChunk;
37use risingwave_common::array::arrow::{IcebergArrowConvert, is_parquet_schema_match_source_schema};
38use risingwave_common::catalog::{ColumnDesc, ColumnId};
39use risingwave_common::metrics::LabelGuardedMetric;
40use risingwave_common::util::tokio_util::compat::FuturesAsyncReadCompatExt;
41use url::Url;
42
43use crate::error::ConnectorResult;
44use crate::parser::ParquetParser;
45use crate::source::{Column, SourceColumnDesc};
46
47pub struct ParquetFileReader<R: FileRead> {
48 meta: FileMetadata,
49 r: R,
50}
51
52impl<R: FileRead> ParquetFileReader<R> {
53 pub fn new(meta: FileMetadata, r: R) -> Self {
54 Self { meta, r }
55 }
56}
57
58impl<R: FileRead> AsyncFileReader for ParquetFileReader<R> {
59 fn get_bytes(&mut self, range: Range<u64>) -> BoxFuture<'_, parquet::errors::Result<Bytes>> {
60 Box::pin(
61 self.r
62 .read(range.start as _..range.end as _)
63 .map_err(|err| parquet::errors::ParquetError::External(Box::new(err))),
64 )
65 }
66
67 fn get_metadata(
68 &mut self,
69 _options: Option<&parquet::arrow::arrow_reader::ArrowReaderOptions>,
70 ) -> BoxFuture<'_, parquet::errors::Result<Arc<ParquetMetaData>>> {
71 async move {
72 let reader = ParquetMetaDataReader::new();
73 let size = self.meta.size;
74 let meta = reader.load_and_finish(self, size).await?;
75
76 Ok(Arc::new(meta))
77 }
78 .boxed()
79 }
80}
81
82pub async fn create_parquet_stream_builder(
83 s3_region: String,
84 s3_access_key: String,
85 s3_secret_key: String,
86 location: String,
87) -> Result<ParquetRecordBatchStreamBuilder<ParquetFileReader<impl FileRead>>, anyhow::Error> {
88 let mut props = HashMap::new();
89 props.insert(S3_REGION, s3_region.clone());
90 props.insert(S3_ACCESS_KEY_ID, s3_access_key.clone());
91 props.insert(S3_SECRET_ACCESS_KEY, s3_secret_key.clone());
92
93 let file_io_builder = FileIOBuilder::new("s3");
94 let file_io = file_io_builder.with_props(props.into_iter()).build()?;
95 let parquet_file = file_io.new_input(&location)?;
96
97 let parquet_metadata = parquet_file.metadata().await?;
98 let parquet_reader = parquet_file.reader().await?;
99 let parquet_file_reader = ParquetFileReader::new(parquet_metadata, parquet_reader);
100
101 ParquetRecordBatchStreamBuilder::new(parquet_file_reader)
102 .await
103 .map_err(Into::into)
104}
105
106pub fn new_s3_operator(
107 s3_region: String,
108 s3_access_key: String,
109 s3_secret_key: String,
110 bucket: String,
111 s3_endpoint: String,
112) -> ConnectorResult<Operator> {
113 let mut builder = S3::default();
114 builder = builder
115 .region(&s3_region)
116 .endpoint(&s3_endpoint)
117 .access_key_id(&s3_access_key)
118 .secret_access_key(&s3_secret_key)
119 .bucket(&bucket)
120 .disable_config_load();
121 let op: Operator = Operator::new(builder)?
122 .layer(LoggingLayer::default())
123 .layer(RetryLayer::default())
124 .finish();
125
126 Ok(op)
127}
128
129pub fn new_gcs_operator(credential: String, bucket: String) -> ConnectorResult<Operator> {
130 let builder = Gcs::default().bucket(&bucket).credential(&credential);
132
133 let operator: Operator = Operator::new(builder)?
134 .layer(LoggingLayer::default())
135 .layer(RetryLayer::default())
136 .finish();
137 Ok(operator)
138}
139
140pub fn new_azblob_operator(
141 endpoint: String,
142 account_name: String,
143 account_key: String,
144 container_name: String,
145) -> ConnectorResult<Operator> {
146 let mut builder = Azblob::default();
148 builder = builder
149 .container(&container_name)
150 .endpoint(&endpoint)
151 .account_name(&account_name)
152 .account_key(&account_key);
153
154 let operator: Operator = Operator::new(builder)?
155 .layer(LoggingLayer::default())
156 .layer(RetryLayer::default())
157 .finish();
158 Ok(operator)
159}
160
161#[derive(Debug, Clone)]
162pub enum FileScanBackend {
163 S3,
164 Gcs,
165 Azblob,
166}
167
168pub fn extract_bucket_and_file_name(
169 location: &str,
170 file_scan_backend: &FileScanBackend,
171) -> ConnectorResult<(String, String)> {
172 let url = Url::parse(location)?;
173 let bucket = url
174 .host_str()
175 .with_context(|| format!("Invalid url: {}, missing bucket", location))?
176 .to_owned();
177 let prefix = match file_scan_backend {
178 FileScanBackend::S3 => format!("s3://{}/", bucket),
179 FileScanBackend::Gcs => format!("gcs://{}/", bucket),
180 FileScanBackend::Azblob => format!("azblob://{}/", bucket),
181 };
182 let file_name = location[prefix.len()..].to_string();
183 Ok((bucket, file_name))
184}
185
186pub async fn list_data_directory(
187 op: Operator,
188 dir: String,
189 file_scan_backend: &FileScanBackend,
190) -> Result<Vec<String>, anyhow::Error> {
191 let (bucket, file_name) = extract_bucket_and_file_name(&dir, file_scan_backend)?;
192 let prefix = match file_scan_backend {
193 FileScanBackend::S3 => format!("s3://{}/", bucket),
194 FileScanBackend::Gcs => format!("gcs://{}/", bucket),
195 FileScanBackend::Azblob => format!("azblob://{}/", bucket),
196 };
197 if dir.starts_with(&prefix) {
198 op.list(&file_name).await.map_err(Into::into).map(|list| {
199 list.into_iter()
200 .map(|entry| prefix.clone() + entry.path())
201 .collect()
202 })
203 } else {
204 bail!("Invalid url: {}, should start with {}", dir, prefix)
205 }
206}
207
208pub fn get_project_mask(
225 columns: Option<Vec<Column>>,
226 metadata: &FileMetaData,
227) -> ConnectorResult<ProjectionMask> {
228 match columns {
229 Some(rw_columns) => {
230 let root_column_names = metadata
231 .schema_descr()
232 .root_schema()
233 .get_fields()
234 .iter()
235 .map(|field| field.name())
236 .collect_vec();
237
238 let converted_arrow_schema =
239 parquet_to_arrow_schema(metadata.schema_descr(), metadata.key_value_metadata())
240 .map_err(anyhow::Error::from)?;
241 let valid_column_indices: Vec<usize> = rw_columns
242 .iter()
243 .filter_map(|column| {
244 root_column_names
245 .iter()
246 .position(|&name| name == column.name)
247 .and_then(|pos| {
248 let arrow_data_type = converted_arrow_schema
249 .field_with_name(&column.name)
250 .ok()?
251 .data_type();
252 let rw_data_type: &risingwave_common::types::DataType =
253 &column.data_type;
254 if is_parquet_schema_match_source_schema(arrow_data_type, rw_data_type)
255 {
256 Some(pos)
257 } else {
258 None
259 }
260 })
261 })
262 .collect();
263
264 Ok(ProjectionMask::roots(
265 metadata.schema_descr(),
266 valid_column_indices,
267 ))
268 }
269 None => Ok(ProjectionMask::all()),
270 }
271}
272
273pub async fn read_parquet_file(
275 op: Operator,
276 file_name: String,
277 rw_columns: Option<Vec<Column>>,
278 parser_columns: Option<Vec<SourceColumnDesc>>,
279 batch_size: usize,
280 offset: usize,
281 file_source_input_row_count_metrics: Option<
282 LabelGuardedMetric<GenericCounter<prometheus::core::AtomicU64>>,
283 >,
284 parquet_source_skip_row_count_metrics: Option<
285 LabelGuardedMetric<GenericCounter<prometheus::core::AtomicU64>>,
286 >,
287) -> ConnectorResult<
288 Pin<Box<dyn Stream<Item = Result<StreamChunk, crate::error::ConnectorError>> + Send>>,
289> {
290 let mut reader: tokio_util::compat::Compat<opendal::FuturesAsyncReader> = op
291 .reader_with(&file_name)
292 .into_future() .await?
294 .into_futures_async_read(..)
295 .await?
296 .compat();
297 let parquet_metadata = reader
298 .get_metadata(None)
299 .await
300 .map_err(anyhow::Error::from)?;
301
302 let file_metadata = parquet_metadata.file_metadata();
303 {
304 tracing::info!(
306 "Reading parquet file: {}, from offset {}, num_row_groups={}, total_rows={}, kv_len={}",
307 file_name,
308 offset,
309 parquet_metadata.row_groups().len(),
310 file_metadata.num_rows(),
311 file_metadata
312 .key_value_metadata()
313 .map(|m| m.len())
314 .unwrap_or(0)
315 );
316 let schema_descr = file_metadata.schema_descr();
318 for col in schema_descr.columns() {
319 let path = col.path().string();
320 let physical = col.physical_type();
321 let logical = col.logical_type();
322 tracing::debug!(
323 file = %file_name,
324 column_path = path,
325 physical_type = ?physical,
326 logical_type = ?logical,
327 type_length = ?col.type_length(),
328 max_def_level = col.max_def_level(),
329 max_rep_level = col.max_rep_level(),
330 "Parquet file column schema: "
331 );
332 }
333 }
334 let projection_mask = get_project_mask(rw_columns, file_metadata)?;
335
336 let record_batch_stream = ParquetRecordBatchStreamBuilder::new(reader)
339 .await?
340 .with_batch_size(batch_size)
341 .with_projection(projection_mask)
342 .with_offset(offset)
343 .build()?;
344 let converted_arrow_schema = parquet_to_arrow_schema(
345 file_metadata.schema_descr(),
346 file_metadata.key_value_metadata(),
347 )?;
348 let columns = match parser_columns {
349 Some(columns) => columns,
350 None => converted_arrow_schema
351 .fields
352 .iter()
353 .enumerate()
354 .map(|(index, field_ref)| {
355 let data_type = IcebergArrowConvert.type_from_field(field_ref).unwrap();
356 let column_desc = ColumnDesc::named(
357 field_ref.name().clone(),
358 ColumnId::new(index as i32),
359 data_type,
360 );
361 SourceColumnDesc::from(&column_desc)
362 })
363 .collect(),
364 };
365 let parquet_parser = ParquetParser::new(columns, file_name, offset)?;
366 let msg_stream: Pin<
367 Box<dyn Stream<Item = Result<StreamChunk, crate::error::ConnectorError>> + Send>,
368 > = parquet_parser.into_stream(
369 record_batch_stream,
370 file_source_input_row_count_metrics,
371 parquet_source_skip_row_count_metrics,
372 );
373 Ok(msg_stream)
374}
375
376pub async fn get_parquet_fields(
377 op: Operator,
378 file_name: String,
379) -> ConnectorResult<risingwave_common::array::arrow::arrow_schema_iceberg::Fields> {
380 let mut reader: tokio_util::compat::Compat<opendal::FuturesAsyncReader> = op
381 .reader_with(&file_name)
382 .into_future() .await?
384 .into_futures_async_read(..)
385 .await?
386 .compat();
387 let parquet_metadata = reader.get_metadata(None).await?;
388
389 let file_metadata = parquet_metadata.file_metadata();
390 let converted_arrow_schema = parquet_to_arrow_schema(
391 file_metadata.schema_descr(),
392 file_metadata.key_value_metadata(),
393 )?;
394 let fields: risingwave_common::array::arrow::arrow_schema_iceberg::Fields =
395 converted_arrow_schema.fields;
396 Ok(fields)
397}