Skip to content

Commit

Permalink
feat: impl drop view (#4231)
Browse files Browse the repository at this point in the history
* 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 <[email protected]>

* chore: apply suggestion

Co-authored-by: Jeremyhi <[email protected]>

* chore: apply suggestion

Co-authored-by: Jeremyhi <[email protected]>

* fix: TYPE_NAME for DropFlowProcedure

---------

Co-authored-by: Jeremyhi <[email protected]>
killme2008 and fengjiachun authored Jul 11, 2024

Verified

This commit was created on GitHub.com and signed with GitHub’s verified signature.
1 parent 7ad248d commit ab22bba
Showing 25 changed files with 830 additions and 137 deletions.
1 change: 1 addition & 0 deletions src/common/meta/src/ddl.rs
Original file line number Diff line number Diff line change
@@ -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;
2 changes: 1 addition & 1 deletion src/common/meta/src/ddl/drop_flow.rs
Original file line number Diff line number Diff line change
@@ -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 {
242 changes: 242 additions & 0 deletions src/common/meta/src/ddl/drop_view.rs
Original file line number Diff line number Diff line change
@@ -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<Self> {
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<Status> {
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<Status> {
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<Status> {
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<Status> {
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<String> {
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,
}
1 change: 1 addition & 0 deletions src/common/meta/src/ddl/tests.rs
Original file line number Diff line number Diff line change
@@ -21,3 +21,4 @@ pub(crate) mod create_view;
mod drop_database;
mod drop_flow;
mod drop_table;
mod drop_view;
2 changes: 1 addition & 1 deletion src/common/meta/src/ddl/tests/create_view.rs
Original file line number Diff line number Diff line change
@@ -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<table::table_name::TableName> {
pub(crate) fn test_table_names() -> HashSet<table::table_name::TableName> {
let mut set = HashSet::new();
set.insert(table::table_name::TableName {
catalog_name: "greptime".to_string(),
180 changes: 180 additions & 0 deletions src/common/meta/src/ddl/tests/drop_view.rs
Original file line number Diff line number Diff line change
@@ -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);
}
50 changes: 44 additions & 6 deletions src/common/meta/src/ddl_manager.rs
Original file line number Diff line number Diff line change
@@ -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<Output>)> {
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<SubmitDdlTaskResponse> {
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
}
}
}
2 changes: 1 addition & 1 deletion src/common/meta/src/error.rs
Original file line number Diff line number Diff line change
@@ -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 { .. }
6 changes: 6 additions & 0 deletions src/common/meta/src/metrics.rs
Original file line number Diff line number Diff line change
@@ -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!(
32 changes: 29 additions & 3 deletions src/common/meta/src/rpc/ddl.rs
Original file line number Diff line number Diff line change
@@ -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<Partition>,
@@ -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<AlterExpr>) -> 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<PbDropViewTask> for DropViewTask {
type Error = error::Error;

3 changes: 3 additions & 0 deletions src/frontend/src/instance.rs
Original file line number Diff line number Diff line change
@@ -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);
}
7 changes: 7 additions & 0 deletions src/operator/src/error.rs
Original file line number Diff line number Diff line change
@@ -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 { .. }
15 changes: 15 additions & 0 deletions src/operator/src/statement.rs
Original file line number Diff line number Diff line change
@@ -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());
71 changes: 70 additions & 1 deletion src/operator/src/statement/ddl.rs
Original file line number Diff line number Diff line change
@@ -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<Output> {
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<SubmitDdlTaskResponse> {
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,
81 changes: 78 additions & 3 deletions src/sql/src/parsers/drop_parser.rs
Original file line number Diff line number Diff line change
@@ -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<Statement> {
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<Statement> {
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<Statement> = 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<Statement> = 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<Statement> = 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());
}
}
24 changes: 24 additions & 0 deletions src/sql/src/statements/drop.rs
Original file line number Diff line number Diff line change
@@ -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;
9 changes: 6 additions & 3 deletions src/sql/src/statements/statement.rs
Original file line number Diff line number Diff line change
@@ -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),
12 changes: 2 additions & 10 deletions tests/cases/standalone/common/view/columns.result
Original file line number Diff line number Diff line change
@@ -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

8 changes: 2 additions & 6 deletions tests/cases/standalone/common/view/columns.sql
Original file line number Diff line number Diff line change
@@ -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;
128 changes: 67 additions & 61 deletions tests/cases/standalone/common/view/create.result

Large diffs are not rendered by default.

13 changes: 6 additions & 7 deletions tests/cases/standalone/common/view/create.sql
Original file line number Diff line number Diff line change
@@ -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;
16 changes: 6 additions & 10 deletions tests/cases/standalone/common/view/show_create.result
Original file line number Diff line number Diff line change
@@ -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

9 changes: 3 additions & 6 deletions tests/cases/standalone/common/view/show_create.sql
Original file line number Diff line number Diff line change
@@ -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;
35 changes: 24 additions & 11 deletions tests/cases/standalone/common/view/view.result
Original file line number Diff line number Diff line change
@@ -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 |
+---------+

18 changes: 11 additions & 7 deletions tests/cases/standalone/common/view/view.sql
Original file line number Diff line number Diff line change
@@ -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;

0 comments on commit ab22bba

Please sign in to comment.