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

Support select .. from 'data.parquet' files in SQL from any SessionContext (optionally) #4850

Closed
alamb opened this issue Jan 8, 2023 · 25 comments
Labels
enhancement New feature or request good first issue Good for newcomers

Comments

@alamb
Copy link
Contributor

alamb commented Jan 8, 2023

Is your feature request related to a problem or challenge? Please describe what you are trying to do.
Similarly to #4580, I think systems built with datafusion would like to allow their users to quickly explore a parquet file with minimal typing

Today have to type a verbose CREATE EXTERNAL TABLE... command

It is critical that this feature can be enabled/disabled so that DataFusion can provide read only access (rather than access to the file system as that would be a security hole)

I am marking this as a good first issue because I think all the code needed exists and there is a solution sketch below -- it should be a matter of coding that doesn't require huge existing knowledge of the datafusion codebase and would be a good exercise in getting familiar

❯ CREATE EXTERNAL TABLE all_types_tiny_pages STORED AS PARQUET LOCATION './parquet-testing/data/alltypes_tiny_pages.parquet';

❯ select * from all_types_tiny_pages limit 10;
+-----+----------+-------------+--------------+---------+------------+-----------+--------------------+-----------------+------------+-------------------------+------+-------+
| id  | bool_col | tinyint_col | smallint_col | int_col | bigint_col | float_col | double_col         | date_string_col | string_col | timestamp_col           | year | month |
+-----+----------+-------------+--------------+---------+------------+-----------+--------------------+-----------------+------------+-------------------------+------+-------+
| 122 | true     | 2           | 2            | 2       | 20         | 2.2       | 20.2               | 01/13/09        | 2          | 2009-01-13T01:02:05.410 | 2009 | 1     |
| 123 | false    | 3           | 3            | 3       | 30         | 3.3       | 30.299999999999997 | 01/13/09        | 3          | 2009-01-13T01:03:05.430 | 2009 | 1     |
| 124 | true     | 4           | 4            | 4       | 40         | 4.4       | 40.4               | 01/13/09        | 4          | 2009-01-13T01:04:05.460 | 2009 | 1     |
| 125 | false    | 5           | 5            | 5       | 50         | 5.5       | 50.5               | 01/13/09        | 5          | 2009-01-13T01:05:05.500 | 2009 | 1     |
| 126 | true     | 6           | 6            | 6       | 60         | 6.6       | 60.599999999999994 | 01/13/09        | 6          | 2009-01-13T01:06:05.550 | 2009 | 1     |
| 127 | false    | 7           | 7            | 7       | 70         | 7.7       | 70.7               | 01/13/09        | 7          | 2009-01-13T01:07:05.610 | 2009 | 1     |
| 128 | true     | 8           | 8            | 8       | 80         | 8.8       | 80.8               | 01/13/09        | 8          | 2009-01-13T01:08:05.680 | 2009 | 1     |
| 129 | false    | 9           | 9            | 9       | 90         | 9.9       | 90.89999999999999  | 01/13/09        | 9          | 2009-01-13T01:09:05.760 | 2009 | 1     |
| 130 | true     | 0           | 0            | 0       | 0          | 0         | 0                  | 01/14/09        | 0          | 2009-01-14T01:10:05.850 | 2009 | 1     |
| 131 | false    | 1           | 1            | 1       | 10         | 1.1       | 10.1               | 01/14/09        | 1          | 2009-01-14T01:11:05.850 | 2009 | 1     |
+-----+----------+-------------+--------------+---------+------------+-----------+--------------------+-----------------+------------+-------------------------+------+-------+
10 rows in set. Query took 0.039 seconds.

@unconsolable added this ability into datafusion-cli as part of #4838 (❤️ )

select * from '/Users/alamb/.influxdb_iox//1/8/1/6/e24b6549-f76c-4fc0-a4f4-152ed60eb4e3.parquet';
+---------+---------------------+------+---------+----------+---------+---------------------+-------+---------+---------+
| blocked | host                | idle | running | sleeping | stopped | time                | total | unknown | zombies |
+---------+---------------------+------+---------+----------+---------+---------------------+-------+---------+---------+
| 0       | MacBook-Pro-8.local | 0    | 2       | 697      | 0       | 2022-07-18T21:05:10 | 700   | 0       | 1       |
| 0       | MacBook-Pro-8.local | 0    | 2       | 696      | 0       | 2022-07-18T21:05:20 | 699   | 0       | 1       |
+---------+---------------------+------+---------+----------+---------+---------------------+-------+---------+---------+
2 rows in set. Query took 0.079 seconds.

