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