Skip to content

Improved experience when remote object store URL does not end in / #16386

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

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
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
53 changes: 39 additions & 14 deletions datafusion/datasource/src/url.rs
Original file line number Diff line number Diff line change
Expand Up @@ -242,33 +242,58 @@ impl ListingTableUrl {
) -> Result<BoxStream<'a, Result<ObjectMeta>>> {
let exec_options = &ctx.config_options().execution;
let ignore_subdirectory = exec_options.listing_table_ignore_subdirectory;
// If the prefix is a file, use a head request, otherwise list
let list = match self.is_collection() {
true => match ctx.runtime_env().cache_manager.get_list_files_cache() {
None => store.list(Some(&self.prefix)),

async fn list_with_cache<'b>(
ctx: &'b dyn Session,
store: &'b dyn ObjectStore,
prefix: &'b Path,
) -> Result<BoxStream<'b, Result<ObjectMeta>>> {
match ctx.runtime_env().cache_manager.get_list_files_cache() {
None => Ok(store
.list(Some(prefix))
.map(|res| res.map_err(DataFusionError::ObjectStore))
.boxed()),
Some(cache) => {
if let Some(res) = cache.get(&self.prefix) {
if let Some(res) = cache.get(prefix) {
debug!("Hit list all files cache");
futures::stream::iter(res.as_ref().clone().into_iter().map(Ok))
.boxed()
Ok(futures::stream::iter(
res.as_ref().clone().into_iter().map(Ok),
)
.map(|res| res.map_err(DataFusionError::ObjectStore))
.boxed())
} else {
let list_res = store.list(Some(&self.prefix));
let vec = list_res.try_collect::<Vec<ObjectMeta>>().await?;
cache.put(&self.prefix, Arc::new(vec.clone()));
futures::stream::iter(vec.into_iter().map(Ok)).boxed()
let vec = store
.list(Some(prefix))
.map(|res| res.map_err(DataFusionError::ObjectStore))
.try_collect::<Vec<ObjectMeta>>()
.await?;
cache.put(prefix, Arc::new(vec.clone()));
Ok(futures::stream::iter(vec.into_iter().map(Ok))
.map(|res| res.map_err(DataFusionError::ObjectStore))
.boxed())
}
}
},
false => futures::stream::once(store.head(&self.prefix)).boxed(),
}
}

let list: BoxStream<'a, Result<ObjectMeta>> = if self.is_collection() {
list_with_cache(ctx, store, &self.prefix).await?
} else {
match store.head(&self.prefix).await {
Ok(meta) => futures::stream::once(async { Ok(meta) })
.map(|res| res.map_err(DataFusionError::ObjectStore))
.boxed(),
Err(_) => list_with_cache(ctx, store, &self.prefix).await?,
}
};

Ok(list
.try_filter(move |meta| {
let path = &meta.location;
let extension_match = path.as_ref().ends_with(file_extension);
let glob_match = self.contains(path, ignore_subdirectory);
futures::future::ready(extension_match && glob_match)
})
.map_err(DataFusionError::ObjectStore)
.boxed())
}

Expand Down