Describe the solution you'd like

I would like to be able to select directly from files (parquet, or other) from any datafusion session context, controlled by a setting. For example

select * from 'foo.parquet';
Plan("table 'foo.parquet' not found")
❯ -- turn on the settingset datafusion.catalog.files_as_tables = true;
0 rows in set. Query took 0.000 seconds.
❯ select * from 'foo.parquet';
+---------+---------------------+------+---------+----------+---------+---------------------+-------+---------+---------+
| blocked | host                | idle | running | sleeping | stopped | time                | total | unknown | zombies |
+---------+---------------------+------+---------+----------+---------+---------------------+-------+---------+---------+
| 0       | MacBook-Pro-8.local | 0    | 2       | 697      | 0       | 2022-07-18T21:05:10 | 700   | 0       | 1       |
| 0       | MacBook-Pro-8.local | 0    | 2       | 696      | 0       | 2022-07-18T21:05:20 | 699   | 0       | 1       |
+---------+---------------------+------+---------+----------+---------+---------------------+-------+---------+---------+
2 rows in set. Query took 0.079 seconds.

Suggested Solution Sketch

  1. Add a new config setting files_as_tables similar to information_schema: https://github.com/apache/arrow-datafusion/blob/f9b72f4230687b884a92f79d21762578d3d56281/datafusion/common/src/config.rs#L167-L169

  2. Add code to make a ListingTable in resolve_table_ref: https://github.com/apache/arrow-datafusion/blob/f9b72f4230687b884a92f79d21762578d3d56281/datafusion/core/src/execution/context.rs#L1551-L1560 (follow the model in https://github.com/apache/arrow-datafusion/pull/4838/files#diff-6353c2268d4d11abf8c1b8804a263db74a3b765a7302fc61caea3924256b52c7R142-R155)

  3. Move implementation from datafusion-cli; remove provider added in support select .. FROM 'parquet.file' in datafusion-cli #4838 and use new setting instead https://github.com/apache/arrow-datafusion/blob/f9b72f4230687b884a92f79d21762578d3d56281/datafusion-cli/src/main.rs#L100
    Add slt tests, similar to existing ones (should be able to refer to existing .parquet / .csv files in testing directories): https://github.com/apache/arrow-datafusion/blob/f9b72f4230687b884a92f79d21762578d3d56281/datafusion/core/tests/sqllogictests/test_files/information_schema.slt#L46

Describe alternatives you've considered
A clear and concise description of any alternative solutions or features you've considered.

Additional context

Here is how information schema works, for reference.

❯ set datafusion.catalog.information_schema = false;
0 rows in set. Query took 0.002 seconds.
❯ show tables;
Plan("SHOW TABLES is not supported unless information_schema is enabled")
❯ set datafusion.catalog.information_schema = true;
0 rows in set. Query took 0.000 seconds.
❯ show tables;
+---------------+--------------------+-------------+------------+
| table_catalog | table_schema       | table_name  | table_type |
+---------------+--------------------+-------------+------------+
| datafusion    | information_schema | tables      | VIEW       |
| datafusion    | information_schema | views       | VIEW       |
| datafusion    | information_schema | columns     | VIEW       |
| datafusion    | information_schema | df_settings | VIEW       |
+---------------+--------------------+-------------+------------+
4 rows in set. Query took 0.005 seconds.
❯ 
@alamb alamb added enhancement New feature or request good first issue Good for newcomers labels Jan 8, 2023
@alamb alamb changed the title Support select .. from 'data.parquet' files in SQL Support select .. from 'data.parquet' files in SQL from any SessionContext (optionally) Jan 8, 2023
@matthewwillian
Copy link
Contributor

I'm happy to take on this issue if no one else is already working on it!

@timvw
Copy link
Contributor

timvw commented Jan 31, 2023

