1mod 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#[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#[cfg(feature = "enterprise")]
128#[async_trait::async_trait]
129pub trait TriggerQuerier: Send + Sync {
130 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 "DATESTYLE" => set_datestyle(set_var.value, query_ctx)?,
485
486 "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 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 #[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 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 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
713fn 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
792fn 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
813fn 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)), (None, None) => None,
832 };
833 Ok(time_range)
834}
835
836fn 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
869pub 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}