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

[FEAT] GlobScanOperator #1550

Merged
merged 6 commits into from
Oct 31, 2023
Merged
Show file tree
Hide file tree
Changes from 5 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
4 changes: 4 additions & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions daft/daft.pyi
Original file line number Diff line number Diff line change
Expand Up @@ -848,4 +848,5 @@ def io_glob(
io_config: IOConfig | None = None,
fanout_limit: int | None = None,
page_size: int | None = None,
limit: int | None = None,
) -> list[dict]: ...
2 changes: 2 additions & 0 deletions src/daft-io/src/azure_blob.rs
Original file line number Diff line number Diff line change
Expand Up @@ -473,6 +473,7 @@ impl ObjectSource for AzureBlobSource {
glob_path: &str,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<super::Result<FileMetadata>>> {
use crate::object_store_glob::glob;
Expand All @@ -485,6 +486,7 @@ impl ObjectSource for AzureBlobSource {
glob_path,
fanout_limit,
page_size.or(Some(1000)),
limit,
io_stats,
)
.await
Expand Down
2 changes: 2 additions & 0 deletions src/daft-io/src/google_cloud.rs
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,7 @@ impl ObjectSource for GCSSource {
glob_path: &str,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<super::Result<FileMetadata>>> {
use crate::object_store_glob::glob;
Expand All @@ -419,6 +420,7 @@ impl ObjectSource for GCSSource {
glob_path,
fanout_limit,
page_size.or(Some(1000)),
limit,
io_stats,
)
.await
Expand Down
3 changes: 2 additions & 1 deletion src/daft-io/src/http.rs
Original file line number Diff line number Diff line change
Expand Up @@ -250,6 +250,7 @@ impl ObjectSource for HttpSource {
glob_path: &str,
_fanout_limit: Option<usize>,
_page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<super::Result<FileMetadata>>> {
use crate::object_store_glob::glob;
Expand All @@ -258,7 +259,7 @@ impl ObjectSource for HttpSource {
let fanout_limit = None;
let page_size = None;

glob(self, glob_path, fanout_limit, page_size, io_stats).await
glob(self, glob_path, fanout_limit, page_size, limit, io_stats).await
}

async fn ls(
Expand Down
19 changes: 19 additions & 0 deletions src/daft-io/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ use lazy_static::lazy_static;
pub mod python;

pub use common_io_config::{AzureConfig, IOConfig, S3Config};
use object_io::FileMetadata;
pub use object_io::GetResult;
#[cfg(feature = "python")]
pub use python::register_modules;
Expand Down Expand Up @@ -165,6 +166,24 @@ impl IOClient {
Ok(new_source)
}

pub async fn glob(
&self,
input: &str,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<Arc<IOStatsContext>>,
) -> Result<Vec<FileMetadata>> {
let (scheme, _) = parse_url(input)?;
let source = self.get_source(&scheme).await?;
let files: Vec<FileMetadata> = source
.glob(input, fanout_limit, page_size, limit, io_stats)
.await?
.try_collect()
.await?;
Ok(files)
}

pub async fn single_url_get(
&self,
input: String,
Expand Down
13 changes: 11 additions & 2 deletions src/daft-io/src/local.rs
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@ impl ObjectSource for LocalSource {
glob_path: &str,
_fanout_limit: Option<usize>,
_page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<super::Result<FileMetadata>>> {
use crate::object_store_glob::glob;
Expand All @@ -160,10 +161,18 @@ impl ObjectSource for LocalSource {
#[cfg(target_env = "msvc")]
{
let glob_path = glob_path.replace("\\", "/");
return glob(self, glob_path.as_str(), fanout_limit, page_size, io_stats).await;
return glob(
self,
glob_path.as_str(),
fanout_limit,
page_size,
limit,
io_stats,
)
.await;
}

glob(self, glob_path, fanout_limit, page_size, io_stats).await
glob(self, glob_path, fanout_limit, page_size, limit, io_stats).await
}

async fn ls(
Expand Down
1 change: 1 addition & 0 deletions src/daft-io/src/object_io.rs
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,7 @@ pub(crate) trait ObjectSource: Sync + Send {
glob_path: &str,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<super::Result<FileMetadata>>>;

Expand Down
59 changes: 40 additions & 19 deletions src/daft-io/src/object_store_glob.rs
Original file line number Diff line number Diff line change
Expand Up @@ -320,18 +320,21 @@ pub(crate) async fn glob(
glob: &str,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<'static, super::Result<FileMetadata>>> {
// If no special characters, we fall back to ls behavior
let full_fragment = GlobFragment::new(glob);
if !full_fragment.has_special_character() {
let mut remaining_results = limit;
let glob = full_fragment.escaped_str().to_string();
return Ok(stream! {
let mut results = source.iter_dir(glob.as_str(), true, page_size, io_stats).await?;
while let Some(result) = results.next().await {
while let Some(result) = results.next().await && remaining_results.map(|rr| rr > 0).unwrap_or(true) {
match result {
Ok(fm) => {
if matches!(fm.filetype, FileType::File) {
remaining_results = remaining_results.map(|rr| rr - 1);
yield Ok(fm)
}
},
Expand Down Expand Up @@ -362,7 +365,7 @@ pub(crate) async fn glob(
.compile_matcher();

// Channel to send results back to caller. Note that all results must have FileType::File.
let (to_rtn_tx, mut to_rtn_rx) = tokio::sync::mpsc::channel(16 * 1024);
let (to_rtn_tx, mut to_rtn_rx) = tokio::sync::mpsc::channel(limit.unwrap_or(16 * 1024));

/// Dispatches a task to visit the specified `path` (a concrete path on the filesystem to either a File or Directory).
/// Based on the current glob_fragment being processed (accessible via `glob_fragments[i]`) this task will:
Expand Down Expand Up @@ -419,16 +422,20 @@ pub(crate) async fn glob(
FileType::File
if state.full_glob_matcher.is_match(fm.filepath.as_str()) =>
{
result_tx.send(Ok(fm)).await.expect("Internal multithreading channel is broken: results may be incorrect");
if let Some(e) = result_tx.send(Ok(fm)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
};
}
_ => (),
}
}
// Silence NotFound errors when in wildcard "search" mode
Err(super::Error::NotFound { .. }) if state.wildcard_mode => (),
Err(e) => result_tx.send(Err(e)).await.expect(
"Internal multithreading channel is broken: results may be incorrect",
),
Err(e) => {
if let Some(e) = result_tx.send(Err(e)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
}
}
// BASE CASE: current fragment is the last fragment in `glob_fragments`
Expand All @@ -446,14 +453,18 @@ pub(crate) async fn glob(
if matches!(fm.filetype, FileType::File)
&& state.full_glob_matcher.is_match(fm.filepath.as_str())
{
result_tx.send(Ok(fm)).await.expect("Internal multithreading channel is broken: results may be incorrect");
if let Some(e) = result_tx.send(Ok(fm)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
}
// Silence NotFound errors when in wildcard "search" mode
Err(super::Error::NotFound { .. }) if state.wildcard_mode => (),
Err(e) => result_tx.send(Err(e)).await.expect(
"Internal multithreading channel is broken: results may be incorrect",
),
Err(e) => {
if let Some(e) = result_tx.send(Err(e)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
}
}
// Last fragment does not contain wildcard: we return it if the full path exists and is a FileType::File
Expand All @@ -474,14 +485,18 @@ pub(crate) async fn glob(
&& matches!(single_file_ls.files[0].filetype, FileType::File)
{
let fm = single_file_ls.files.drain(..).next().unwrap();
result_tx.send(Ok(fm)).await.expect("Internal multithreading channel is broken: results may be incorrect");
if let Some(e) = result_tx.send(Ok(fm)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
}
// Silence NotFound errors when in wildcard "search" mode
Err(super::Error::NotFound { .. }) if state.wildcard_mode => (),
Err(e) => result_tx.send(Err(e)).await.expect(
"Internal multithreading channel is broken: results may be incorrect",
),
Err(e) => {
if let Some(e) = result_tx.send(Err(e)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
};
}

Expand Down Expand Up @@ -535,15 +550,19 @@ pub(crate) async fn glob(
FileType::File
if state.full_glob_matcher.is_match(fm.filepath.as_str()) =>
{
result_tx.send(Ok(fm)).await.expect("Internal multithreading channel is broken: results may be incorrect");
if let Some(e) = result_tx.send(Ok(fm)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
_ => (),
},
// Always silence NotFound since we are in wildcard "search" mode here by definition
Err(super::Error::NotFound { .. }) => (),
Err(e) => result_tx.send(Err(e)).await.expect(
"Internal multithreading channel is broken: results may be incorrect",
),
Err(e) => {
if let Some(e) = result_tx.send(Err(e)).await.err() {
log::debug!("Sender unable to send results into channel during glob (this is expected if a limit was applied, which results in early termination): {e}");
}
}
}
}

Expand Down Expand Up @@ -579,7 +598,9 @@ pub(crate) async fn glob(
);

let to_rtn_stream = stream! {
while let Some(v) = to_rtn_rx.recv().await {
let mut remaining_results = limit;
while remaining_results.map(|rr| rr > 0).unwrap_or(true) && let Some(v) = to_rtn_rx.recv().await {
remaining_results = remaining_results.map(|rr| rr - 1);
yield v
}
};
Expand Down
2 changes: 2 additions & 0 deletions src/daft-io/src/python.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ mod py {
io_config: Option<common_io_config::python::IOConfig>,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
) -> PyResult<&PyList> {
let multithreaded_io = multithreaded_io.unwrap_or(true);
let io_stats = IOStatsContext::new(format!("io_glob for {path}"));
Expand All @@ -39,6 +40,7 @@ mod py {
path.as_ref(),
fanout_limit,
page_size,
limit,
Some(io_stats_handle),
)
.await?
Expand Down
2 changes: 2 additions & 0 deletions src/daft-io/src/s3_like.rs
Original file line number Diff line number Diff line change
Expand Up @@ -781,6 +781,7 @@ impl ObjectSource for S3LikeSource {
glob_path: &str,
fanout_limit: Option<usize>,
page_size: Option<i32>,
limit: Option<usize>,
io_stats: Option<IOStatsRef>,
) -> super::Result<BoxStream<super::Result<FileMetadata>>> {
use crate::object_store_glob::glob;
Expand All @@ -793,6 +794,7 @@ impl ObjectSource for S3LikeSource {
glob_path,
fanout_limit,
page_size.or(Some(1000)),
limit,
io_stats,
)
.await
Expand Down
4 changes: 4 additions & 0 deletions src/daft-scan/Cargo.toml
Original file line number Diff line number Diff line change
@@ -1,13 +1,17 @@
[dependencies]
common-error = {path = "../common/error", default-features = false}
daft-core = {path = "../daft-core", default-features = false}
daft-csv = {path = "../daft-csv", default-features = false}
daft-dsl = {path = "../daft-dsl", default-features = false}
daft-io = {path = "../daft-io", default-features = false}
daft-parquet = {path = "../daft-parquet", default-features = false}
daft-stats = {path = "../daft-stats", default-features = false}
daft-table = {path = "../daft-table", default-features = false}
pyo3 = {workspace = true, optional = true}
pyo3-log = {workspace = true}
serde = {workspace = true}
snafu = {workspace = true}
tokio = {workspace = true}

[features]
default = ["python"]
Expand Down
Loading
Loading