operator/
statement.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15mod admin;
16mod comment;
17mod copy_database;
18mod copy_query_to;
19mod copy_table_from;
20mod copy_table_to;
21mod cursor;
22pub mod ddl;
23mod describe;
24mod dml;
25mod kill;
26mod set;
27mod show;
28mod tql;
29
30use std::collections::HashMap;
31use std::sync::Arc;
32
33use api::v1::RowInsertRequests;
34use catalog::CatalogManagerRef;
35use catalog::kvbackend::KvBackendCatalogManager;
36use catalog::process_manager::ProcessManagerRef;
37use client::RecordBatches;
38use client::error::{ExternalSnafu as ClientExternalSnafu, Result as ClientResult};
39use client::inserter::{InsertOptions, Inserter};
40use common_error::ext::BoxedError;
41use common_meta::cache_invalidator::CacheInvalidatorRef;
42use common_meta::key::flow::{FlowMetadataManager, FlowMetadataManagerRef};
43use common_meta::key::schema_name::SchemaNameKey;
44use common_meta::key::view_info::{ViewInfoManager, ViewInfoManagerRef};
45use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
46use common_meta::kv_backend::KvBackendRef;
47use common_meta::procedure_executor::ProcedureExecutorRef;
48use common_query::Output;
49use common_telemetry::{debug, tracing, warn};
50use common_time::Timestamp;
51use common_time::range::TimestampRange;
52use datafusion_expr::LogicalPlan;
53use datatypes::prelude::ConcreteDataType;
54use datatypes::schema::ColumnSchema;
55use humantime::format_duration;
56use itertools::Itertools;
57use partition::manager::PartitionRuleManagerRef;
58use query::QueryEngineRef;
59use query::parser::QueryStatement;
60use session::context::{Channel, QueryContextBuilder, QueryContextRef};
61use session::table_name::table_idents_to_full_name;
62use set::{set_query_timeout, set_read_preference};
63use snafu::{OptionExt, ResultExt, ensure};
64use sql::ast::ObjectNamePartExt;
65use sql::statements::OptionMap;
66use sql::statements::copy::{
67    CopyDatabase, CopyDatabaseArgument, CopyQueryToArgument, CopyTable, CopyTableArgument,
68};
69use sql::statements::set_variables::SetVariables;
70use sql::statements::show::ShowCreateTableVariant;
71use sql::statements::statement::Statement;
72use sql::util::format_raw_object_name;
73use sqlparser::ast::ObjectName;
74use store_api::mito_engine_options::{APPEND_MODE_KEY, TTL_KEY};
75use table::TableRef;
76use table::requests::{CopyDatabaseRequest, CopyDirection, CopyQueryToRequest, CopyTableRequest};
77use table::table_name::TableName;
78use table::table_reference::TableReference;
79
80use self::set::{
81    set_bytea_output, set_datestyle, set_intervalstyle, set_search_path, set_timezone,
82    validate_client_encoding,
83};
84use crate::error::{
85    self, CatalogSnafu, ExecLogicalPlanSnafu, ExternalSnafu, InvalidSqlSnafu, NotSupportedSnafu,
86    PlanStatementSnafu, Result, SchemaNotFoundSnafu, SqlCommonSnafu, TableMetadataManagerSnafu,
87    TableNotFoundSnafu, UnexpectedSnafu, UpgradeCatalogManagerRefSnafu,
88};
89use crate::insert::InserterRef;
90use crate::statement::copy_database::{COPY_DATABASE_TIME_END_KEY, COPY_DATABASE_TIME_START_KEY};
91use crate::statement::set::set_allow_query_fallback;
92
93/// A configurator that customizes or enhances a [`StatementExecutor`].
94#[async_trait::async_trait]
95pub trait StatementExecutorConfigurator: Send + Sync {
96    async fn configure(
97        &self,
98        executor: StatementExecutor,
99        ctx: ExecutorConfigureContext,
100    ) -> std::result::Result<StatementExecutor, BoxedError>;
101}
102
103pub type StatementExecutorConfiguratorRef = Arc<dyn StatementExecutorConfigurator>;
104
105pub struct ExecutorConfigureContext {
106    pub kv_backend: KvBackendRef,
107}
108
109#[derive(Clone)]
110pub struct StatementExecutor {
111    catalog_manager: CatalogManagerRef,
112    query_engine: QueryEngineRef,
113    procedure_executor: ProcedureExecutorRef,
114    table_metadata_manager: TableMetadataManagerRef,
115    flow_metadata_manager: FlowMetadataManagerRef,
116    view_info_manager: ViewInfoManagerRef,
117    partition_manager: PartitionRuleManagerRef,
118    cache_invalidator: CacheInvalidatorRef,
119    inserter: InserterRef,
120    process_manager: Option<ProcessManagerRef>,
121    #[cfg(feature = "enterprise")]
122    trigger_querier: Option<TriggerQuerierRef>,
123}
124
125pub type StatementExecutorRef = Arc<StatementExecutor>;
126
127/// Trait for querying trigger info, such as `SHOW CREATE TRIGGER` etc.
128#[cfg(feature = "enterprise")]
129#[async_trait::async_trait]
130pub trait TriggerQuerier: Send + Sync {
131    // Query the `SHOW CREATE TRIGGER` statement for the given trigger.
132    async fn show_create_trigger(
133        &self,
134        catalog: &str,
135        trigger: &str,
136        query_ctx: &QueryContextRef,
137    ) -> std::result::Result<Output, BoxedError>;
138
139    fn as_any(&self) -> &dyn std::any::Any;
140}
141
142#[cfg(feature = "enterprise")]
143pub type TriggerQuerierRef = Arc<dyn TriggerQuerier>;
144
145impl StatementExecutor {
146    #[allow(clippy::too_many_arguments)]
147    pub fn new(
148        catalog_manager: CatalogManagerRef,
149        query_engine: QueryEngineRef,
150        procedure_executor: ProcedureExecutorRef,
151        kv_backend: KvBackendRef,
152        cache_invalidator: CacheInvalidatorRef,
153        inserter: InserterRef,
154        partition_manager: PartitionRuleManagerRef,
155        process_manager: Option<ProcessManagerRef>,
156    ) -> Self {
157        Self {
158            catalog_manager,
159            query_engine,
160            procedure_executor,
161            table_metadata_manager: Arc::new(TableMetadataManager::new(kv_backend.clone())),
162            flow_metadata_manager: Arc::new(FlowMetadataManager::new(kv_backend.clone())),
163            view_info_manager: Arc::new(ViewInfoManager::new(kv_backend.clone())),
164            partition_manager,
165            cache_invalidator,
166            inserter,
167            process_manager,
168            #[cfg(feature = "enterprise")]
169            trigger_querier: None,
170        }
171    }
172
173    #[cfg(feature = "enterprise")]
174    pub fn with_trigger_querier(mut self, querier: TriggerQuerierRef) -> Self {
175        self.trigger_querier = Some(querier);
176        self
177    }
178
179    #[cfg(feature = "testing")]
180    pub async fn execute_stmt(
181        &self,
182        stmt: QueryStatement,
183        query_ctx: QueryContextRef,
184    ) -> Result<Output> {
185        match stmt {
186            QueryStatement::Sql(stmt) => self.execute_sql(stmt, query_ctx).await,
187            QueryStatement::Promql(_, _) => self.plan_exec(stmt, query_ctx).await,
188        }
189    }
190
191    #[tracing::instrument(skip_all)]
192    pub async fn execute_sql(&self, stmt: Statement, query_ctx: QueryContextRef) -> Result<Output> {
193        match stmt {
194            Statement::Query(_) | Statement::Explain(_) | Statement::Delete(_) => {
195                self.plan_exec(QueryStatement::Sql(stmt), query_ctx).await
196            }
197
198            Statement::DeclareCursor(declare_cursor) => {
199                self.declare_cursor(declare_cursor, query_ctx).await
200            }
201            Statement::FetchCursor(fetch_cursor) => {
202                self.fetch_cursor(fetch_cursor, query_ctx).await
203            }
204            Statement::CloseCursor(close_cursor) => {
205                self.close_cursor(close_cursor, query_ctx).await
206            }
207
208            Statement::Insert(insert) => self.insert(insert, query_ctx).await,
209
210            Statement::Tql(tql) => self.execute_tql(tql, query_ctx).await,
211
212            Statement::DescribeTable(stmt) => self.describe_table(stmt, query_ctx).await,
213
214            Statement::ShowDatabases(stmt) => self.show_databases(stmt, query_ctx).await,
215
216            Statement::ShowTables(stmt) => self.show_tables(stmt, query_ctx).await,
217
218            Statement::ShowTableStatus(stmt) => self.show_table_status(stmt, query_ctx).await,
219
220            Statement::ShowCollation(kind) => self.show_collation(kind, query_ctx).await,
221
222            Statement::ShowCharset(kind) => self.show_charset(kind, query_ctx).await,
223
224            Statement::ShowViews(stmt) => self.show_views(stmt, query_ctx).await,
225
226            Statement::ShowFlows(stmt) => self.show_flows(stmt, query_ctx).await,
227
228            #[cfg(feature = "enterprise")]
229            Statement::ShowTriggers(stmt) => self.show_triggers(stmt, query_ctx).await,
230
231            Statement::Copy(sql::statements::copy::Copy::CopyQueryTo(stmt)) => {
232                let query_output = self
233                    .plan_exec(QueryStatement::Sql(*stmt.query), query_ctx)
234                    .await?;
235                let req = to_copy_query_request(stmt.arg)?;
236
237                self.copy_query_to(req, query_output)
238                    .await
239                    .map(Output::new_with_affected_rows)
240            }
241
242            Statement::Copy(sql::statements::copy::Copy::CopyTable(stmt)) => {
243                let req = to_copy_table_request(stmt, query_ctx.clone())?;
244                match req.direction {
245                    CopyDirection::Export => self
246                        .copy_table_to(req, query_ctx)
247                        .await
248                        .map(Output::new_with_affected_rows),
249                    CopyDirection::Import => self.copy_table_from(req, query_ctx).await,
250                }
251            }
252
253            Statement::Copy(sql::statements::copy::Copy::CopyDatabase(copy_database)) => {
254                match copy_database {
255                    CopyDatabase::To(arg) => {
256                        self.copy_database_to(
257                            to_copy_database_request(arg, &query_ctx)?,
258                            query_ctx.clone(),
259                        )
260                        .await
261                    }
262                    CopyDatabase::From(arg) => {
263                        self.copy_database_from(
264                            to_copy_database_request(arg, &query_ctx)?,
265                            query_ctx,
266                        )
267                        .await
268                    }
269                }
270            }
271
272            Statement::CreateTable(stmt) => {
273                let _ = self.create_table(stmt, query_ctx).await?;
274                Ok(Output::new_with_affected_rows(0))
275            }
276            Statement::CreateTableLike(stmt) => {
277                let _ = self.create_table_like(stmt, query_ctx).await?;
278                Ok(Output::new_with_affected_rows(0))
279            }
280            Statement::CreateExternalTable(stmt) => {
281                let _ = self.create_external_table(stmt, query_ctx).await?;
282                Ok(Output::new_with_affected_rows(0))
283            }
284            Statement::CreateFlow(stmt) => self.create_flow(stmt, query_ctx).await,
285            #[cfg(feature = "enterprise")]
286            Statement::CreateTrigger(stmt) => self.create_trigger(stmt, query_ctx).await,
287            Statement::DropFlow(stmt) => {
288                self.drop_flow(
289                    query_ctx.current_catalog().to_string(),
290                    format_raw_object_name(stmt.flow_name()),
291                    stmt.drop_if_exists(),
292                    query_ctx,
293                )
294                .await
295            }
296            #[cfg(feature = "enterprise")]
297            Statement::DropTrigger(stmt) => {
298                self.drop_trigger(
299                    query_ctx.current_catalog().to_string(),
300                    format_raw_object_name(stmt.trigger_name()),
301                    stmt.drop_if_exists(),
302                    query_ctx,
303                )
304                .await
305            }
306            Statement::CreateView(stmt) => {
307                let _ = self.create_view(stmt, query_ctx).await?;
308                Ok(Output::new_with_affected_rows(0))
309            }
310            Statement::DropView(stmt) => {
311                let (catalog_name, schema_name, view_name) =
312                    table_idents_to_full_name(&stmt.view_name, &query_ctx)
313                        .map_err(BoxedError::new)
314                        .context(ExternalSnafu)?;
315
316                self.drop_view(
317                    catalog_name,
318                    schema_name,
319                    view_name,
320                    stmt.drop_if_exists,
321                    query_ctx,
322                )
323                .await
324            }
325            Statement::AlterTable(alter_table) => self.alter_table(alter_table, query_ctx).await,
326
327            Statement::AlterDatabase(alter_database) => {
328                self.alter_database(alter_database, query_ctx).await
329            }
330
331            #[cfg(feature = "enterprise")]
332            Statement::AlterTrigger(alter_trigger) => {
333                self.alter_trigger(alter_trigger, query_ctx).await
334            }
335
336            Statement::DropTable(stmt) => {
337                let mut table_names = Vec::with_capacity(stmt.table_names().len());
338                for table_name_stmt in stmt.table_names() {
339                    let (catalog, schema, table) =
340                        table_idents_to_full_name(table_name_stmt, &query_ctx)
341                            .map_err(BoxedError::new)
342                            .context(ExternalSnafu)?;
343                    table_names.push(TableName::new(catalog, schema, table));
344                }
345                self.drop_tables(&table_names[..], stmt.drop_if_exists(), query_ctx.clone())
346                    .await
347            }
348            Statement::DropDatabase(stmt) => {
349                self.drop_database(
350                    query_ctx.current_catalog().to_string(),
351                    format_raw_object_name(stmt.name()),
352                    stmt.drop_if_exists(),
353                    query_ctx,
354                )
355                .await
356            }
357            Statement::TruncateTable(stmt) => {
358                let (catalog, schema, table) =
359                    table_idents_to_full_name(stmt.table_name(), &query_ctx)
360                        .map_err(BoxedError::new)
361                        .context(ExternalSnafu)?;
362                let table_name = TableName::new(catalog, schema, table);
363                let time_ranges = self
364                    .convert_truncate_time_ranges(&table_name, stmt.time_ranges(), &query_ctx)
365                    .await?;
366                self.truncate_table(table_name, time_ranges, query_ctx)
367                    .await
368            }
369            Statement::CreateDatabase(stmt) => {
370                self.create_database(
371                    &format_raw_object_name(&stmt.name),
372                    stmt.if_not_exists,
373                    stmt.options.into_map(),
374                    query_ctx,
375                )
376                .await
377            }
378            Statement::ShowCreateDatabase(show) => {
379                let (catalog, database) =
380                    idents_to_full_database_name(&show.database_name, &query_ctx)
381                        .map_err(BoxedError::new)
382                        .context(ExternalSnafu)?;
383                let table_metadata_manager = self
384                    .catalog_manager
385                    .as_any()
386                    .downcast_ref::<KvBackendCatalogManager>()
387                    .map(|manager| manager.table_metadata_manager_ref().clone())
388                    .context(UpgradeCatalogManagerRefSnafu)?;
389                let opts: HashMap<String, String> = table_metadata_manager
390                    .schema_manager()
391                    .get(SchemaNameKey::new(&catalog, &database))
392                    .await
393                    .context(TableMetadataManagerSnafu)?
394                    .context(SchemaNotFoundSnafu {
395                        schema_info: &database,
396                    })?
397                    .into_inner()
398                    .into();
399
400                self.show_create_database(&database, opts.into()).await
401            }
402            Statement::ShowCreateTable(show) => {
403                let (catalog, schema, table) =
404                    table_idents_to_full_name(&show.table_name, &query_ctx)
405                        .map_err(BoxedError::new)
406                        .context(ExternalSnafu)?;
407
408                let table_ref = self
409                    .catalog_manager
410                    .table(&catalog, &schema, &table, Some(&query_ctx))
411                    .await
412                    .context(CatalogSnafu)?
413                    .context(TableNotFoundSnafu { table_name: &table })?;
414                let table_name = TableName::new(catalog, schema, table);
415
416                match show.variant {
417                    ShowCreateTableVariant::Original => {
418                        self.show_create_table(table_name, table_ref, query_ctx)
419                            .await
420                    }
421                    ShowCreateTableVariant::PostgresForeignTable => {
422                        self.show_create_table_for_pg(table_name, table_ref, query_ctx)
423                            .await
424                    }
425                }
426            }
427            Statement::ShowCreateFlow(show) => self.show_create_flow(show, query_ctx).await,
428            Statement::ShowCreateView(show) => self.show_create_view(show, query_ctx).await,
429            #[cfg(feature = "enterprise")]
430            Statement::ShowCreateTrigger(show) => self.show_create_trigger(show, query_ctx).await,
431            Statement::SetVariables(set_var) => self.set_variables(set_var, query_ctx),
432            Statement::ShowVariables(show_variable) => self.show_variable(show_variable, query_ctx),
433            Statement::Comment(stmt) => self.comment(stmt, query_ctx).await,
434            Statement::ShowColumns(show_columns) => {
435                self.show_columns(show_columns, query_ctx).await
436            }
437            Statement::ShowIndex(show_index) => self.show_index(show_index, query_ctx).await,
438            Statement::ShowRegion(show_region) => self.show_region(show_region, query_ctx).await,
439            Statement::ShowStatus(_) => self.show_status(query_ctx).await,
440            Statement::ShowSearchPath(_) => self.show_search_path(query_ctx).await,
441            Statement::Use(db) => self.use_database(db, query_ctx).await,
442            Statement::Admin(admin) => self.execute_admin_command(admin, query_ctx).await,
443            Statement::Kill(kill) => self.execute_kill(query_ctx, kill).await,
444            Statement::ShowProcesslist(show) => self.show_processlist(show, query_ctx).await,
445        }
446    }
447
448    pub async fn use_database(&self, db: String, query_ctx: QueryContextRef) -> Result<Output> {
449        let catalog = query_ctx.current_catalog();
450        ensure!(
451            self.catalog_manager
452                .schema_exists(catalog, db.as_ref(), Some(&query_ctx))
453                .await
454                .context(CatalogSnafu)?,
455            SchemaNotFoundSnafu { schema_info: &db }
456        );
457
458        query_ctx.set_current_schema(&db);
459
460        Ok(Output::new_with_record_batches(RecordBatches::empty()))
461    }
462
463    fn set_variables(&self, set_var: SetVariables, query_ctx: QueryContextRef) -> Result<Output> {
464        let var_name = set_var.variable.to_string().to_uppercase();
465
466        debug!(
467            "Trying to set {}={} for session: {} ",
468            var_name,
469            set_var.value.iter().map(|e| e.to_string()).join(", "),
470            query_ctx.conn_info()
471        );
472
473        match var_name.as_str() {
474            "READ_PREFERENCE" => set_read_preference(set_var.value, query_ctx)?,
475
476            "@@TIME_ZONE" | "@@SESSION.TIME_ZONE" | "TIMEZONE" | "TIME_ZONE" => {
477                set_timezone(set_var.value, query_ctx)?
478            }
479
480            "BYTEA_OUTPUT" => set_bytea_output(set_var.value, query_ctx)?,
481
482            // Same as "bytea_output", we just ignore it here.
483            // Not harmful since it only relates to how date is viewed in client app's output.
484            // The tracked issue is https://github.com/GreptimeTeam/greptimedb/issues/3442.
485            "DATESTYLE" => set_datestyle(set_var.value, query_ctx)?,
486            "INTERVALSTYLE" => set_intervalstyle(set_var.value, query_ctx)?,
487
488            // Allow query to fallback when failed to push down.
489            "ALLOW_QUERY_FALLBACK" => set_allow_query_fallback(set_var.value, query_ctx)?,
490
491            "CLIENT_ENCODING" => validate_client_encoding(set_var)?,
492            "@@SESSION.MAX_EXECUTION_TIME" | "MAX_EXECUTION_TIME" => match query_ctx.channel() {
493                Channel::Mysql => set_query_timeout(set_var.value, query_ctx)?,
494                Channel::Postgres => {
495                    warn!(
496                        "Unsupported set variable {} for channel {:?}",
497                        var_name,
498                        query_ctx.channel()
499                    );
500                    query_ctx.set_warning(format!("Unsupported set variable {}", var_name))
501                }
502                _ => {
503                    return NotSupportedSnafu {
504                        feat: format!("Unsupported set variable {}", var_name),
505                    }
506                    .fail();
507                }
508            },
509            "STATEMENT_TIMEOUT" => match query_ctx.channel() {
510                Channel::Postgres => set_query_timeout(set_var.value, query_ctx)?,
511                Channel::Mysql => {
512                    warn!(
513                        "Unsupported set variable {} for channel {:?}",
514                        var_name,
515                        query_ctx.channel()
516                    );
517                    query_ctx.set_warning(format!("Unsupported set variable {}", var_name));
518                }
519                _ => {
520                    return NotSupportedSnafu {
521                        feat: format!("Unsupported set variable {}", var_name),
522                    }
523                    .fail();
524                }
525            },
526            "SEARCH_PATH" => {
527                if query_ctx.channel() == Channel::Postgres {
528                    set_search_path(set_var.value, query_ctx)?
529                } else {
530                    return NotSupportedSnafu {
531                        feat: format!("Unsupported set variable {}", var_name),
532                    }
533                    .fail();
534                }
535            }
536            _ => {
537                if query_ctx.channel() == Channel::Postgres || query_ctx.channel() == Channel::Mysql
538                {
539                    // For unknown SET statements, we give a warning with success.
540                    // This prevents the SET call from becoming a blocker of MySQL/Postgres clients'
541                    // connection establishment.
542                    warn!(
543                        "Unsupported set variable {} for channel {:?}",
544                        var_name,
545                        query_ctx.channel()
546                    );
547                    query_ctx.set_warning(format!("Unsupported set variable {}", var_name));
548                } else {
549                    return NotSupportedSnafu {
550                        feat: format!("Unsupported set variable {}", var_name),
551                    }
552                    .fail();
553                }
554            }
555        }
556        Ok(Output::new_with_affected_rows(0))
557    }
558
559    #[tracing::instrument(skip_all)]
560    pub async fn plan(
561        &self,
562        stmt: &QueryStatement,
563        query_ctx: QueryContextRef,
564    ) -> Result<LogicalPlan> {
565        self.query_engine
566            .planner()
567            .plan(stmt, query_ctx)
568            .await
569            .context(PlanStatementSnafu)
570    }
571
572    /// Execute [`LogicalPlan`] directly.
573    #[tracing::instrument(skip_all)]
574    pub async fn exec_plan(&self, plan: LogicalPlan, query_ctx: QueryContextRef) -> Result<Output> {
575        self.query_engine
576            .execute(plan, query_ctx)
577            .await
578            .context(ExecLogicalPlanSnafu)
579    }
580
581    pub fn optimize_logical_plan(&self, plan: LogicalPlan) -> Result<LogicalPlan> {
582        self.query_engine
583            .planner()
584            .optimize(plan)
585            .context(PlanStatementSnafu)
586    }
587
588    #[tracing::instrument(skip_all)]
589    async fn plan_exec(&self, stmt: QueryStatement, query_ctx: QueryContextRef) -> Result<Output> {
590        let plan = self.plan(&stmt, query_ctx.clone()).await?;
591        self.exec_plan(plan, query_ctx).await
592    }
593
594    async fn get_table(&self, table_ref: &TableReference<'_>) -> Result<TableRef> {
595        let TableReference {
596            catalog,
597            schema,
598            table,
599        } = table_ref;
600        self.catalog_manager
601            .table(catalog, schema, table, None)
602            .await
603            .context(CatalogSnafu)?
604            .with_context(|| TableNotFoundSnafu {
605                table_name: table_ref.to_string(),
606            })
607    }
608
609    pub fn procedure_executor(&self) -> &ProcedureExecutorRef {
610        &self.procedure_executor
611    }
612
613    pub fn cache_invalidator(&self) -> &CacheInvalidatorRef {
614        &self.cache_invalidator
615    }
616
617    /// Convert truncate time ranges for the given table from sql values to timestamps
618    ///
619    pub async fn convert_truncate_time_ranges(
620        &self,
621        table_name: &TableName,
622        sql_values_time_range: &[(sqlparser::ast::Value, sqlparser::ast::Value)],
623        query_ctx: &QueryContextRef,
624    ) -> Result<Vec<(Timestamp, Timestamp)>> {
625        if sql_values_time_range.is_empty() {
626            return Ok(vec![]);
627        }
628        let table = self.get_table(&table_name.table_ref()).await?;
629        let info = table.table_info();
630        let time_index_dt = info
631            .meta
632            .schema
633            .timestamp_column()
634            .context(UnexpectedSnafu {
635                violated: "Table must have a timestamp column",
636            })?;
637
638        let time_unit = time_index_dt
639            .data_type
640            .as_timestamp()
641            .with_context(|| UnexpectedSnafu {
642                violated: format!(
643                    "Table {}'s time index column must be a timestamp type, found: {:?}",
644                    table_name, time_index_dt
645                ),
646            })?
647            .unit();
648
649        let start_column = ColumnSchema::new(
650            "range_start",
651            ConcreteDataType::timestamp_datatype(time_unit),
652            false,
653        );
654        let end_column = ColumnSchema::new(
655            "range_end",
656            ConcreteDataType::timestamp_datatype(time_unit),
657            false,
658        );
659        let mut time_ranges = Vec::with_capacity(sql_values_time_range.len());
660        for (start, end) in sql_values_time_range {
661            let start = common_sql::convert::sql_value_to_value(
662                &start_column,
663                start,
664                Some(&query_ctx.timezone()),
665                None,
666                false,
667            )
668            .context(SqlCommonSnafu)
669            .and_then(|v| {
670                if let datatypes::value::Value::Timestamp(t) = v {
671                    Ok(t)
672                } else {
673                    error::InvalidSqlSnafu {
674                        err_msg: format!("Expected a timestamp value, found {v:?}"),
675                    }
676                    .fail()
677                }
678            })?;
679
680            let end = common_sql::convert::sql_value_to_value(
681                &end_column,
682                end,
683                Some(&query_ctx.timezone()),
684                None,
685                false,
686            )
687            .context(SqlCommonSnafu)
688            .and_then(|v| {
689                if let datatypes::value::Value::Timestamp(t) = v {
690                    Ok(t)
691                } else {
692                    error::InvalidSqlSnafu {
693                        err_msg: format!("Expected a timestamp value, found {v:?}"),
694                    }
695                    .fail()
696                }
697            })?;
698            time_ranges.push((start, end));
699        }
700        Ok(time_ranges)
701    }
702
703    /// Returns the inserter for the statement executor.
704    pub(crate) fn inserter(&self) -> &InserterRef {
705        &self.inserter
706    }
707}
708
709fn to_copy_query_request(stmt: CopyQueryToArgument) -> Result<CopyQueryToRequest> {
710    let CopyQueryToArgument {
711        with,
712        connection,
713        location,
714    } = stmt;
715
716    Ok(CopyQueryToRequest {
717        location,
718        with: with.into_map(),
719        connection: connection.into_map(),
720    })
721}
722
723// Verifies time related format is valid
724fn verify_time_related_format(with: &OptionMap) -> Result<()> {
725    let time_format = with.get(common_datasource::file_format::TIME_FORMAT);
726    let date_format = with.get(common_datasource::file_format::DATE_FORMAT);
727    let timestamp_format = with.get(common_datasource::file_format::TIMESTAMP_FORMAT);
728    let file_format = with.get(common_datasource::file_format::FORMAT_TYPE);
729
730    if !matches!(file_format, Some(f) if f.eq_ignore_ascii_case("csv")) {
731        ensure!(
732            time_format.is_none() && date_format.is_none() && timestamp_format.is_none(),
733            error::TimestampFormatNotSupportedSnafu {
734                format: "<unknown>".to_string(),
735                file_format: file_format.unwrap_or_default(),
736            }
737        );
738    }
739
740    for (key, format_opt) in [
741        (common_datasource::file_format::TIME_FORMAT, time_format),
742        (common_datasource::file_format::DATE_FORMAT, date_format),
743        (
744            common_datasource::file_format::TIMESTAMP_FORMAT,
745            timestamp_format,
746        ),
747    ] {
748        if let Some(format) = format_opt {
749            chrono::format::strftime::StrftimeItems::new(format)
750                .parse()
751                .map_err(|_| error::InvalidCopyParameterSnafu { key, value: format }.build())?;
752        }
753    }
754
755    Ok(())
756}
757
758fn to_copy_table_request(stmt: CopyTable, query_ctx: QueryContextRef) -> Result<CopyTableRequest> {
759    let direction = match stmt {
760        CopyTable::To(_) => CopyDirection::Export,
761        CopyTable::From(_) => CopyDirection::Import,
762    };
763
764    let CopyTableArgument {
765        location,
766        connection,
767        with,
768        table_name,
769        limit,
770        ..
771    } = match stmt {
772        CopyTable::To(arg) => arg,
773        CopyTable::From(arg) => arg,
774    };
775    let (catalog_name, schema_name, table_name) =
776        table_idents_to_full_name(&table_name, &query_ctx)
777            .map_err(BoxedError::new)
778            .context(ExternalSnafu)?;
779
780    let timestamp_range = timestamp_range_from_option_map(&with, &query_ctx)?;
781
782    verify_time_related_format(&with)?;
783
784    let pattern = with
785        .get(common_datasource::file_format::FILE_PATTERN)
786        .map(|x| x.to_string());
787
788    Ok(CopyTableRequest {
789        catalog_name,
790        schema_name,
791        table_name,
792        location,
793        with: with.into_map(),
794        connection: connection.into_map(),
795        pattern,
796        direction,
797        timestamp_range,
798        limit,
799    })
800}
801
802/// Converts [CopyDatabaseArgument] to [CopyDatabaseRequest].
803/// This function extracts the necessary info including catalog/database name, time range, etc.
804fn to_copy_database_request(
805    arg: CopyDatabaseArgument,
806    query_ctx: &QueryContextRef,
807) -> Result<CopyDatabaseRequest> {
808    let (catalog_name, database_name) = idents_to_full_database_name(&arg.database_name, query_ctx)
809        .map_err(BoxedError::new)
810        .context(ExternalSnafu)?;
811    let time_range = timestamp_range_from_option_map(&arg.with, query_ctx)?;
812
813    Ok(CopyDatabaseRequest {
814        catalog_name,
815        schema_name: database_name,
816        location: arg.location,
817        with: arg.with.into_map(),
818        connection: arg.connection.into_map(),
819        time_range,
820    })
821}
822
823/// Extracts timestamp range from OptionMap with keys `start_time` and `end_time`.
824/// The timestamp ranges should be a valid timestamp string as defined in [Timestamp::from_str].
825/// The timezone used for conversion will respect that inside `query_ctx`.
826fn timestamp_range_from_option_map(
827    options: &OptionMap,
828    query_ctx: &QueryContextRef,
829) -> Result<Option<TimestampRange>> {
830    let start_timestamp = extract_timestamp(options, COPY_DATABASE_TIME_START_KEY, query_ctx)?;
831    let end_timestamp = extract_timestamp(options, COPY_DATABASE_TIME_END_KEY, query_ctx)?;
832    let time_range = match (start_timestamp, end_timestamp) {
833        (Some(start), Some(end)) => Some(TimestampRange::new(start, end).with_context(|| {
834            error::InvalidTimestampRangeSnafu {
835                start: start.to_iso8601_string(),
836                end: end.to_iso8601_string(),
837            }
838        })?),
839        (Some(start), None) => Some(TimestampRange::from_start(start)),
840        (None, Some(end)) => Some(TimestampRange::until_end(end, false)), // exclusive end
841        (None, None) => None,
842    };
843    Ok(time_range)
844}
845
846/// Extracts timestamp from a [HashMap<String, String>] with given key.
847fn extract_timestamp(
848    map: &OptionMap,
849    key: &str,
850    query_ctx: &QueryContextRef,
851) -> Result<Option<Timestamp>> {
852    map.get(key)
853        .map(|v| {
854            Timestamp::from_str(v, Some(&query_ctx.timezone()))
855                .map_err(|_| error::InvalidCopyParameterSnafu { key, value: v }.build())
856        })
857        .transpose()
858}
859
860fn idents_to_full_database_name(
861    obj_name: &ObjectName,
862    query_ctx: &QueryContextRef,
863) -> Result<(String, String)> {
864    match &obj_name.0[..] {
865        [database] => Ok((
866            query_ctx.current_catalog().to_owned(),
867            database.to_string_unquoted(),
868        )),
869        [catalog, database] => Ok((catalog.to_string_unquoted(), database.to_string_unquoted())),
870        _ => InvalidSqlSnafu {
871            err_msg: format!(
872                "expect database name to be <catalog>.<database>, <database>, found: {obj_name}",
873            ),
874        }
875        .fail(),
876    }
877}
878
879/// The [`Inserter`] implementation for the statement executor.
880pub struct InserterImpl {
881    statement_executor: StatementExecutorRef,
882    options: Option<InsertOptions>,
883}
884
885impl InserterImpl {
886    pub fn new(statement_executor: StatementExecutorRef, options: Option<InsertOptions>) -> Self {
887        Self {
888            statement_executor,
889            options,
890        }
891    }
892}
893
894#[async_trait::async_trait]
895impl Inserter for InserterImpl {
896    async fn insert_rows(
897        &self,
898        context: &client::inserter::Context<'_>,
899        requests: RowInsertRequests,
900    ) -> ClientResult<()> {
901        let mut ctx_builder = QueryContextBuilder::default()
902            .current_catalog(context.catalog.to_string())
903            .current_schema(context.schema.to_string());
904        if let Some(options) = self.options.as_ref() {
905            ctx_builder = ctx_builder
906                .set_extension(
907                    TTL_KEY.to_string(),
908                    format_duration(options.ttl).to_string(),
909                )
910                .set_extension(APPEND_MODE_KEY.to_string(), options.append_mode.to_string());
911        }
912        let query_ctx = ctx_builder.build().into();
913
914        self.statement_executor
915            .inserter()
916            .handle_row_inserts(
917                requests,
918                query_ctx,
919                self.statement_executor.as_ref(),
920                false,
921                false,
922            )
923            .await
924            .map_err(BoxedError::new)
925            .context(ClientExternalSnafu)
926            .map(|_| ())
927    }
928
929    fn set_options(&mut self, options: &InsertOptions) {
930        self.options = Some(*options);
931    }
932}
933
934#[cfg(test)]
935mod tests {
936    use std::assert_matches::assert_matches;
937    use std::collections::HashMap;
938
939    use common_time::range::TimestampRange;
940    use common_time::{Timestamp, Timezone};
941    use session::context::QueryContextBuilder;
942    use sql::statements::OptionMap;
943
944    use crate::error;
945    use crate::statement::copy_database::{
946        COPY_DATABASE_TIME_END_KEY, COPY_DATABASE_TIME_START_KEY,
947    };
948    use crate::statement::{timestamp_range_from_option_map, verify_time_related_format};
949
950    fn check_timestamp_range((start, end): (&str, &str)) -> error::Result<Option<TimestampRange>> {
951        let query_ctx = QueryContextBuilder::default()
952            .timezone(Timezone::from_tz_string("Asia/Shanghai").unwrap())
953            .build()
954            .into();
955        let map = OptionMap::from(
956            [
957                (COPY_DATABASE_TIME_START_KEY.to_string(), start.to_string()),
958                (COPY_DATABASE_TIME_END_KEY.to_string(), end.to_string()),
959            ]
960            .into_iter()
961            .collect::<HashMap<_, _>>(),
962        );
963        timestamp_range_from_option_map(&map, &query_ctx)
964    }
965
966    #[test]
967    fn test_timestamp_range_from_option_map() {
968        assert_eq!(
969            Some(
970                TimestampRange::new(
971                    Timestamp::new_second(1649635200),
972                    Timestamp::new_second(1649664000),
973                )
974                .unwrap(),
975            ),
976            check_timestamp_range(("2022-04-11 08:00:00", "2022-04-11 16:00:00"),).unwrap()
977        );
978
979        assert_matches!(
980            check_timestamp_range(("2022-04-11 08:00:00", "2022-04-11 07:00:00")).unwrap_err(),
981            error::Error::InvalidTimestampRange { .. }
982        );
983    }
984
985    #[test]
986    fn test_verify_timestamp_format() {
987        let map = OptionMap::from(
988            [
989                (
990                    common_datasource::file_format::TIMESTAMP_FORMAT.to_string(),
991                    "%Y-%m-%d %H:%M:%S".to_string(),
992                ),
993                (
994                    common_datasource::file_format::FORMAT_TYPE.to_string(),
995                    "csv".to_string(),
996                ),
997            ]
998            .into_iter()
999            .collect::<HashMap<_, _>>(),
1000        );
1001        assert!(verify_time_related_format(&map).is_ok());
1002
1003        let map = OptionMap::from(
1004            [
1005                (
1006                    common_datasource::file_format::TIMESTAMP_FORMAT.to_string(),
1007                    "%Y-%m-%d %H:%M:%S".to_string(),
1008                ),
1009                (
1010                    common_datasource::file_format::FORMAT_TYPE.to_string(),
1011                    "json".to_string(),
1012                ),
1013            ]
1014            .into_iter()
1015            .collect::<HashMap<_, _>>(),
1016        );
1017
1018        assert_matches!(
1019            verify_time_related_format(&map).unwrap_err(),
1020            error::Error::TimestampFormatNotSupported { .. }
1021        );
1022        let map = OptionMap::from(
1023            [
1024                (
1025                    common_datasource::file_format::TIMESTAMP_FORMAT.to_string(),
1026                    "%111112".to_string(),
1027                ),
1028                (
1029                    common_datasource::file_format::FORMAT_TYPE.to_string(),
1030                    "csv".to_string(),
1031                ),
1032            ]
1033            .into_iter()
1034            .collect::<HashMap<_, _>>(),
1035        );
1036
1037        assert_matches!(
1038            verify_time_related_format(&map).unwrap_err(),
1039            error::Error::InvalidCopyParameter { .. }
1040        );
1041    }
1042}