saikrishna1-bidgely commented on code in PR #4908:
URL: https://github.com/apache/arrow-datafusion/pull/4908#discussion_r1096915932


##########
datafusion/core/src/execution/context.rs:
##########
@@ -613,50 +615,29 @@ impl SessionContext {
     /// [`read_table`](Self::read_table) with a [`ListingTable`].
     async fn _read_type<'a>(
         &self,
-        table_path: impl AsRef<str>,
+        table_paths: Vec<impl AsRef<str>>,
         options: impl ReadOptions<'a>,
     ) -> Result<DataFrame> {
-        let table_path = ListingTableUrl::parse(table_path)?;
+        let table_paths = table_paths
+            .iter()
+            .map(ListingTableUrl::parse)
+            .collect::<Result<Vec<ListingTableUrl>>>()?;
         let session_config = self.copied_config();
         let listing_options = options.to_listing_options(&session_config);
         let resolved_schema = match options
-            .get_resolved_schema(&session_config, self.state(), 
table_path.clone())
+            .get_resolved_schema(&session_config, self.state(), 
table_paths[0].clone())
             .await
         {
             Ok(resolved_schema) => resolved_schema,
             Err(e) => return Err(e),
         };
-        let config = ListingTableConfig::new(table_path)
+        let config = ListingTableConfig::new_with_multi_paths(table_paths)
             .with_listing_options(listing_options)
             .with_schema(resolved_schema);
         let provider = ListingTable::try_new(config)?;
         self.read_table(Arc::new(provider))
     }
 
-    /// Creates a [`DataFrame`] for reading an Avro data source.
-    ///
-    /// For more control such as reading multiple files, you can use
-    /// [`read_table`](Self::read_table) with a [`ListingTable`].
-    pub async fn read_avro(
-        &self,
-        table_path: impl AsRef<str>,
-        options: AvroReadOptions<'_>,
-    ) -> Result<DataFrame> {
-        self._read_type(table_path, options).await
-    }

Review Comment:
   Hi @alamb, In the above example, we will need to use the `use` statement 
even when we are using `read_csv("s3://bucket/key", options)`. Instead I 
propose the following:
   1. What if we implement AsRef for Vec. Then, we only need to use the `use` 
statement when we are calling for multiple  paths. The default single path code 
will remain unchanged. The AsRef will simply concatenate all the strings in Vec 
with `,` delimiter. And in the `_read_type`, we will split on `,`.
   2. How about having a different method `read_csvs` for taking multiple 
paths. `read_csvs` is a lot cleaner than `read_csv_with_multi_paths`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to