From ab22bbac843790ab47b4acaf01e872a28cc36b80 Mon Sep 17 00:00:00 2001 From: dennis zhuang Date: Thu, 11 Jul 2024 12:53:54 -0700 Subject: [PATCH] feat: impl drop view (#4231) * feat: impl drop view * fix: metric name * fix: comments * test: add DropViewProcedure test * test: drop view meets a table * test: update sqlness tests by drop view * feat: apply suggestion from AI * chore: apply suggestion Co-authored-by: Jeremyhi * chore: apply suggestion Co-authored-by: Jeremyhi * chore: apply suggestion Co-authored-by: Jeremyhi * fix: TYPE_NAME for DropFlowProcedure --------- Co-authored-by: Jeremyhi --- src/common/meta/src/ddl.rs | 1 + src/common/meta/src/ddl/drop_flow.rs | 2 +- src/common/meta/src/ddl/drop_view.rs | 242 ++++++++++++++++++ src/common/meta/src/ddl/tests.rs | 1 + src/common/meta/src/ddl/tests/create_view.rs | 2 +- src/common/meta/src/ddl/tests/drop_view.rs | 180 +++++++++++++ src/common/meta/src/ddl_manager.rs | 50 +++- src/common/meta/src/error.rs | 2 +- src/common/meta/src/metrics.rs | 6 + src/common/meta/src/rpc/ddl.rs | 32 ++- src/frontend/src/instance.rs | 3 + src/operator/src/error.rs | 7 + src/operator/src/statement.rs | 15 ++ src/operator/src/statement/ddl.rs | 71 ++++- src/sql/src/parsers/drop_parser.rs | 81 +++++- src/sql/src/statements/drop.rs | 24 ++ src/sql/src/statements/statement.rs | 9 +- .../standalone/common/view/columns.result | 12 +- .../cases/standalone/common/view/columns.sql | 8 +- .../standalone/common/view/create.result | 128 ++++----- tests/cases/standalone/common/view/create.sql | 13 +- .../standalone/common/view/show_create.result | 16 +- .../standalone/common/view/show_create.sql | 9 +- .../cases/standalone/common/view/view.result | 35 ++- tests/cases/standalone/common/view/view.sql | 18 +- 25 files changed, 830 insertions(+), 137 deletions(-) create mode 100644 src/common/meta/src/ddl/drop_view.rs create mode 100644 src/common/meta/src/ddl/tests/drop_view.rs diff --git a/src/common/meta/src/ddl.rs b/src/common/meta/src/ddl.rs index e49648726872..7186997906e3 100644 --- a/src/common/meta/src/ddl.rs +++ b/src/common/meta/src/ddl.rs @@ -42,6 +42,7 @@ pub mod create_view; pub mod drop_database; pub mod drop_flow; pub mod drop_table; +pub mod drop_view; pub mod flow_meta; mod physical_table_metadata; pub mod table_meta; diff --git a/src/common/meta/src/ddl/drop_flow.rs b/src/common/meta/src/ddl/drop_flow.rs index eed57d446fbe..59b813c452ce 100644 --- a/src/common/meta/src/ddl/drop_flow.rs +++ b/src/common/meta/src/ddl/drop_flow.rs @@ -49,7 +49,7 @@ pub struct DropFlowProcedure { } impl DropFlowProcedure { - pub const TYPE_NAME: &'static str = "metasrv-procedure:DropFlow"; + pub const TYPE_NAME: &'static str = "metasrv-procedure::DropFlow"; pub fn new(cluster_id: ClusterId, task: DropFlowTask, context: DdlContext) -> Self { Self { diff --git a/src/common/meta/src/ddl/drop_view.rs b/src/common/meta/src/ddl/drop_view.rs new file mode 100644 index 000000000000..b56b39eee091 --- /dev/null +++ b/src/common/meta/src/ddl/drop_view.rs @@ -0,0 +1,242 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use async_trait::async_trait; +use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; +use common_procedure::{ + Context as ProcedureContext, LockKey, Procedure, Result as ProcedureResult, Status, +}; +use common_telemetry::info; +use serde::{Deserialize, Serialize}; +use snafu::{ensure, OptionExt, ResultExt}; +use strum::AsRefStr; +use table::metadata::{RawTableInfo, TableId, TableType}; +use table::table_reference::TableReference; + +use super::utils::handle_retry_error; +use crate::cache_invalidator::Context; +use crate::ddl::DdlContext; +use crate::error::{self, Result}; +use crate::instruction::CacheIdent; +use crate::key::table_name::TableNameKey; +use crate::lock_key::{CatalogLock, SchemaLock, TableLock}; +use crate::rpc::ddl::DropViewTask; +use crate::{metrics, ClusterId}; + +/// The procedure for dropping a view. +pub struct DropViewProcedure { + /// The context of procedure runtime. + pub(crate) context: DdlContext, + /// The serializable data. + pub(crate) data: DropViewData, +} + +impl DropViewProcedure { + pub const TYPE_NAME: &'static str = "metasrv-procedure::DropView"; + + pub fn new(cluster_id: ClusterId, task: DropViewTask, context: DdlContext) -> Self { + Self { + context, + data: DropViewData { + state: DropViewState::Prepare, + cluster_id, + task, + }, + } + } + + pub fn from_json(json: &str, context: DdlContext) -> ProcedureResult { + let data: DropViewData = serde_json::from_str(json).context(FromJsonSnafu)?; + + Ok(Self { context, data }) + } + + #[cfg(test)] + pub(crate) fn state(&self) -> DropViewState { + self.data.state + } + + /// Checks whether view exists. + /// - Early returns if view not exists and `drop_if_exists` is `true`. + /// - Throws an error if view not exists and `drop_if_exists` is `false`. + pub(crate) async fn on_prepare(&mut self) -> Result { + let table_ref = self.data.table_ref(); + + let exists = self + .context + .table_metadata_manager + .table_name_manager() + .exists(TableNameKey::new( + table_ref.catalog, + table_ref.schema, + table_ref.table, + )) + .await?; + + if !exists && self.data.task.drop_if_exists { + return Ok(Status::done()); + } + + ensure!( + exists, + error::ViewNotFoundSnafu { + view_name: table_ref.to_string(), + } + ); + + self.check_view_metadata().await?; + self.data.state = DropViewState::DeleteMetadata; + + Ok(Status::executing(true)) + } + + async fn check_view_metadata(&mut self) -> Result<()> { + let view_id = self.data.view_id(); + let table_info_value = self + .context + .table_metadata_manager + .table_info_manager() + .get(view_id) + .await? + .with_context(|| error::TableInfoNotFoundSnafu { + table: self.data.table_ref().to_string(), + })?; + + self.ensure_is_view(&table_info_value.table_info)?; + self.ensure_view_info_exists(view_id).await?; + + Ok(()) + } + + fn ensure_is_view(&self, table_info: &RawTableInfo) -> Result<()> { + ensure!( + table_info.table_type == TableType::View, + error::InvalidViewInfoSnafu { + err_msg: format!("{} is not a view", self.data.table_ref()), + } + ); + Ok(()) + } + + async fn ensure_view_info_exists(&self, view_id: TableId) -> Result<()> { + self.context + .table_metadata_manager + .view_info_manager() + .get(view_id) + .await? + .with_context(|| error::ViewNotFoundSnafu { + view_name: self.data.table_ref().to_string(), + })?; + Ok(()) + } + + async fn on_delete_metadata(&mut self) -> Result { + let view_id = self.data.view_id(); + self.context + .table_metadata_manager + .destroy_view_info(view_id, &self.data.table_ref().into()) + .await?; + + info!("Deleted view metadata for view {view_id}"); + + self.data.state = DropViewState::InvalidateViewCache; + Ok(Status::executing(true)) + } + + async fn on_broadcast(&mut self) -> Result { + let view_id = self.data.view_id(); + let ctx = Context { + subject: Some("Invalidate view cache by dropping view".to_string()), + }; + + self.context + .cache_invalidator + .invalidate( + &ctx, + &[ + CacheIdent::TableId(view_id), + CacheIdent::TableName(self.data.table_ref().into()), + ], + ) + .await?; + + Ok(Status::done()) + } +} + +#[async_trait] +impl Procedure for DropViewProcedure { + fn type_name(&self) -> &str { + Self::TYPE_NAME + } + + async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult { + let state = &self.data.state; + let _timer = metrics::METRIC_META_PROCEDURE_DROP_VIEW + .with_label_values(&[state.as_ref()]) + .start_timer(); + + match self.data.state { + DropViewState::Prepare => self.on_prepare().await, + DropViewState::DeleteMetadata => self.on_delete_metadata().await, + DropViewState::InvalidateViewCache => self.on_broadcast().await, + } + .map_err(handle_retry_error) + } + + fn dump(&self) -> ProcedureResult { + serde_json::to_string(&self.data).context(ToJsonSnafu) + } + + fn lock_key(&self) -> LockKey { + let table_ref = &self.data.table_ref(); + let view_id = self.data.view_id(); + let lock_key = vec![ + CatalogLock::Read(table_ref.catalog).into(), + SchemaLock::read(table_ref.catalog, table_ref.schema).into(), + TableLock::Write(view_id).into(), + ]; + + LockKey::new(lock_key) + } +} + +/// The serializable data +#[derive(Debug, Serialize, Deserialize)] +pub(crate) struct DropViewData { + state: DropViewState, + cluster_id: ClusterId, + task: DropViewTask, +} + +impl DropViewData { + fn table_ref(&self) -> TableReference { + self.task.table_ref() + } + + fn view_id(&self) -> TableId { + self.task.view_id + } +} + +/// The state of drop view +#[derive(Debug, Serialize, Deserialize, AsRefStr, PartialEq, Clone, Copy)] +pub(crate) enum DropViewState { + /// Prepares to drop the view + Prepare, + /// Deletes metadata + DeleteMetadata, + /// Invalidate view cache + InvalidateViewCache, +} diff --git a/src/common/meta/src/ddl/tests.rs b/src/common/meta/src/ddl/tests.rs index 9a0db96a37e0..0700259cf844 100644 --- a/src/common/meta/src/ddl/tests.rs +++ b/src/common/meta/src/ddl/tests.rs @@ -21,3 +21,4 @@ pub(crate) mod create_view; mod drop_database; mod drop_flow; mod drop_table; +mod drop_view; diff --git a/src/common/meta/src/ddl/tests/create_view.rs b/src/common/meta/src/ddl/tests/create_view.rs index f89caa24d975..53d19c8a2d54 100644 --- a/src/common/meta/src/ddl/tests/create_view.rs +++ b/src/common/meta/src/ddl/tests/create_view.rs @@ -32,7 +32,7 @@ use crate::error::Error; use crate::rpc::ddl::CreateViewTask; use crate::test_util::{new_ddl_context, MockDatanodeManager}; -fn test_table_names() -> HashSet { +pub(crate) fn test_table_names() -> HashSet { let mut set = HashSet::new(); set.insert(table::table_name::TableName { catalog_name: "greptime".to_string(), diff --git a/src/common/meta/src/ddl/tests/drop_view.rs b/src/common/meta/src/ddl/tests/drop_view.rs new file mode 100644 index 000000000000..1e0cb668597a --- /dev/null +++ b/src/common/meta/src/ddl/tests/drop_view.rs @@ -0,0 +1,180 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::Arc; + +use common_error::ext::ErrorExt; +use common_error::status_code::StatusCode; +use common_procedure_test::execute_procedure_until_done; +use store_api::storage::TableId; + +use crate::ddl::drop_view::{DropViewProcedure, DropViewState}; +use crate::ddl::test_util::create_table::test_create_table_task; +use crate::ddl::tests::create_view::{test_create_view_task, test_table_names}; +use crate::key::table_route::TableRouteValue; +use crate::rpc::ddl::DropViewTask; +use crate::test_util::{new_ddl_context, MockDatanodeManager}; + +fn new_drop_view_task(view: &str, view_id: TableId, drop_if_exists: bool) -> DropViewTask { + DropViewTask { + catalog: "greptime".to_string(), + schema: "public".to_string(), + view: view.to_string(), + view_id, + drop_if_exists, + } +} + +#[tokio::test] +async fn test_on_prepare_view_not_exists_err() { + let node_manager = Arc::new(MockDatanodeManager::new(())); + let ddl_context = new_ddl_context(node_manager); + let cluster_id = 1; + let view_id = 1024; + let mut task = test_create_view_task("foo"); + task.view_info.ident.table_id = view_id; + + ddl_context + .table_metadata_manager + .create_view_metadata( + task.view_info.clone(), + task.create_view.logical_plan.clone(), + test_table_names(), + vec!["a".to_string()], + vec!["number".to_string()], + "the definition".to_string(), + ) + .await + .unwrap(); + + let task = new_drop_view_task("bar", view_id, false); + let mut procedure = DropViewProcedure::new(cluster_id, task, ddl_context); + let err = procedure.on_prepare().await.unwrap_err(); + assert_eq!(err.status_code(), StatusCode::TableNotFound); +} + +#[tokio::test] +async fn test_on_prepare_not_view_err() { + let node_manager = Arc::new(MockDatanodeManager::new(())); + let ddl_context = new_ddl_context(node_manager); + let cluster_id = 1; + let view_id = 1024; + let view_name = "foo"; + let task = test_create_table_task(view_name, view_id); + // Create a table, not a view. + ddl_context + .table_metadata_manager + .create_table_metadata( + task.table_info.clone(), + TableRouteValue::physical(vec![]), + HashMap::new(), + ) + .await + .unwrap(); + + let task = new_drop_view_task(view_name, view_id, false); + let mut procedure = DropViewProcedure::new(cluster_id, task, ddl_context); + // It's not a view, expect error + let err = procedure.on_prepare().await.unwrap_err(); + assert_eq!(err.status_code(), StatusCode::InvalidArguments); +} + +#[tokio::test] +async fn test_on_prepare_success() { + let node_manager = Arc::new(MockDatanodeManager::new(())); + let ddl_context = new_ddl_context(node_manager); + let cluster_id = 1; + let view_id = 1024; + let view_name = "foo"; + let mut task = test_create_view_task("foo"); + task.view_info.ident.table_id = view_id; + + ddl_context + .table_metadata_manager + .create_view_metadata( + task.view_info.clone(), + task.create_view.logical_plan.clone(), + test_table_names(), + vec!["a".to_string()], + vec!["number".to_string()], + "the definition".to_string(), + ) + .await + .unwrap(); + + let task = new_drop_view_task("bar", view_id, true); + // Drop if exists + let mut procedure = DropViewProcedure::new(cluster_id, task, ddl_context.clone()); + procedure.on_prepare().await.unwrap(); + + let task = new_drop_view_task(view_name, view_id, false); + // Prepare success + let mut procedure = DropViewProcedure::new(cluster_id, task, ddl_context); + procedure.on_prepare().await.unwrap(); + assert_eq!(DropViewState::DeleteMetadata, procedure.state()); +} + +#[tokio::test] +async fn test_drop_view_success() { + let node_manager = Arc::new(MockDatanodeManager::new(())); + let ddl_context = new_ddl_context(node_manager); + let cluster_id = 1; + let view_id = 1024; + let view_name = "foo"; + let mut task = test_create_view_task("foo"); + task.view_info.ident.table_id = view_id; + + ddl_context + .table_metadata_manager + .create_view_metadata( + task.view_info.clone(), + task.create_view.logical_plan.clone(), + test_table_names(), + vec!["a".to_string()], + vec!["number".to_string()], + "the definition".to_string(), + ) + .await + .unwrap(); + + assert!(ddl_context + .table_metadata_manager + .view_info_manager() + .get(view_id) + .await + .unwrap() + .is_some()); + + let task = new_drop_view_task(view_name, view_id, false); + // Prepare success + let mut procedure = DropViewProcedure::new(cluster_id, task, ddl_context.clone()); + execute_procedure_until_done(&mut procedure).await; + assert_eq!(DropViewState::InvalidateViewCache, procedure.state()); + + // Assert view info is removed + assert!(ddl_context + .table_metadata_manager + .view_info_manager() + .get(view_id) + .await + .unwrap() + .is_none()); + + // Drop again + let task = new_drop_view_task(view_name, view_id, false); + let mut procedure = DropViewProcedure::new(cluster_id, task, ddl_context); + let err = procedure.on_prepare().await.unwrap_err(); + assert_eq!(err.status_code(), StatusCode::TableNotFound); +} diff --git a/src/common/meta/src/ddl_manager.rs b/src/common/meta/src/ddl_manager.rs index b9adcc9fb8e8..1b561b30430a 100644 --- a/src/common/meta/src/ddl_manager.rs +++ b/src/common/meta/src/ddl_manager.rs @@ -33,6 +33,7 @@ use crate::ddl::create_view::CreateViewProcedure; use crate::ddl::drop_database::DropDatabaseProcedure; use crate::ddl::drop_flow::DropFlowProcedure; use crate::ddl::drop_table::DropTableProcedure; +use crate::ddl::drop_view::DropViewProcedure; use crate::ddl::truncate_table::TruncateTableProcedure; use crate::ddl::{utils, DdlContext, ExecutorContext, ProcedureExecutor}; use crate::error::{ @@ -50,8 +51,8 @@ use crate::rpc::ddl::DdlTask::{ }; use crate::rpc::ddl::{ AlterTableTask, CreateDatabaseTask, CreateFlowTask, CreateTableTask, CreateViewTask, - DropDatabaseTask, DropFlowTask, DropTableTask, QueryContext, SubmitDdlTaskRequest, - SubmitDdlTaskResponse, TruncateTableTask, + DropDatabaseTask, DropFlowTask, DropTableTask, DropViewTask, QueryContext, + SubmitDdlTaskRequest, SubmitDdlTaskResponse, TruncateTableTask, }; use crate::rpc::procedure; use crate::rpc::procedure::{MigrateRegionRequest, MigrateRegionResponse, ProcedureStateResponse}; @@ -131,7 +132,8 @@ impl DdlManager { DropFlowProcedure, TruncateTableProcedure, CreateDatabaseProcedure, - DropDatabaseProcedure + DropDatabaseProcedure, + DropViewProcedure ); for (type_name, loader_factory) in loaders { @@ -306,8 +308,8 @@ impl DdlManager { self.submit_procedure(procedure_with_id).await } - #[tracing::instrument(skip_all)] /// Submits and executes a drop flow task. + #[tracing::instrument(skip_all)] pub async fn submit_drop_flow_task( &self, cluster_id: ClusterId, @@ -320,6 +322,20 @@ impl DdlManager { self.submit_procedure(procedure_with_id).await } + /// Submits and executes a drop view task. + #[tracing::instrument(skip_all)] + pub async fn submit_drop_view_task( + &self, + cluster_id: ClusterId, + drop_view: DropViewTask, + ) -> Result<(ProcedureId, Option)> { + let context = self.create_context(); + let procedure = DropViewProcedure::new(cluster_id, drop_view, context); + let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); + + self.submit_procedure(procedure_with_id).await + } + /// Submits and executes a truncate table task. #[tracing::instrument(skip_all)] pub async fn submit_truncate_table_task( @@ -599,6 +615,28 @@ async fn handle_drop_flow_task( }) } +async fn handle_drop_view_task( + ddl_manager: &DdlManager, + cluster_id: ClusterId, + drop_view_task: DropViewTask, +) -> Result { + let (id, _) = ddl_manager + .submit_drop_view_task(cluster_id, drop_view_task.clone()) + .await?; + + let procedure_id = id.to_string(); + info!( + "View {}({}) is dropped via procedure_id {id:?}", + drop_view_task.table_ref(), + drop_view_task.view_id, + ); + + Ok(SubmitDdlTaskResponse { + key: procedure_id.into(), + ..Default::default() + }) +} + async fn handle_create_flow_task( ddl_manager: &DdlManager, cluster_id: ClusterId, @@ -750,8 +788,8 @@ impl ProcedureExecutor for DdlManager { CreateView(create_view_task) => { handle_create_view_task(self, cluster_id, create_view_task).await } - DropView(_create_view_task) => { - todo!("implemented in the following PR"); + DropView(drop_view_task) => { + handle_drop_view_task(self, cluster_id, drop_view_task).await } } } diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index 53a8eb0aacc5..c8867fd623cd 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -666,7 +666,6 @@ impl ErrorExt for Error { | RouteInfoCorrupted { .. } | InvalidProtoMsg { .. } | InvalidTableMetadata { .. } - | InvalidViewInfo { .. } | MoveRegion { .. } | Unexpected { .. } | TableInfoNotFound { .. } @@ -706,6 +705,7 @@ impl ErrorExt for Error { | Unsupported { .. } => StatusCode::Internal, ProcedureNotFound { .. } + | InvalidViewInfo { .. } | PrimaryKeyNotFound { .. } | EmptyKey { .. } | InvalidEngineType { .. } diff --git a/src/common/meta/src/metrics.rs b/src/common/meta/src/metrics.rs index 0aa1d9a3cc6e..4626074f73f5 100644 --- a/src/common/meta/src/metrics.rs +++ b/src/common/meta/src/metrics.rs @@ -55,6 +55,12 @@ lazy_static! { "greptime_meta_procedure_drop_flow", "meta procedure drop flow", &["step"] + ) + .unwrap(); + pub static ref METRIC_META_PROCEDURE_DROP_VIEW: HistogramVec = register_histogram_vec!( + "greptime_meta_procedure_drop_view", + "meta procedure drop view", + &["step"] ) .unwrap(); pub static ref METRIC_META_PROCEDURE_CREATE_TABLES: HistogramVec = register_histogram_vec!( diff --git a/src/common/meta/src/rpc/ddl.rs b/src/common/meta/src/rpc/ddl.rs index d38e45d769e0..b5cf693c9a1a 100644 --- a/src/common/meta/src/rpc/ddl.rs +++ b/src/common/meta/src/rpc/ddl.rs @@ -64,14 +64,22 @@ pub enum DdlTask { } impl DdlTask { + /// Creates a [`DdlTask`] to create a flow. pub fn new_create_flow(expr: CreateFlowTask) -> Self { DdlTask::CreateFlow(expr) } + /// Creates a [`DdlTask`] to drop a flow. pub fn new_drop_flow(expr: DropFlowTask) -> Self { DdlTask::DropFlow(expr) } + /// Creates a [`DdlTask`] to drop a view. + pub fn new_drop_view(expr: DropViewTask) -> Self { + DdlTask::DropView(expr) + } + + /// Creates a [`DdlTask`] to create a table. pub fn new_create_table( expr: CreateTableExpr, partitions: Vec, @@ -80,6 +88,7 @@ impl DdlTask { DdlTask::CreateTable(CreateTableTask::new(expr, partitions, table_info)) } + /// Creates a [`DdlTask`] to create several logical tables. pub fn new_create_logical_tables(table_data: Vec<(CreateTableExpr, RawTableInfo)>) -> Self { DdlTask::CreateLogicalTables( table_data @@ -89,6 +98,7 @@ impl DdlTask { ) } + /// Creates a [`DdlTask`] to alter several logical tables. pub fn new_alter_logical_tables(table_data: Vec) -> Self { DdlTask::AlterLogicalTables( table_data @@ -98,6 +108,7 @@ impl DdlTask { ) } + /// Creates a [`DdlTask`] to drop a table. pub fn new_drop_table( catalog: String, schema: String, @@ -114,6 +125,7 @@ impl DdlTask { }) } + /// Creates a [`DdlTask`] to create a database. pub fn new_create_database( catalog: String, schema: String, @@ -128,6 +140,7 @@ impl DdlTask { }) } + /// Creates a [`DdlTask`] to drop a database. pub fn new_drop_database(catalog: String, schema: String, drop_if_exists: bool) -> Self { DdlTask::DropDatabase(DropDatabaseTask { catalog, @@ -136,10 +149,12 @@ impl DdlTask { }) } + /// Creates a [`DdlTask`] to alter a table. pub fn new_alter_table(alter_table: AlterExpr) -> Self { DdlTask::AlterTable(AlterTableTask { alter_table }) } + /// Creates a [`DdlTask`] to truncate a table. pub fn new_truncate_table( catalog: String, schema: String, @@ -154,7 +169,7 @@ impl DdlTask { }) } - // Create a `[DdlTask::CreateView]` task. + /// Creates a [`DdlTask`] to create a view. pub fn new_create_view(create_view: CreateViewExpr, view_info: RawTableInfo) -> Self { DdlTask::CreateView(CreateViewTask { create_view, @@ -312,7 +327,7 @@ pub struct CreateViewTask { } impl CreateViewTask { - /// Returns the `[TableReference]` of view. + /// Returns the [`TableReference`] of view. pub fn table_ref(&self) -> TableReference { TableReference { catalog: &self.create_view.catalog_name, @@ -415,7 +430,7 @@ impl<'de> Deserialize<'de> for CreateViewTask { } /// A `DROP VIEW` task. -#[derive(Debug, PartialEq, Clone)] +#[derive(Debug, PartialEq, Clone, Serialize, Deserialize)] pub struct DropViewTask { pub catalog: String, pub schema: String, @@ -424,6 +439,17 @@ pub struct DropViewTask { pub drop_if_exists: bool, } +impl DropViewTask { + /// Returns the [`TableReference`] of view. + pub fn table_ref(&self) -> TableReference { + TableReference { + catalog: &self.catalog, + schema: &self.schema, + table: &self.view, + } + } +} + impl TryFrom for DropViewTask { type Error = error::Error; diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index 19a70f33b42b..90f29c9382e9 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -491,6 +491,9 @@ pub fn check_permission( validate_param(table_name, query_ctx)?; } } + Statement::DropView(stmt) => { + validate_param(&stmt.view_name, query_ctx)?; + } Statement::ShowTables(stmt) => { validate_db_permission!(stmt, query_ctx); } diff --git a/src/operator/src/error.rs b/src/operator/src/error.rs index 1e9de3675727..4c657ceeb0eb 100644 --- a/src/operator/src/error.rs +++ b/src/operator/src/error.rs @@ -153,6 +153,12 @@ pub enum Error { view_name: String, expected: usize, actual: usize, + }, + + #[snafu(display("Invalid view \"{view_name}\": {msg}"))] + InvalidView { + msg: String, + view_name: String, #[snafu(implicit)] location: Location, }, @@ -777,6 +783,7 @@ impl ErrorExt for Error { | Error::UnsupportedRegionRequest { .. } | Error::InvalidTableName { .. } | Error::InvalidViewName { .. } + | Error::InvalidView { .. } | Error::InvalidExpr { .. } | Error::ViewColumnsMismatch { .. } | Error::InvalidViewStmt { .. } diff --git a/src/operator/src/statement.rs b/src/operator/src/statement.rs index 2dae5c709911..bb7fe25cc189 100644 --- a/src/operator/src/statement.rs +++ b/src/operator/src/statement.rs @@ -192,6 +192,21 @@ impl StatementExecutor { let _ = self.create_view(stmt, query_ctx).await?; Ok(Output::new_with_affected_rows(0)) } + Statement::DropView(stmt) => { + let (catalog_name, schema_name, view_name) = + table_idents_to_full_name(&stmt.view_name, &query_ctx) + .map_err(BoxedError::new) + .context(ExternalSnafu)?; + + self.drop_view( + catalog_name, + schema_name, + view_name, + stmt.drop_if_exists, + query_ctx, + ) + .await + } Statement::Alter(alter_table) => self.alter_table(alter_table, query_ctx).await, Statement::DropTable(stmt) => { let mut table_names = Vec::with_capacity(stmt.table_names().len()); diff --git a/src/operator/src/statement/ddl.rs b/src/operator/src/statement/ddl.rs index 070a73f3e638..6e646bc5fe5c 100644 --- a/src/operator/src/statement/ddl.rs +++ b/src/operator/src/statement/ddl.rs @@ -29,7 +29,8 @@ use common_meta::instruction::CacheIdent; use common_meta::key::schema_name::{SchemaNameKey, SchemaNameValue}; use common_meta::key::NAME_PATTERN; use common_meta::rpc::ddl::{ - CreateFlowTask, DdlTask, DropFlowTask, SubmitDdlTaskRequest, SubmitDdlTaskResponse, + CreateFlowTask, DdlTask, DropFlowTask, DropViewTask, SubmitDdlTaskRequest, + SubmitDdlTaskResponse, }; use common_meta::rpc::router::{Partition, Partition as MetaPartition}; use common_query::Output; @@ -644,6 +645,74 @@ impl StatementExecutor { .context(error::ExecuteDdlSnafu) } + /// Drop a view + #[tracing::instrument(skip_all)] + pub(crate) async fn drop_view( + &self, + catalog: String, + schema: String, + view: String, + drop_if_exists: bool, + query_context: QueryContextRef, + ) -> Result { + let view_info = if let Some(view) = self + .catalog_manager + .table(&catalog, &schema, &view) + .await + .context(CatalogSnafu)? + { + view.table_info() + } else if drop_if_exists { + // DROP VIEW IF EXISTS meets view not found - ignored + return Ok(Output::new_with_affected_rows(0)); + } else { + return TableNotFoundSnafu { + table_name: format_full_table_name(&catalog, &schema, &view), + } + .fail(); + }; + + // Ensure the exists one is view, we can't drop other table types + ensure!( + view_info.table_type == TableType::View, + error::InvalidViewSnafu { + msg: "not a view", + view_name: format_full_table_name(&catalog, &schema, &view), + } + ); + + let view_id = view_info.table_id(); + + let task = DropViewTask { + catalog, + schema, + view, + view_id, + drop_if_exists, + }; + + self.drop_view_procedure(task, query_context).await?; + + Ok(Output::new_with_affected_rows(0)) + } + + /// Submit [DropViewTask] to procedure executor. + async fn drop_view_procedure( + &self, + expr: DropViewTask, + query_context: QueryContextRef, + ) -> Result { + let request = SubmitDdlTaskRequest { + query_context, + task: DdlTask::new_drop_view(expr), + }; + + self.procedure_executor + .submit_ddl_task(&ExecutorContext::default(), request) + .await + .context(error::ExecuteDdlSnafu) + } + #[tracing::instrument(skip_all)] pub async fn alter_logical_tables( &self, diff --git a/src/sql/src/parsers/drop_parser.rs b/src/sql/src/parsers/drop_parser.rs index a59f0d9d7105..110b000bb3f4 100644 --- a/src/sql/src/parsers/drop_parser.rs +++ b/src/sql/src/parsers/drop_parser.rs @@ -16,9 +16,9 @@ use snafu::{ensure, ResultExt}; use sqlparser::dialect::keywords::Keyword; use sqlparser::tokenizer::Token; -use crate::error::{self, InvalidTableNameSnafu, Result}; +use crate::error::{self, InvalidFlowNameSnafu, InvalidTableNameSnafu, Result}; use crate::parser::{ParserContext, FLOW}; -use crate::statements::drop::{DropDatabase, DropFlow, DropTable}; +use crate::statements::drop::{DropDatabase, DropFlow, DropTable, DropView}; use crate::statements::statement::Statement; /// DROP statement parser implementation @@ -28,6 +28,7 @@ impl<'a> ParserContext<'a> { match self.parser.peek_token().token { Token::Word(w) => match w.keyword { Keyword::TABLE => self.parse_drop_table(), + Keyword::VIEW => self.parse_drop_view(), Keyword::SCHEMA | Keyword::DATABASE => self.parse_drop_database(), Keyword::NoKeyword => { let uppercase = w.value.to_uppercase(); @@ -42,6 +43,31 @@ impl<'a> ParserContext<'a> { } } + fn parse_drop_view(&mut self) -> Result { + let _ = self.parser.next_token(); + + let if_exists = self.parser.parse_keywords(&[Keyword::IF, Keyword::EXISTS]); + let raw_view_ident = self + .parse_object_name() + .with_context(|_| error::UnexpectedSnafu { + sql: self.sql, + expected: "a view name", + actual: self.peek_token_as_string(), + })?; + let view_ident = Self::canonicalize_object_name(raw_view_ident); + ensure!( + !view_ident.0.is_empty(), + InvalidTableNameSnafu { + name: view_ident.to_string() + } + ); + + Ok(Statement::DropView(DropView { + view_name: view_ident, + drop_if_exists: if_exists, + })) + } + fn parse_drop_flow(&mut self) -> Result { let _ = self.parser.next_token(); @@ -56,7 +82,7 @@ impl<'a> ParserContext<'a> { let flow_ident = Self::canonicalize_object_name(raw_flow_ident); ensure!( !flow_ident.0.is_empty(), - InvalidTableNameSnafu { + InvalidFlowNameSnafu { name: flow_ident.to_string() } ); @@ -263,4 +289,53 @@ mod tests { )) ) } + + #[test] + pub fn test_drop_view() { + let sql = "DROP VIEW foo"; + let result = + ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default()); + let mut stmts: Vec = result.unwrap(); + let stmt = stmts.pop().unwrap(); + assert_eq!( + stmt, + Statement::DropView(DropView { + view_name: ObjectName(vec![Ident::new("foo")]), + drop_if_exists: false, + }) + ); + assert_eq!(sql, stmt.to_string()); + + let sql = "DROP VIEW greptime.public.foo"; + let result = + ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default()); + let mut stmts: Vec = result.unwrap(); + let stmt = stmts.pop().unwrap(); + assert_eq!( + stmt, + Statement::DropView(DropView { + view_name: ObjectName(vec![ + Ident::new("greptime"), + Ident::new("public"), + Ident::new("foo") + ]), + drop_if_exists: false, + }) + ); + assert_eq!(sql, stmt.to_string()); + + let sql = "DROP VIEW IF EXISTS foo"; + let result = + ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default()); + let mut stmts: Vec = result.unwrap(); + let stmt = stmts.pop().unwrap(); + assert_eq!( + stmt, + Statement::DropView(DropView { + view_name: ObjectName(vec![Ident::new("foo")]), + drop_if_exists: true, + }) + ); + assert_eq!(sql, stmt.to_string()); + } } diff --git a/src/sql/src/statements/drop.rs b/src/sql/src/statements/drop.rs index 2048e2c74bf9..a46450db78f7 100644 --- a/src/sql/src/statements/drop.rs +++ b/src/sql/src/statements/drop.rs @@ -137,6 +137,30 @@ impl Display for DropFlow { } } +/// `DROP VIEW` statement. +#[derive(Debug, Clone, PartialEq, Eq, Visit, VisitMut)] +pub struct DropView { + // The view name + pub view_name: ObjectName, + // drop view if exists + pub drop_if_exists: bool, +} + +impl Display for DropView { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "DROP VIEW{} {}", + if self.drop_if_exists { + " IF EXISTS" + } else { + "" + }, + self.view_name + ) + } +} + #[cfg(test)] mod tests { use std::assert_matches::assert_matches; diff --git a/src/sql/src/statements/statement.rs b/src/sql/src/statements/statement.rs index 4f6fa1cbcd6a..ab7f4ffba55e 100644 --- a/src/sql/src/statements/statement.rs +++ b/src/sql/src/statements/statement.rs @@ -25,7 +25,7 @@ use crate::statements::create::{ }; use crate::statements::delete::Delete; use crate::statements::describe::DescribeTable; -use crate::statements::drop::{DropDatabase, DropFlow, DropTable}; +use crate::statements::drop::{DropDatabase, DropFlow, DropTable, DropView}; use crate::statements::explain::Explain; use crate::statements::insert::Insert; use crate::statements::query::Query; @@ -55,14 +55,16 @@ pub enum Statement { CreateTableLike(CreateTableLike), // CREATE FLOW CreateFlow(CreateFlow), - // DROP FLOW - DropFlow(DropFlow), // CREATE VIEW ... AS CreateView(CreateView), // DROP TABLE DropTable(DropTable), // DROP DATABASE DropDatabase(DropDatabase), + // DROP FLOW + DropFlow(DropFlow), + // DROP View + DropView(DropView), // CREATE DATABASE CreateDatabase(CreateDatabase), /// ALTER TABLE @@ -119,6 +121,7 @@ impl Display for Statement { Statement::DropFlow(s) => s.fmt(f), Statement::DropTable(s) => s.fmt(f), Statement::DropDatabase(s) => s.fmt(f), + Statement::DropView(s) => s.fmt(f), Statement::CreateDatabase(s) => s.fmt(f), Statement::Alter(s) => s.fmt(f), Statement::ShowDatabases(s) => s.fmt(f), diff --git a/tests/cases/standalone/common/view/columns.result b/tests/cases/standalone/common/view/columns.result index 3393819c3b44..567b14c82c5e 100644 --- a/tests/cases/standalone/common/view/columns.result +++ b/tests/cases/standalone/common/view/columns.result @@ -1,11 +1,3 @@ -CREATE DATABASE schema_for_view_test; - -Affected Rows: 1 - -USE schema_for_view_test; - -Affected Rows: 0 - CREATE TABLE t1 (n INT, ts TIMESTAMP TIME INDEX); Affected Rows: 0 @@ -238,11 +230,11 @@ SELECT * FROM v1; | 1970-01-01T00:00:00.005 | | +-------------------------+---+ -USE public; +DROP VIEW v1; Affected Rows: 0 -DROP DATABASE schema_for_view_test; +DROP TABLE t1; Affected Rows: 0 diff --git a/tests/cases/standalone/common/view/columns.sql b/tests/cases/standalone/common/view/columns.sql index 04a42a8cd306..2aee8d8c6302 100644 --- a/tests/cases/standalone/common/view/columns.sql +++ b/tests/cases/standalone/common/view/columns.sql @@ -1,7 +1,3 @@ -CREATE DATABASE schema_for_view_test; - -USE schema_for_view_test; - CREATE TABLE t1 (n INT, ts TIMESTAMP TIME INDEX); INSERT INTO t1 VALUES (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (10, 10); @@ -59,6 +55,6 @@ ALTER TABLE t1 DROP COLUMN n; -- See https://github.com/apache/datafusion/issues/6489 SELECT * FROM v1; -USE public; +DROP VIEW v1; -DROP DATABASE schema_for_view_test; +DROP TABLE t1; diff --git a/tests/cases/standalone/common/view/create.result b/tests/cases/standalone/common/view/create.result index bfbff6b970eb..c1ef01117a46 100644 --- a/tests/cases/standalone/common/view/create.result +++ b/tests/cases/standalone/common/view/create.result @@ -1,12 +1,4 @@ --- test CREATE VIEW --- -CREATE DATABASE schema_for_view_test; - -Affected Rows: 1 - -USE schema_for_view_test; - -Affected Rows: 0 - CREATE TABLE test_table(a STRING, ts TIMESTAMP TIME INDEX); Affected Rows: 0 @@ -22,17 +14,17 @@ Error: 2000(InvalidSyntax), sql parser error: Expected SELECT, VALUES, or a subq --- Table already exists --- CREATE VIEW test_table as SELECT * FROM public.numbers; -Error: 4000(TableAlreadyExists), Table already exists: `greptime.schema_for_view_test.test_table` +Error: 4000(TableAlreadyExists), Table already exists: `greptime.public.test_table` --- Table already exists even when create_if_not_exists --- CREATE VIEW IF NOT EXISTS test_table as SELECT * FROM public.numbers; -Error: 4000(TableAlreadyExists), Table already exists: `greptime.schema_for_view_test.test_table` +Error: 4000(TableAlreadyExists), Table already exists: `greptime.public.test_table` --- Table already exists even when or_replace --- CREATE OR REPLACE VIEW test_table as SELECT * FROM public.numbers; -Error: 4000(TableAlreadyExists), Table already exists: `greptime.schema_for_view_test.test_table` +Error: 4000(TableAlreadyExists), Table already exists: `greptime.public.test_table` CREATE VIEW test_view as SELECT * FROM public.numbers; @@ -41,7 +33,7 @@ Affected Rows: 0 --- View already exists ---- CREATE VIEW test_view as SELECT * FROM public.numbers; -Error: 4000(TableAlreadyExists), View already exists: `greptime.schema_for_view_test.test_view` +Error: 4000(TableAlreadyExists), View already exists: `greptime.public.test_view` CREATE VIEW IF NOT EXISTS test_view as SELECT * FROM public.numbers; @@ -56,68 +48,70 @@ SHOW TABLES; +------------+ | Tables | +------------+ +| numbers | | test_table | | test_view | +------------+ SHOW FULL TABLES; -+------------+------------+ -| Tables | Table_type | -+------------+------------+ -| test_table | BASE TABLE | -| test_view | VIEW | -+------------+------------+ ++------------+-----------------+ +| Tables | Table_type | ++------------+-----------------+ +| numbers | LOCAL TEMPORARY | +| test_table | BASE TABLE | +| test_view | VIEW | ++------------+-----------------+ -- SQLNESS REPLACE (\s\d+\s) ID -- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{3}) DATETIME SELECT * FROM INFORMATION_SCHEMA.TABLES ORDER BY TABLE_NAME, TABLE_TYPE; -+---------------+----------------------+---------------------------------------+-----------------+----------+-------------+-----------------+--------------+------------------+----------------+-------------+---------+------------+------------+-----------+----------------+-------------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ -| table_catalog | table_schema | table_name | table_type | table_id | data_length | max_data_length | index_length | max_index_length | avg_row_length | engine | version | row_format | table_rows | data_free | auto_increment | create_time | update_time | check_time | table_collation | checksum | create_options | table_comment | temporary | -+---------------+----------------------+---------------------------------------+-----------------+----------+-------------+-----------------+--------------+------------------+----------------+-------------+---------+------------+------------+-----------+----------------+-------------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ -| greptime | information_schema | build_info | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | character_sets | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | check_constraints | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | cluster_info | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | collation_character_set_applicability | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | collations | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | column_privileges | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | column_statistics | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | columns | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | engines | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | events | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | files | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | global_status | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | key_column_usage | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | public | numbers | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | test_engine |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | optimizer_trace | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | parameters | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | partitions | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | profiling | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | referential_constraints | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | region_peers | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | routines | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | runtime_metrics | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | schema_privileges | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | schemata | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | session_status | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | table_constraints | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | table_privileges | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | information_schema | tables | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -| greptime | schema_for_view_test | test_table | BASE TABLE |ID |ID |ID |ID |ID |ID | mito |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | N | -| greptime | schema_for_view_test | test_view | VIEW |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | 1970-01-01T00:00:00 | | | |ID | | | N | -| greptime | information_schema | triggers | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | -+---------------+----------------------+---------------------------------------+-----------------+----------+-------------+-----------------+--------------+------------------+----------------+-------------+---------+------------+------------+-----------+----------------+-------------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ ++---------------+--------------------+---------------------------------------+-----------------+----------+-------------+-----------------+--------------+------------------+----------------+-------------+---------+------------+------------+-----------+----------------+-------------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ +| table_catalog | table_schema | table_name | table_type | table_id | data_length | max_data_length | index_length | max_index_length | avg_row_length | engine | version | row_format | table_rows | data_free | auto_increment | create_time | update_time | check_time | table_collation | checksum | create_options | table_comment | temporary | ++---------------+--------------------+---------------------------------------+-----------------+----------+-------------+-----------------+--------------+------------------+----------------+-------------+---------+------------+------------+-----------+----------------+-------------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ +| greptime | information_schema | build_info | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | character_sets | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | check_constraints | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | cluster_info | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | collation_character_set_applicability | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | collations | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | column_privileges | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | column_statistics | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | columns | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | engines | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | events | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | files | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | global_status | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | key_column_usage | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | public | numbers | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | test_engine |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | optimizer_trace | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | parameters | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | partitions | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | profiling | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | referential_constraints | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | region_peers | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | routines | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | runtime_metrics | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | schema_privileges | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | schemata | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | session_status | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | table_constraints | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | table_privileges | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | information_schema | tables | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | +| greptime | public | test_table | BASE TABLE |ID |ID |ID |ID |ID |ID | mito |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | N | +| greptime | public | test_view | VIEW |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | 1970-01-01T00:00:00 | | | |ID | | | N | +| greptime | information_schema | triggers | LOCAL TEMPORARY |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | DATETIME | | | |ID | | | Y | ++---------------+--------------------+---------------------------------------+-----------------+----------+-------------+-----------------+--------------+------------------+----------------+-------------+---------+------------+------------+-----------+----------------+-------------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ -- SQLNESS REPLACE (\s\d+\s) ID SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE = 'VIEW'; -+---------------+----------------------+------------+------------+----------+-------------+-----------------+--------------+------------------+----------------+--------+---------+------------+------------+-----------+----------------+---------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ -| table_catalog | table_schema | table_name | table_type | table_id | data_length | max_data_length | index_length | max_index_length | avg_row_length | engine | version | row_format | table_rows | data_free | auto_increment | create_time | update_time | check_time | table_collation | checksum | create_options | table_comment | temporary | -+---------------+----------------------+------------+------------+----------+-------------+-----------------+--------------+------------------+----------------+--------+---------+------------+------------+-----------+----------------+---------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ -| greptime | schema_for_view_test | test_view | VIEW |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | 1970-01-01T00:00:00 | | | |ID | | | N | -+---------------+----------------------+------------+------------+----------+-------------+-----------------+--------------+------------------+----------------+--------+---------+------------+------------+-----------+----------------+---------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ ++---------------+--------------+------------+------------+----------+-------------+-----------------+--------------+------------------+----------------+--------+---------+------------+------------+-----------+----------------+---------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ +| table_catalog | table_schema | table_name | table_type | table_id | data_length | max_data_length | index_length | max_index_length | avg_row_length | engine | version | row_format | table_rows | data_free | auto_increment | create_time | update_time | check_time | table_collation | checksum | create_options | table_comment | temporary | ++---------------+--------------+------------+------------+----------+-------------+-----------------+--------------+------------------+----------------+--------+---------+------------+------------+-----------+----------------+---------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ +| greptime | public | test_view | VIEW |ID |ID |ID |ID |ID |ID | |ID | Fixed |ID |ID |ID | 1970-01-01T00:00:00 | | | |ID | | | N | ++---------------+--------------+------------+------------+----------+-------------+-----------------+--------------+------------------+----------------+--------+---------+------------+------------+-----------+----------------+---------------------+-------------+------------+-----------------+----------+----------------+---------------+-----------+ SHOW COLUMNS FROM test_view; @@ -151,11 +145,23 @@ SELECT * FROM test_view LIMIT 10; | 9 | +--------+ -USE public; +DROP VIEW test_view; Affected Rows: 0 -DROP DATABASE schema_for_view_test; +DROP TABLE test_table; Affected Rows: 0 +SELECT * FROM test_view LIMIT 10; + +Error: 3000(PlanQuery), Failed to plan SQL: Error during planning: Table not found: greptime.public.test_view + +SHOW TABLES; + ++---------+ +| Tables | ++---------+ +| numbers | ++---------+ + diff --git a/tests/cases/standalone/common/view/create.sql b/tests/cases/standalone/common/view/create.sql index c4d1a57b5fcb..e7c25c5bed88 100644 --- a/tests/cases/standalone/common/view/create.sql +++ b/tests/cases/standalone/common/view/create.sql @@ -1,9 +1,4 @@ --- test CREATE VIEW --- - -CREATE DATABASE schema_for_view_test; - -USE schema_for_view_test; - CREATE TABLE test_table(a STRING, ts TIMESTAMP TIME INDEX); CREATE VIEW test_view; @@ -47,6 +42,10 @@ SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = 'test_view'; SELECT * FROM test_view LIMIT 10; -USE public; +DROP VIEW test_view; + +DROP TABLE test_table; -DROP DATABASE schema_for_view_test; +SELECT * FROM test_view LIMIT 10; + +SHOW TABLES; diff --git a/tests/cases/standalone/common/view/show_create.result b/tests/cases/standalone/common/view/show_create.result index 33c2055909fd..9960afc9609d 100644 --- a/tests/cases/standalone/common/view/show_create.result +++ b/tests/cases/standalone/common/view/show_create.result @@ -1,11 +1,3 @@ -CREATE DATABASE schema_for_view_test; - -Affected Rows: 1 - -USE schema_for_view_test; - -Affected Rows: 0 - CREATE TABLE t1(a INT, b STRING, c TIMESTAMP TIME INDEX); Affected Rows: 0 @@ -126,11 +118,15 @@ SELECT * FROM v1; | 1970-01-01T00:00:00.004 | +-------------------------+ -USE public; +DROP VIEW v1; Affected Rows: 0 -DROP DATABASE schema_for_view_test; +DROP TABLE t1; Affected Rows: 0 +SHOW CREATE VIEW v1; + +Error: 4001(TableNotFound), View not found: v1 + diff --git a/tests/cases/standalone/common/view/show_create.sql b/tests/cases/standalone/common/view/show_create.sql index 4c0e16073f8c..75736eed883d 100644 --- a/tests/cases/standalone/common/view/show_create.sql +++ b/tests/cases/standalone/common/view/show_create.sql @@ -1,7 +1,3 @@ -CREATE DATABASE schema_for_view_test; - -USE schema_for_view_test; - CREATE TABLE t1(a INT, b STRING, c TIMESTAMP TIME INDEX); INSERT INTO t1 VALUES (41, "hello", 1), (42, "world", 2), (43, "greptime", 3); @@ -38,7 +34,8 @@ SHOW CREATE VIEW v1; SELECT * FROM v1; +DROP VIEW v1; -USE public; +DROP TABLE t1; -DROP DATABASE schema_for_view_test; +SHOW CREATE VIEW v1; diff --git a/tests/cases/standalone/common/view/view.result b/tests/cases/standalone/common/view/view.result index 562b7dcd5523..c9346925e12a 100644 --- a/tests/cases/standalone/common/view/view.result +++ b/tests/cases/standalone/common/view/view.result @@ -1,12 +1,4 @@ -- From: https://github.com/duckdb/duckdb/blob/main/test/sql/catalog/view/test_view.test -- -CREATE DATABASE schema_for_view_test; - -Affected Rows: 1 - -USE schema_for_view_test; - -Affected Rows: 0 - CREATE TABLE t1(i TIMESTAMP TIME INDEX); Affected Rows: 0 @@ -58,13 +50,34 @@ Error: 1004(InvalidArguments), Invalid SQL, error: column count mismatch, column CREATE VIEW v1 AS SELECT * FROM dontexist; -Error: 3000(PlanQuery), Failed to plan SQL: Error during planning: Table not found: greptime.schema_for_view_test.dontexist +Error: 3000(PlanQuery), Failed to plan SQL: Error during planning: Table not found: greptime.public.dontexist -USE public; +DROP VIEW v1; Affected Rows: 0 -DROP DATABASE schema_for_view_test; +SELECT * FROM v1; + +Error: 3000(PlanQuery), Failed to plan SQL: Error during planning: Table not found: greptime.public.v1 + +--- view not exists --- +DROP VIEW v2; + +Error: 4001(TableNotFound), Table not found: greptime.public.v2 + +DROP VIEW IF EXISTS v2; Affected Rows: 0 +DROP TABLE t1; + +Affected Rows: 0 + +SHOW TABLES; + ++---------+ +| Tables | ++---------+ +| numbers | ++---------+ + diff --git a/tests/cases/standalone/common/view/view.sql b/tests/cases/standalone/common/view/view.sql index d6e44f54c1f7..84cf1cdd9da7 100644 --- a/tests/cases/standalone/common/view/view.sql +++ b/tests/cases/standalone/common/view/view.sql @@ -1,9 +1,4 @@ -- From: https://github.com/duckdb/duckdb/blob/main/test/sql/catalog/view/test_view.test -- - -CREATE DATABASE schema_for_view_test; - -USE schema_for_view_test; - CREATE TABLE t1(i TIMESTAMP TIME INDEX); INSERT INTO t1 VALUES (41), (42), (43); @@ -35,6 +30,15 @@ INSERT INTO v1 VALUES (1); CREATE VIEW v1 AS SELECT * FROM dontexist; -USE public; +DROP VIEW v1; + +SELECT * FROM v1; + +--- view not exists --- +DROP VIEW v2; + +DROP VIEW IF EXISTS v2; + +DROP TABLE t1; -DROP DATABASE schema_for_view_test; +SHOW TABLES;