-
Notifications
You must be signed in to change notification settings - Fork 1.3k
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
Allow SessionContext::read_csv
, etc to read multiple files
#4908
Conversation
I'll add for Edit: completed with these functions too. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks like a nice improvement @saikrishna1-bidgely
I think we should add a test for this new functionality so that we don't accidentally break the new APIs going forward.
Also I was wondering about the signature
Rather than a slice, what would you think about taking something that could be turned into an iter:
So instead of
pub async fn read_parquet_with_multi_paths(
&self,
table_paths: &[impl AsRef<str>],
options: ParquetReadOptions<'_>,
) -> Result<DataFrame> {
Something more like
pub async fn read_parquet_with_multi_paths(
&self,
table_paths: impl IntoIterator<Item = &str>],
options: ParquetReadOptions<'_>,
) -> Result<DataFrame> {
I also think it would be ideal to figure out some way to have the same API take both a single path and an iterator -- do you think the above would work?
@@ -551,12 +551,14 @@ impl SessionContext { | |||
} | |||
|
|||
/// Creates a [`DataFrame`] for reading an Avro data source. | |||
pub async fn read_avro( | |||
pub async fn read_avro_with_multi_paths( |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what do you think about calling this read_avro_from_paths
? rather than multi_paths
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I used _with_multi_paths
since arrow uses a similar name but _from_paths
seems cleaner. I'm fine with either.
return self.read_avro_with_multi_paths(&[table_path], options).await; | ||
} | ||
|
||
/// Creates a [`DataFrame`] for reading an Json data source. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Perhaps we can update the docstring as well?
I agree, something like
It is not simple to implement a function that can take both str and iter of str since rust doesn't have function overloading or variadic arguments. We can look into following:
What do you think about having a single method which only takes a list of paths? For a single path, the callee can create a slice/Vec. This would be a lot simpler to do. |
I was thinking about this PR and I have an alternate suggestion It seems to me that For example, I bet if we added a doc example like the following /// Creates a [`DataFrame`] for reading a Parquet data source from a single file or directory.
///
/// Note: if you want to read from multiple files, or control other behaviors
/// you can use the [`ListingTable`] API directly. For example to read multiple files
///
/// ```
/// Example here (basically copy/paste the implementation of read_parquet and support multiple files)
/// ```
pub async fn read_parquet(
&self,
table_path: impl AsRef<str>,
options: ParquetReadOptions<'_>,
) -> Result<DataFrame> {
... We could give similar treatment to the docstrings for |
Quick question, wouldn't we want to support multiple paths anyways since we would want to use it in DataFusion-CLI? Also, if we are able to crack the implementation which can take both single and multiple paths in the same function, API itself should be unchanged, right? So, I think the following are our options:
1 is definitely the simplest but I think we should support it as it would make using DataFusion simpler. |
I think that is likely a separate question -- DataFusion CLI can potentially create a ListingTable directly rather than using the higher level SessionContext API as well
Yes, I agree
I agree with your assessments -- I do think documentation on how to create a ListingTable would go a long way. It seems we are lacking in such docs now https://docs.rs/datafusion/16.0.0/datafusion/datasource/listing/struct.ListingTable.html I think documentation will help regardless of what else we chose to do -- I'll go write some now. Thank you for this good discussion |
Here is a proposal to at least add some more docs: #5001 -- it is not necessarily mutually exclusive with updating the signatures as well |
@alamb I finally got a way to implement overloading using Traits. use std::vec;
fn main() {
struct PATH {
}
impl PATH {
pub fn new() -> Self {
Self {
}
}
}
pub trait Reader<T>: Sized {
fn read_csv(&self, value: T) -> i32;
}
impl<'a> Reader<&'a str> for PATH {
fn read_csv(&self, p: &'a str) -> i32 {
self.read_csv(vec![p])
}
}
impl<'a> Reader<Vec<&'a str>> for PATH {
fn read_csv(&self, v: Vec<&'a str>) -> i32 {
v.len() as i32
}
}
let p = PATH::new();
println!("{:?}", p.read_csv("path"));
println!("{:?}", p.read_csv(vec!["path", "paths2"]));
} This way, callees using I tried to implement using Enum and Union but I wasn't able to do so and they will change the function signature. |
Thank you @saikrishna1-bidgely -- sounds like great progress. I think if we go the trait route as long as we document how to use it (basically we can make a doc example showing the use of a &str) I think we'll be good. Thanks again! |
d518619
to
654d770
Compare
} | ||
|
||
#[async_trait] | ||
impl<'a> Reader<'a, String> for SessionContext { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can't use AsRef here. If we do that, we won't be able to implement for Vec<str>
. We have to implement both &str
and String
separately.
/// | ||
/// For more control such as reading multiple files, you can use | ||
/// [`read_table`](Self::read_table) with a [`ListingTable`]. | ||
async fn read_csv(&self, table_paths: T, options: CsvReadOptions<'_>) -> Result<DataFrame> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For now implemented only for read_csv
but once it gets finalised for it, I'll implement for the rest of the methods.
…aining AsRef<str> trait.
SessionContext::read_csv
, etc to read multiple files
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you @saikrishna1-bidgely -- I really like this PR ❤️ -- thank you for sticking with it.
The only thing I think is missing is a test (mostly to ensure this ability is not removed by accident in the future).
For a test, What do you think about
- adding a doc example on one of the methods (e.g like
SessionContext::read_parquet
that shows reading from a list of strings) - Add a note on the other methods (e.g.
SessionContext::read_csv
) pointing at the method with the example.
I can help with this documentation if you like
options: impl ReadOptions<'a>, | ||
) -> Result<DataFrame> { | ||
let table_path = ListingTableUrl::parse(table_path)?; | ||
let table_paths = table_paths.to_urls()?; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
❤️
Regarding the docs, I think we should extend the example in |
@alamb updated the docs. Should I remove |
@alamb removed methods from |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @saikrishna1-bidgely I think just a few more comment updates and this will be ready to go. Thanks for sticking with it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you @saikrishna1-bidgely -- I think this is looking ready to go.
Cool! Do we need more approvals before merging? |
Nope, I was just giving it some time after approval for other maintainers to have a look if they wanted. 🚀 |
Closing/ reopening to rerun CI (for some reason several of the tests were canceled) |
Thanks again for sticking with this @saikrishna1-bidgely |
1 similar comment
Thanks again for sticking with this @saikrishna1-bidgely |
Benchmark runs are scheduled for baseline = ae89960 and contender = cfbb14d. cfbb14d is a master commit associated with this PR. Results will be available as each benchmark for each run completes. |
…4908) * Added a traitDataFilePaths to convert strings and vector of strings to a vector of URLs. * Added docs and tests. Updated DataFilePaths to accept any vector containing AsRef<str> trait. * Added docs to read_ methods and extended the SessionContext doc. * Ran Cargo fmt * removed CallReadTrait methods * Update read_csv example Co-authored-by: Andrew Lamb <[email protected]> * removed addition to SessionContext example --------- Co-authored-by: Lakkam Sai Krishna Reddy <[email protected]> Co-authored-by: Andrew Lamb <[email protected]>
closes #4909