You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2024/02/26 18:05:14 UTC

(arrow-datafusion) branch main updated: docs: update parquet_sql_multiple_files.rs with a relative path ex (#9310)

This is an automated email from the ASF dual-hosted git repository.

alamb pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion.git


The following commit(s) were added to refs/heads/main by this push:
     new b8c6e0bb3f docs: update parquet_sql_multiple_files.rs with a relative path ex (#9310)
b8c6e0bb3f is described below

commit b8c6e0bb3f3f8ab60cf394b46d6aacc616b67d41
Author: Trent Hauck <tr...@trenthauck.com>
AuthorDate: Mon Feb 26 10:05:09 2024 -0800

    docs: update parquet_sql_multiple_files.rs with a relative path ex (#9310)
    
    * docs: update parquet_sql_multiple_files.rs with a relative path ex
    
    * style: run cargo fmt
    
    * docs: update comment
    
    * docs: better
---
 .../examples/parquet_sql_multiple_files.rs         | 66 ++++++++++++++++++----
 1 file changed, 56 insertions(+), 10 deletions(-)

diff --git a/datafusion-examples/examples/parquet_sql_multiple_files.rs b/datafusion-examples/examples/parquet_sql_multiple_files.rs
index 451de96f2e..0e2968f203 100644
--- a/datafusion-examples/examples/parquet_sql_multiple_files.rs
+++ b/datafusion-examples/examples/parquet_sql_multiple_files.rs
@@ -17,31 +17,35 @@
 
 use datafusion::datasource::file_format::parquet::ParquetFormat;
 use datafusion::datasource::listing::ListingOptions;
-use datafusion::error::Result;
 use datafusion::prelude::*;
-use datafusion_common::{FileType, GetExt};
+use object_store::local::LocalFileSystem;
+use std::path::Path;
 use std::sync::Arc;
 
 /// This example demonstrates executing a simple query against an Arrow data source (a directory
-/// with multiple Parquet files) and fetching results
+/// with multiple Parquet files) and fetching results. The query is run twice, once showing
+/// how to used `register_listing_table` with an absolute path, and once registering an
+/// ObjectStore to use a relative path.
 #[tokio::main]
-async fn main() -> Result<()> {
+async fn main() -> Result<(), Box<dyn std::error::Error>> {
     // create local execution context
     let ctx = SessionContext::new();
 
-    let testdata = datafusion::test_util::parquet_test_data();
+    let test_data = datafusion::test_util::parquet_test_data();
 
     // Configure listing options
     let file_format = ParquetFormat::default().with_enable_pruning(Some(true));
     let listing_options = ListingOptions::new(Arc::new(file_format))
-        .with_file_extension(FileType::PARQUET.get_ext());
+        // This is a workaround for this example since `test_data` contains
+        // many different parquet different files,
+        // in practice use FileType::PARQUET.get_ext().
+        .with_file_extension("alltypes_plain.parquet");
 
-    // Register a listing table - this will use all files in the directory as data sources
-    // for the query
+    // First example were we use an absolute path, which requires no additional setup.
     ctx.register_listing_table(
         "my_table",
-        &format!("file://{testdata}/alltypes_plain.parquet"),
-        listing_options,
+        &format!("file://{test_data}/"),
+        listing_options.clone(),
         None,
         None,
     )
@@ -60,5 +64,47 @@ async fn main() -> Result<()> {
     // print the results
     df.show().await?;
 
+    // Second example were we temporarily move into the test data's parent directory and
+    // simulate a relative path, this requires registering an ObjectStore.
+    let cur_dir = std::env::current_dir()?;
+
+    let test_data_path = Path::new(&test_data);
+    let test_data_path_parent = test_data_path
+        .parent()
+        .ok_or("test_data path needs a parent")?;
+
+    std::env::set_current_dir(test_data_path_parent)?;
+
+    let local_fs = Arc::new(LocalFileSystem::default());
+
+    let u = url::Url::parse("file://./")?;
+    ctx.runtime_env().register_object_store(&u, local_fs);
+
+    // Register a listing table - this will use all files in the directory as data sources
+    // for the query
+    ctx.register_listing_table(
+        "relative_table",
+        "./data",
+        listing_options.clone(),
+        None,
+        None,
+    )
+    .await?;
+
+    // execute the query
+    let df = ctx
+        .sql(
+            "SELECT * \
+        FROM relative_table \
+        LIMIT 1",
+        )
+        .await?;
+
+    // print the results
+    df.show().await?;
+
+    // Reset the current directory
+    std::env::set_current_dir(cur_dir)?;
+
     Ok(())
 }