datafusion_cli/
exec.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Execution functions
19
20use crate::cli_context::CliSessionContext;
21use crate::helper::split_from_semicolon;
22use crate::print_format::PrintFormat;
23use crate::{
24    command::{Command, OutputFormat},
25    helper::CliHelper,
26    object_storage::get_object_store,
27    print_options::{MaxRows, PrintOptions},
28};
29use datafusion::common::instant::Instant;
30use datafusion::common::{plan_datafusion_err, plan_err};
31use datafusion::config::ConfigFileType;
32use datafusion::datasource::listing::ListingTableUrl;
33use datafusion::error::{DataFusionError, Result};
34use datafusion::execution::memory_pool::MemoryConsumer;
35use datafusion::logical_expr::{DdlStatement, LogicalPlan};
36use datafusion::physical_plan::execution_plan::EmissionType;
37use datafusion::physical_plan::spill::get_record_batch_memory_size;
38use datafusion::physical_plan::{ExecutionPlanProperties, execute_stream};
39use datafusion::sql::parser::{DFParser, Statement};
40use datafusion::sql::sqlparser;
41use datafusion::sql::sqlparser::dialect::dialect_from_str;
42use futures::StreamExt;
43use log::warn;
44use object_store::Error::Generic;
45use rustyline::Editor;
46use rustyline::error::ReadlineError;
47use std::collections::HashMap;
48use std::fs::File;
49use std::io::BufReader;
50use std::io::prelude::*;
51use tokio::signal;
52
53/// run and execute SQL statements and commands, against a context with the given print options
54pub async fn exec_from_commands(
55    ctx: &dyn CliSessionContext,
56    commands: Vec<String>,
57    print_options: &PrintOptions,
58) -> Result<()> {
59    for sql in commands {
60        exec_and_print(ctx, print_options, sql).await?;
61    }
62
63    Ok(())
64}
65
66/// run and execute SQL statements and commands from a file, against a context with the given print options
67pub async fn exec_from_lines(
68    ctx: &dyn CliSessionContext,
69    reader: &mut BufReader<File>,
70    print_options: &PrintOptions,
71) -> Result<()> {
72    let mut query = "".to_owned();
73
74    for line in reader.lines() {
75        match line {
76            Ok(line) if line.starts_with("#!") => {
77                continue;
78            }
79            Ok(line) if line.starts_with("--") => {
80                continue;
81            }
82            Ok(line) => {
83                let line = line.trim_end();
84                query.push_str(line);
85                if line.ends_with(';') {
86                    match exec_and_print(ctx, print_options, query).await {
87                        Ok(_) => {}
88                        Err(err) => eprintln!("{err}"),
89                    }
90                    query = "".to_string();
91                } else {
92                    query.push('\n');
93                }
94            }
95            _ => {
96                break;
97            }
98        }
99    }
100
101    // run the left over query if the last statement doesn't contain ‘;’
102    // ignore if it only consists of '\n'
103    if query.contains(|c| c != '\n') {
104        exec_and_print(ctx, print_options, query).await?;
105    }
106
107    Ok(())
108}
109
110pub async fn exec_from_files(
111    ctx: &dyn CliSessionContext,
112    files: Vec<String>,
113    print_options: &PrintOptions,
114) -> Result<()> {
115    let files = files
116        .into_iter()
117        .map(|file_path| File::open(file_path).unwrap())
118        .collect::<Vec<_>>();
119
120    for file in files {
121        let mut reader = BufReader::new(file);
122        exec_from_lines(ctx, &mut reader, print_options).await?;
123    }
124
125    Ok(())
126}
127
128/// run and execute SQL statements and commands against a context with the given print options
129pub async fn exec_from_repl(
130    ctx: &dyn CliSessionContext,
131    print_options: &mut PrintOptions,
132) -> rustyline::Result<()> {
133    let mut rl = Editor::new()?;
134    rl.set_helper(Some(CliHelper::new(
135        &ctx.task_ctx().session_config().options().sql_parser.dialect,
136        print_options.color,
137    )));
138    rl.load_history(".history").ok();
139
140    loop {
141        match rl.readline("> ") {
142            Ok(line) if line.starts_with('\\') => {
143                rl.add_history_entry(line.trim_end())?;
144                let command = line.split_whitespace().collect::<Vec<_>>().join(" ");
145                if let Ok(cmd) = &command[1..].parse::<Command>() {
146                    match cmd {
147                        Command::Quit => break,
148                        Command::OutputFormat(subcommand) => {
149                            if let Some(subcommand) = subcommand {
150                                if let Ok(command) = subcommand.parse::<OutputFormat>() {
151                                    if let Err(e) = command.execute(print_options).await {
152                                        eprintln!("{e}")
153                                    }
154                                } else {
155                                    eprintln!(
156                                        "'\\{}' is not a valid command, you can use '\\?' to see all commands",
157                                        &line[1..]
158                                    );
159                                }
160                            } else {
161                                println!("Output format is {:?}.", print_options.format);
162                            }
163                        }
164                        _ => {
165                            if let Err(e) = cmd.execute(ctx, print_options).await {
166                                eprintln!("{e}")
167                            }
168                        }
169                    }
170                } else {
171                    eprintln!(
172                        "'\\{}' is not a valid command, you can use '\\?' to see all commands",
173                        &line[1..]
174                    );
175                }
176            }
177            Ok(line) => {
178                let lines = split_from_semicolon(&line);
179                for line in lines {
180                    rl.add_history_entry(line.trim_end())?;
181                    tokio::select! {
182                        res = exec_and_print(ctx, print_options, line) => match res {
183                            Ok(_) => {}
184                            Err(err) => eprintln!("{err}"),
185                        },
186                        _ = signal::ctrl_c() => {
187                            println!("^C");
188                            continue
189                        },
190                    }
191                    // dialect might have changed
192                    rl.helper_mut().unwrap().set_dialect(
193                        &ctx.task_ctx().session_config().options().sql_parser.dialect,
194                    );
195                }
196            }
197            Err(ReadlineError::Interrupted) => {
198                println!("^C");
199                continue;
200            }
201            Err(ReadlineError::Eof) => {
202                println!("\\q");
203                break;
204            }
205            Err(err) => {
206                eprintln!("Unknown error happened {err:?}");
207                break;
208            }
209        }
210    }
211
212    rl.save_history(".history")
213}
214
215pub(super) async fn exec_and_print(
216    ctx: &dyn CliSessionContext,
217    print_options: &PrintOptions,
218    sql: String,
219) -> Result<()> {
220    let task_ctx = ctx.task_ctx();
221    let options = task_ctx.session_config().options();
222    let dialect = &options.sql_parser.dialect;
223    let dialect = dialect_from_str(dialect).ok_or_else(|| {
224        plan_datafusion_err!(
225            "Unsupported SQL dialect: {dialect}. Available dialects: \
226                 Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, \
227                 MsSQL, ClickHouse, BigQuery, Ansi, DuckDB, Databricks."
228        )
229    })?;
230
231    let statements = DFParser::parse_sql_with_dialect(&sql, dialect.as_ref())?;
232    for statement in statements {
233        StatementExecutor::new(statement)
234            .execute(ctx, print_options)
235            .await?;
236    }
237
238    Ok(())
239}
240
241/// Executor for SQL statements, including special handling for S3 region detection retry logic
242struct StatementExecutor {
243    statement: Statement,
244    statement_for_retry: Option<Statement>,
245}
246
247impl StatementExecutor {
248    fn new(statement: Statement) -> Self {
249        let statement_for_retry = matches!(statement, Statement::CreateExternalTable(_))
250            .then(|| statement.clone());
251
252        Self {
253            statement,
254            statement_for_retry,
255        }
256    }
257
258    async fn execute(
259        self,
260        ctx: &dyn CliSessionContext,
261        print_options: &PrintOptions,
262    ) -> Result<()> {
263        let now = Instant::now();
264        let (df, adjusted) = self
265            .create_and_execute_logical_plan(ctx, print_options)
266            .await?;
267        let physical_plan = df.create_physical_plan().await?;
268        let task_ctx = ctx.task_ctx();
269        let options = task_ctx.session_config().options();
270
271        // Track memory usage for the query result if it's bounded
272        let mut reservation =
273            MemoryConsumer::new("DataFusion-Cli").register(task_ctx.memory_pool());
274
275        if physical_plan.boundedness().is_unbounded() {
276            if physical_plan.pipeline_behavior() == EmissionType::Final {
277                return plan_err!(
278                    "The given query can generate a valid result only once \
279                    the source finishes, but the source is unbounded"
280                );
281            }
282            // As the input stream comes, we can generate results.
283            // However, memory safety is not guaranteed.
284            let stream = execute_stream(physical_plan, task_ctx.clone())?;
285            print_options
286                .print_stream(stream, now, &options.format)
287                .await?;
288        } else {
289            // Bounded stream; collected results size is limited by the maxrows option
290            let schema = physical_plan.schema();
291            let mut stream = execute_stream(physical_plan, task_ctx.clone())?;
292            let mut results = vec![];
293            let mut row_count = 0_usize;
294            let max_rows = match print_options.maxrows {
295                MaxRows::Unlimited => usize::MAX,
296                MaxRows::Limited(n) => n,
297            };
298            while let Some(batch) = stream.next().await {
299                let batch = batch?;
300                let curr_num_rows = batch.num_rows();
301                // Stop collecting results if the number of rows exceeds the limit
302                // results batch should include the last batch that exceeds the limit
303                if row_count < max_rows + curr_num_rows {
304                    // Try to grow the reservation to accommodate the batch in memory
305                    reservation.try_grow(get_record_batch_memory_size(&batch))?;
306                    results.push(batch);
307                }
308                row_count += curr_num_rows;
309            }
310            adjusted.into_inner().print_batches(
311                schema,
312                &results,
313                now,
314                row_count,
315                &options.format,
316            )?;
317            reservation.free();
318        }
319
320        Ok(())
321    }
322
323    async fn create_and_execute_logical_plan(
324        mut self,
325        ctx: &dyn CliSessionContext,
326        print_options: &PrintOptions,
327    ) -> Result<(datafusion::dataframe::DataFrame, AdjustedPrintOptions)> {
328        let adjusted = AdjustedPrintOptions::new(print_options.clone())
329            .with_statement(&self.statement);
330
331        let plan = create_plan(ctx, self.statement, false).await?;
332        let adjusted = adjusted.with_plan(&plan);
333
334        let df = match ctx.execute_logical_plan(plan).await {
335            Ok(df) => Ok(df),
336            Err(DataFusionError::ObjectStore(err))
337                if matches!(err.as_ref(), Generic { store, source: _ } if "S3".eq_ignore_ascii_case(store))
338                    && self.statement_for_retry.is_some() =>
339            {
340                warn!(
341                    "S3 region is incorrect, auto-detecting the correct region (this may be slow). Consider updating your region configuration."
342                );
343                let plan =
344                    create_plan(ctx, self.statement_for_retry.take().unwrap(), true)
345                        .await?;
346                ctx.execute_logical_plan(plan).await
347            }
348            Err(e) => Err(e),
349        }?;
350
351        Ok((df, adjusted))
352    }
353}
354
355/// Track adjustments to the print options based on the plan / statement being executed
356#[derive(Debug)]
357struct AdjustedPrintOptions {
358    inner: PrintOptions,
359}
360
361impl AdjustedPrintOptions {
362    fn new(inner: PrintOptions) -> Self {
363        Self { inner }
364    }
365    /// Adjust print options based on any statement specific requirements
366    fn with_statement(mut self, statement: &Statement) -> Self {
367        if let Statement::Statement(sql_stmt) = statement {
368            // SHOW / SHOW ALL
369            if let sqlparser::ast::Statement::ShowVariable { .. } = sql_stmt.as_ref() {
370                self.inner.maxrows = MaxRows::Unlimited
371            }
372        }
373        self
374    }
375
376    /// Adjust print options based on any plan specific requirements
377    fn with_plan(mut self, plan: &LogicalPlan) -> Self {
378        // For plans like `Explain` ignore `MaxRows` option and always display
379        // all rows
380        if matches!(
381            plan,
382            LogicalPlan::Explain(_)
383                | LogicalPlan::DescribeTable(_)
384                | LogicalPlan::Analyze(_)
385        ) {
386            self.inner.maxrows = MaxRows::Unlimited;
387        }
388        self
389    }
390
391    /// Finalize and return the inner `PrintOptions`
392    fn into_inner(mut self) -> PrintOptions {
393        if self.inner.format == PrintFormat::Automatic {
394            self.inner.format = PrintFormat::Table;
395        }
396
397        self.inner
398    }
399}
400
401fn config_file_type_from_str(ext: &str) -> Option<ConfigFileType> {
402    match ext.to_lowercase().as_str() {
403        "csv" => Some(ConfigFileType::CSV),
404        "json" => Some(ConfigFileType::JSON),
405        "parquet" => Some(ConfigFileType::PARQUET),
406        _ => None,
407    }
408}
409
410async fn create_plan(
411    ctx: &dyn CliSessionContext,
412    statement: Statement,
413    resolve_region: bool,
414) -> Result<LogicalPlan, DataFusionError> {
415    let mut plan = ctx.session_state().statement_to_plan(statement).await?;
416
417    // Note that cmd is a mutable reference so that create_external_table function can remove all
418    // datafusion-cli specific options before passing through to datafusion. Otherwise, datafusion
419    // will raise Configuration errors.
420    if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan {
421        // To support custom formats, treat error as None
422        let format = config_file_type_from_str(&cmd.file_type);
423        register_object_store_and_config_extensions(
424            ctx,
425            &cmd.location,
426            &cmd.options,
427            format,
428            resolve_region,
429        )
430        .await?;
431    }
432
433    if let LogicalPlan::Copy(copy_to) = &mut plan {
434        let format = config_file_type_from_str(&copy_to.file_type.get_ext());
435
436        register_object_store_and_config_extensions(
437            ctx,
438            &copy_to.output_url,
439            &copy_to.options,
440            format,
441            false,
442        )
443        .await?;
444    }
445    Ok(plan)
446}
447
448/// Asynchronously registers an object store and its configuration extensions
449/// to the session context.
450///
451/// This function dynamically registers a cloud object store based on the given
452/// location and options. It first parses the location to determine the scheme
453/// and constructs the URL accordingly. Depending on the scheme, it also registers
454/// relevant options. The function then alters the default table options with the
455/// given custom options. Finally, it retrieves and registers the object store
456/// in the session context.
457///
458/// # Parameters
459///
460/// * `ctx`: A reference to the `SessionContext` for registering the object store.
461/// * `location`: A string reference representing the location of the object store.
462/// * `options`: A reference to a hash map containing configuration options for
463///   the object store.
464///
465/// # Returns
466///
467/// A `Result<()>` which is an Ok value indicating successful registration, or
468/// an error upon failure.
469///
470/// # Errors
471///
472/// This function can return an error if the location parsing fails, options
473/// alteration fails, or if the object store cannot be retrieved and registered
474/// successfully.
475pub(crate) async fn register_object_store_and_config_extensions(
476    ctx: &dyn CliSessionContext,
477    location: &String,
478    options: &HashMap<String, String>,
479    format: Option<ConfigFileType>,
480    resolve_region: bool,
481) -> Result<()> {
482    // Parse the location URL to extract the scheme and other components
483    let table_path = ListingTableUrl::parse(location)?;
484
485    // Extract the scheme (e.g., "s3", "gcs") from the parsed URL
486    let scheme = table_path.scheme();
487
488    // Obtain a reference to the URL
489    let url = table_path.as_ref();
490
491    // Register the options based on the scheme extracted from the location
492    ctx.register_table_options_extension_from_scheme(scheme);
493
494    // Clone and modify the default table options based on the provided options
495    let mut table_options = ctx.session_state().default_table_options();
496    if let Some(format) = format {
497        table_options.set_config_format(format);
498    }
499    table_options.alter_with_string_hash_map(options)?;
500
501    // Retrieve the appropriate object store based on the scheme, URL, and modified table options
502    let store = get_object_store(
503        &ctx.session_state(),
504        scheme,
505        url,
506        &table_options,
507        resolve_region,
508    )
509    .await?;
510
511    // Register the retrieved object store in the session context's runtime environment
512    ctx.register_object_store(url, store);
513
514    Ok(())
515}
516
517#[cfg(test)]
518mod tests {
519    use super::*;
520
521    use datafusion::common::plan_err;
522
523    use datafusion::prelude::SessionContext;
524    use url::Url;
525
526    async fn create_external_table_test(location: &str, sql: &str) -> Result<()> {
527        let ctx = SessionContext::new();
528        let plan = ctx.state().create_logical_plan(sql).await?;
529
530        if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan {
531            let format = config_file_type_from_str(&cmd.file_type);
532            register_object_store_and_config_extensions(
533                &ctx,
534                &cmd.location,
535                &cmd.options,
536                format,
537                false,
538            )
539            .await?;
540        } else {
541            return plan_err!("LogicalPlan is not a CreateExternalTable");
542        }
543
544        // Ensure the URL is supported by the object store
545        ctx.runtime_env()
546            .object_store(ListingTableUrl::parse(location)?)?;
547
548        Ok(())
549    }
550
551    async fn copy_to_table_test(location: &str, sql: &str) -> Result<()> {
552        let ctx = SessionContext::new();
553        // AWS CONFIG register.
554
555        let plan = ctx.state().create_logical_plan(sql).await?;
556
557        if let LogicalPlan::Copy(cmd) = &plan {
558            let format = config_file_type_from_str(&cmd.file_type.get_ext());
559            register_object_store_and_config_extensions(
560                &ctx,
561                &cmd.output_url,
562                &cmd.options,
563                format,
564                false,
565            )
566            .await?;
567        } else {
568            return plan_err!("LogicalPlan is not a CreateExternalTable");
569        }
570
571        // Ensure the URL is supported by the object store
572        ctx.runtime_env()
573            .object_store(ListingTableUrl::parse(location)?)?;
574
575        Ok(())
576    }
577
578    #[tokio::test]
579    async fn create_object_store_table_http() -> Result<()> {
580        // Should be OK
581        let location = "http://example.com/file.parquet";
582        let sql =
583            format!("CREATE EXTERNAL TABLE test STORED AS PARQUET LOCATION '{location}'");
584        create_external_table_test(location, &sql).await?;
585
586        Ok(())
587    }
588    #[tokio::test]
589    async fn copy_to_external_object_store_test() -> Result<()> {
590        let aws_envs = vec![
591            "AWS_ENDPOINT",
592            "AWS_ACCESS_KEY_ID",
593            "AWS_SECRET_ACCESS_KEY",
594            "AWS_ALLOW_HTTP",
595        ];
596        for aws_env in aws_envs {
597            if std::env::var(aws_env).is_err() {
598                eprint!("aws envs not set, skipping s3 test");
599                return Ok(());
600            }
601        }
602
603        let locations = vec![
604            "s3://bucket/path/file.parquet",
605            "oss://bucket/path/file.parquet",
606            "cos://bucket/path/file.parquet",
607            "gcs://bucket/path/file.parquet",
608        ];
609        let ctx = SessionContext::new();
610        let task_ctx = ctx.task_ctx();
611        let dialect = &task_ctx.session_config().options().sql_parser.dialect;
612        let dialect = dialect_from_str(dialect).ok_or_else(|| {
613            plan_datafusion_err!(
614                "Unsupported SQL dialect: {dialect}. Available dialects: \
615                 Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, \
616                 MsSQL, ClickHouse, BigQuery, Ansi, DuckDB, Databricks."
617            )
618        })?;
619        for location in locations {
620            let sql = format!("copy (values (1,2)) to '{location}' STORED AS PARQUET;");
621            let statements = DFParser::parse_sql_with_dialect(&sql, dialect.as_ref())?;
622            for statement in statements {
623                //Should not fail
624                let mut plan = create_plan(&ctx, statement, false).await?;
625                if let LogicalPlan::Copy(copy_to) = &mut plan {
626                    assert_eq!(copy_to.output_url, location);
627                    assert_eq!(copy_to.file_type.get_ext(), "parquet".to_string());
628                    ctx.runtime_env()
629                        .object_store_registry
630                        .get_store(&Url::parse(&copy_to.output_url).unwrap())?;
631                } else {
632                    return plan_err!("LogicalPlan is not a CopyTo");
633                }
634            }
635        }
636        Ok(())
637    }
638
639    #[tokio::test]
640    async fn copy_to_object_store_table_s3() -> Result<()> {
641        let access_key_id = "fake_access_key_id";
642        let secret_access_key = "fake_secret_access_key";
643        let location = "s3://bucket/path/file.parquet";
644
645        // Missing region, use object_store defaults
646        let sql = format!("COPY (values (1,2)) TO '{location}' STORED AS PARQUET
647            OPTIONS ('aws.access_key_id' '{access_key_id}', 'aws.secret_access_key' '{secret_access_key}')");
648        copy_to_table_test(location, &sql).await?;
649
650        Ok(())
651    }
652
653    #[tokio::test]
654    async fn create_object_store_table_s3() -> Result<()> {
655        let access_key_id = "fake_access_key_id";
656        let secret_access_key = "fake_secret_access_key";
657        let region = "fake_us-east-2";
658        let session_token = "fake_session_token";
659        let location = "s3://bucket/path/file.parquet";
660
661        // Missing region, use object_store defaults
662        let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET
663            OPTIONS('aws.access_key_id' '{access_key_id}', 'aws.secret_access_key' '{secret_access_key}') LOCATION '{location}'");
664        create_external_table_test(location, &sql).await?;
665
666        // Should be OK
667        let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET
668            OPTIONS('aws.access_key_id' '{access_key_id}', 'aws.secret_access_key' '{secret_access_key}', 'aws.region' '{region}', 'aws.session_token' '{session_token}') LOCATION '{location}'");
669        create_external_table_test(location, &sql).await?;
670
671        Ok(())
672    }
673
674    #[tokio::test]
675    async fn create_object_store_table_oss() -> Result<()> {
676        let access_key_id = "fake_access_key_id";
677        let secret_access_key = "fake_secret_access_key";
678        let endpoint = "fake_endpoint";
679        let location = "oss://bucket/path/file.parquet";
680
681        // Should be OK
682        let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET
683            OPTIONS('aws.access_key_id' '{access_key_id}', 'aws.secret_access_key' '{secret_access_key}', 'aws.oss.endpoint' '{endpoint}') LOCATION '{location}'");
684        create_external_table_test(location, &sql).await?;
685
686        Ok(())
687    }
688
689    #[tokio::test]
690    async fn create_object_store_table_cos() -> Result<()> {
691        let access_key_id = "fake_access_key_id";
692        let secret_access_key = "fake_secret_access_key";
693        let endpoint = "fake_endpoint";
694        let location = "cos://bucket/path/file.parquet";
695
696        // Should be OK
697        let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET
698            OPTIONS('aws.access_key_id' '{access_key_id}', 'aws.secret_access_key' '{secret_access_key}', 'aws.cos.endpoint' '{endpoint}') LOCATION '{location}'");
699        create_external_table_test(location, &sql).await?;
700
701        Ok(())
702    }
703
704    #[tokio::test]
705    async fn create_object_store_table_gcs() -> Result<()> {
706        let service_account_path = "fake_service_account_path";
707        let service_account_key = "{\"private_key\": \"fake_private_key.pem\",\"client_email\":\"fake_client_email\", \"private_key_id\":\"id\"}";
708        let application_credentials_path = "fake_application_credentials_path";
709        let location = "gcs://bucket/path/file.parquet";
710
711        // for service_account_path
712        let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET
713            OPTIONS('gcp.service_account_path' '{service_account_path}') LOCATION '{location}'");
714        let err = create_external_table_test(location, &sql)
715            .await
716            .unwrap_err();
717        assert!(err.to_string().contains("os error 2"));
718
719        // for service_account_key
720        let sql = format!(
721            "CREATE EXTERNAL TABLE test STORED AS PARQUET OPTIONS('gcp.service_account_key' '{service_account_key}') LOCATION '{location}'"
722        );
723        let err = create_external_table_test(location, &sql)
724            .await
725            .unwrap_err()
726            .to_string();
727        assert!(err.contains("No RSA key found in pem file"), "{err}");
728
729        // for application_credentials_path
730        let sql = format!("CREATE EXTERNAL TABLE test STORED AS PARQUET
731            OPTIONS('gcp.application_credentials_path' '{application_credentials_path}') LOCATION '{location}'");
732        let err = create_external_table_test(location, &sql)
733            .await
734            .unwrap_err();
735        assert!(err.to_string().contains("os error 2"));
736
737        Ok(())
738    }
739
740    #[tokio::test]
741    async fn create_external_table_local_file() -> Result<()> {
742        let location = "path/to/file.parquet";
743
744        // Ensure that local files are also registered
745        let sql =
746            format!("CREATE EXTERNAL TABLE test STORED AS PARQUET LOCATION '{location}'");
747        create_external_table_test(location, &sql).await.unwrap();
748
749        Ok(())
750    }
751
752    #[tokio::test]
753    async fn create_external_table_format_option() -> Result<()> {
754        let location = "path/to/file.cvs";
755
756        // Test with format options
757        let sql = format!(
758            "CREATE EXTERNAL TABLE test STORED AS CSV LOCATION '{location}' OPTIONS('format.has_header' 'true')"
759        );
760        create_external_table_test(location, &sql).await.unwrap();
761
762        Ok(())
763    }
764}