Skip to content
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

fix: list one page in fs source executor #20292

Merged
merged 3 commits into from
Jan 24, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ use anyhow::anyhow;
use async_trait::async_trait;
use chrono::{DateTime, Utc};
use futures::stream::{self, BoxStream};
use futures::StreamExt;
use futures::{StreamExt, TryStreamExt};
use opendal::{Metakey, Operator};
use risingwave_common::types::Timestamptz;

Expand Down Expand Up @@ -54,7 +54,9 @@ impl<Src: OpendalSource> SplitEnumerator for OpendalEnumerator<Src> {
let empty_split: OpendalFsSplit<Src> = OpendalFsSplit::empty_split();
let prefix = self.prefix.as_deref().unwrap_or("/");

match self.op.list(prefix).await {
let mut lister = self.op.lister(prefix).await?;
// fetch one item as validation, no need to get all
match lister.try_next().await {
Ok(_) => return Ok(vec![empty_split]),
Err(e) => {
return Err(anyhow!(e)
Expand Down
17 changes: 8 additions & 9 deletions src/connector/src/source/filesystem/s3/enumerator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -99,15 +99,14 @@ impl SplitEnumerator for S3SplitEnumerator {
}

async fn list_splits(&mut self) -> crate::error::ConnectorResult<Vec<Self::Split>> {
let mut objects = Vec::new();
loop {
let (files, has_finished) = self.get_next_page::<FsSplit>().await?;
objects.extend(files);
if has_finished {
break;
}
}
Ok(objects)
// fetch one page as validation, no need to get all pages
let (_, _) = self.get_next_page::<FsSplit>().await?;

Ok(vec![FsSplit {
name: "empty_split".to_owned(),
offset: 0,
size: 0,
}])
}
Comment on lines 101 to 110
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this correct for legacy s3 enumerator? IIUC its splits need to be enumerated in meta 🤔

}

Expand Down
Loading