Enhancement: Instead of only relying on file extension name (as per the current implementation) we could use some inspiration from duckdb for loading/importing data such that the user can indicate the format to use (https://duckdb.org/docs/data/csv)

Eg: "select ... from read_csv('filename', options)" instead of plain "select .. from 'filename'"..

This would be helpful in cases where files are named "hitdata.tsv" (https://experienceleague.adobe.com/docs/analytics/export/analytics-data-feed/data-feed-overview.html?lang=en)

@alamb
Copy link
Contributor Author

alamb commented Jan 31, 2023

I would love to see a function like read_csv or maybe read_file('filename') 👍

@unconsolable
Copy link
Contributor

IMHO, these are table functions. I wonder does datafusion support table function now?
ref. #3773

@alamb
Copy link
Contributor Author

alamb commented Feb 1, 2023

I agree with @unconsolable

@holicc
Copy link

holicc commented Sep 8, 2023

I would love to see a function like read_csv or maybe read_file('filename') 👍

@alamb Are there any processes here? I'd like to try this.

@alamb
Copy link
Contributor Author

alamb commented Sep 8, 2023

@alamb Are there any processes here? I'd like to try this.

Hi @holicc -- no there is no progress on this issue that I know of.

Given the conversation above, it seems like there are two options:

  1. Implement table functions (like read_file(...))
  2. Implement some automatic catalog (the way that datafusion-cli does)

Have you given any thought to the approach to take?

@holicc
Copy link

holicc commented Sep 8, 2023

@alamb Maybe both options are good. Like DuckDB (see documentation: http://duckdb.org/docs/archive/0.8.1/data/csv/overview):

  1. When using read_file, you might need to specify file parse options, e.g. read_csv('simple.csv', delim='|', header=True).
  2. The way that datafusion-cli does it also makes it convenient for us.

@alamb
Copy link
Contributor Author

alamb commented Sep 8, 2023

Having both makes sense to me.

I would say making a table function is likely a large piece of design work as there is no pre-existing model to follow for table functions (we probably need some sort of registry and pass that expands them out to some sort of TableProvider

@holicc
Copy link

holicc commented Sep 8, 2023

I found out that we are using the sqlparser crate to parse SQL. Therefore, we can modify the parsed statements with the visitorMut trait, collect all tables, and replace them with a new table name. Finally, we can use the register_csv API to register those tables. This way may be a little tricky.

@alamb
Copy link
Contributor Author

alamb commented Sep 8, 2023

I found out that we are using the sqlparser crate to parse SQL. Therefore, we can modify the parsed statements with the visitorMut trait, collect all tables, and replace them with a new table name. Finally, we can use the register_csv API to register those tables. This way may be a little tricky.

yes, we could

I would recommend thinking a little more holistically about table functions

Specifically adding them to the registry: https://docs.rs/datafusion/latest/datafusion/execution/registry/trait.FunctionRegistry.html

And then teaching our SQL planner how to look up those functions

@holicc
Copy link

holicc commented Sep 14, 2023

IMHO, TableFunction is essentially a TableProvider. Therefore, we can make a new trait that returns a TableProvider and execute the table function in create_relation and using TableScan to build a LogicPlan

Add a new trait:

#[async_trait]
pub trait TableFunction: Sync + Send {
    async fn execute(
        &self,
        table: impl Into<OwnedTableReference>,
        state: &SessionState,
    ) -> Result<Arc<dyn TableProvider>>;
}

add new method to FunctionRegistry

pub trait FunctionRegistry {
    ...
    /// Returns a reference to the table function named `name`.
    fn table_function(&self, name: &str) -> Result<Arc<dyn TableFunction>>;
}

@alamb
Copy link
Contributor Author

alamb commented Sep 14, 2023

Therefore, we can make a new trait that returns a TableProvider and execute the table function in create_relation and using TableScan to build a LogicPlan

I think one big difference is that a table function can take some number of parameters

SELECT * FROM read_parquet('test.parquet');

The table function is read_parquet(..) which gets arguments of a string test.parquet.

The SQL parser will give the argument as an Expr.
https://docs.rs/sqlparser/latest/sqlparser/ast/enum.TableFactor.html#variant.TableFunction

So perhaps we could make the TableFuncton trait like:

#[async_trait]
pub trait TableFunction: Sync + Send {
    async fn execute(
        &self,
        arg: Expr,
        state: &SessionState,
    ) -> Result<Arc<dyn TableProvider>>;
}

To take that arbitrary argument in

@timvw
Copy link
Contributor

timvw commented Sep 15, 2023

And what about multiple arguments? Eg: read_csv('blah.csv', delimiter = ';', ... )

@alamb
Copy link
Contributor Author

alamb commented Sep 15, 2023

And what about multiple arguments? Eg: read_csv('blah.csv', delimiter = ';', ... )

Good point @timvw -- we would probably want to allow that in the API. I tried it out and it does appear that the sql parser supports that kind of syntax 👍

select foo from read_parquet('foo.parquet', 'bar');
Error during planning: table 'datafusion.public.read_parquet' not found

@holicc
Copy link

holicc commented Sep 18, 2023

@alamb I'm having trouble inferring the schema because I can't get a SessionState from the ContextProvider. Can you help me?

let listing_options = CsvReadOptions::default().to_listing_options(&self.config);
let url = ListingTableUrl::parse(file_path)?;
let cfg = ListingTableConfig::new(url)
            .with_listing_options(listing_options)
            .with_schema(Arc::new(arrow_schema::Schema::empty()));

// FIXME How to get a SessionState?
cfg.infer_schema(state);

let table = ListingTable::try_new(cfg)?;
let source = Arc::new(DefaultTableSource::new(Arc::new(table)));

@alamb
Copy link
Contributor Author

alamb commented Sep 18, 2023

Hi @holicc -- I don't think it is possible to do this in the SQL parser level (as the the datafusion crate depends on the datafusion-sql crate, not the reverse.

Thus I think the resolution from a table name to a ListingTable may have to happen later in the process. In the Suggested Solution Sketch above I tried to suggest taking inspiration from the dynamic lookup works today in datafusion-cli. Perhaps we can follow the same model here

@alamb
Copy link
Contributor Author

alamb commented May 30, 2024

cc @goldmedal here is an issue that describes supporting reading style urls as tables

select * from 's3://foo/bar.parquet'

Probably a good first step would be to move the DynamicFileCatalogProvider

struct DynamicFileCatalogProvider {
into the core somewhere (but not registering it with SessionContext)

Then a second step would be to add an option (like the information_schema tables) that would enable installing the DynamicFileCatalogProvider during construction

@goldmedal
Copy link
Contributor

Probably a good first step would be to move the DynamicFileCatalogProvider

struct DynamicFileCatalogProvider {

into the core somewhere (but not registering it with SessionContext)
Then a second step would be to add an option (like the information_schema tables) that would enable installing the DynamicFileCatalogProvider during construction

HI @alamb, I'm working on it. I think I'll create PRs for each steps.

First PR: Move to core.

After roughly surveyed, I found I need to move not only DynamicFileCatalogProvider but also something object_store-related.
I plan to place

  • DynamicFileCatalogProvider in datafusion/core/src/catalog
  • object_store-related in datafusion/core/src/datasource/file_format

Second PR: Add Option

I haven't survey it. Seems I can refer to how information_schema implemented.

@edmondop
Copy link
Contributor

edmondop commented Jun 2, 2024

@alamb spark SQL syntax works like so:

select * from parquet.`s3://foo-bar`

what do you think?

  • I wouldn't rely on the extension, and I don't know how that works when you are querying a partitioned table (where the .parquet files are in nested folders
  • if you use Parquet + a metadata format (Iceberg, Hudi, Delta) you might still have Parquet files, but a protocol layer on top of it, so one can support select * from iceberg.s3://foo-bar``

@alamb
Copy link
Contributor Author

alamb commented Jun 4, 2024

@edmondop I think this would be a great to add as an example / thing to implement as an extension.

@edmondop
Copy link
Contributor

edmondop commented Jun 9, 2024

I wanted to confirm I understood correctly what options we are picking. It seems to me the following are viable:

  1. Support "implicit" tables via select from (moving the change from datafusion-cli in core)
  2. Support table functions and create a separate function for each format (read_csv, read_parquet, read_json too maybe)
  3. Move formats handling to extensions?

We are taking option 1 right now, is that right @alamb ?

@alamb
Copy link
Contributor Author

alamb commented Jun 11, 2024

I am not quite sure what to do here to be honest

@alamb
Copy link
Contributor Author

alamb commented Jun 18, 2024

Update here is that @goldmedal made a PR for this issue #10745

However the initial PR brings many dependencies (like aws crates) to datafusion core that is likely not great. I had some suggestions on how we could split up the code to keep the dynamic file provider in the core whil ekeeping aws etc out: #10745 (comment)

@alamb
Copy link
Contributor Author

alamb commented Sep 18, 2024

Completed in #10986

@alamb alamb closed this as completed Sep 18, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement New feature or request good first issue Good for newcomers
Projects
None yet
Development

No branches or pull requests

7 participants