mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-05 21:02:58 +00:00
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 <jiachun_feng@proton.me> * chore: apply suggestion Co-authored-by: Jeremyhi <jiachun_feng@proton.me> * chore: apply suggestion Co-authored-by: Jeremyhi <jiachun_feng@proton.me> * fix: TYPE_NAME for DropFlowProcedure --------- Co-authored-by: Jeremyhi <jiachun_feng@proton.me>
This commit is contained in:
@@ -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;
|
||||
|
||||
@@ -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
src/common/meta/src/ddl/drop_view.rs
Normal file
242
src/common/meta/src/ddl/drop_view.rs
Normal file
@@ -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,
|
||||
}
|
||||
@@ -21,3 +21,4 @@ pub(crate) mod create_view;
|
||||
mod drop_database;
|
||||
mod drop_flow;
|
||||
mod drop_table;
|
||||
mod drop_view;
|
||||
|
||||
@@ -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
src/common/meta/src/ddl/tests/drop_view.rs
Normal file
180
src/common/meta/src/ddl/tests/drop_view.rs
Normal file
@@ -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);
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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 { .. }
|
||||
|
||||
@@ -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!(
|
||||
|
||||
@@ -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;
|
||||
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
|
||||
@@ -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 { .. }
|
||||
|
||||
@@ -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());
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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),
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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 |
|
||||
+---------+
|
||||
|
||||
|
||||
@@ -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 DATABASE schema_for_view_test;
|
||||
DROP TABLE test_table;
|
||||
|
||||
SELECT * FROM test_view LIMIT 10;
|
||||
|
||||
SHOW TABLES;
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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 |
|
||||
+---------+
|
||||
|
||||
|
||||
@@ -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;
|
||||
|
||||
DROP DATABASE schema_for_view_test;
|
||||
SELECT * FROM v1;
|
||||
|
||||
--- view not exists ---
|
||||
DROP VIEW v2;
|
||||
|
||||
DROP VIEW IF EXISTS v2;
|
||||
|
||||
DROP TABLE t1;
|
||||
|
||||
SHOW TABLES;
|
||||
|
||||
Reference in New Issue
Block a user