feat: show create view and creating view with columns (#4086)

* feat: parse column names when creating view

* feat: save the view definition into view info

* feat: supports view columns and show create view

* feat: save plan columns for validation

* fix: typo

* chore: comments and style

* chore: apply suggestions

* test: assert CreateView display result

* chore: style

Co-authored-by: Weny Xu <wenymedia@gmail.com>

* chore: avoid the clone

Co-authored-by: Weny Xu <wenymedia@gmail.com>

* fix: compile error after rebeasing

---------

Co-authored-by: Weny Xu <wenymedia@gmail.com>
This commit is contained in:
dennis zhuang
2024-07-09 11:35:11 -07:00
committed by GitHub
parent 458e5d7e66
commit 33ed745049
33 changed files with 1372 additions and 159 deletions

2
Cargo.lock generated
View File

@@ -4208,7 +4208,7 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b"
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=b83f00958fe4cbc77b85b7407bca206e98bdc845#b83f00958fe4cbc77b85b7407bca206e98bdc845"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=5c801650435d464891114502539b701c77a1b914#5c801650435d464891114502539b701c77a1b914"
dependencies = [
"prost 0.12.6",
"serde",

View File

@@ -119,7 +119,7 @@ etcd-client = { version = "0.13" }
fst = "0.4.7"
futures = "0.3"
futures-util = "0.3"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "b83f00958fe4cbc77b85b7407bca206e98bdc845" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "5c801650435d464891114502539b701c77a1b914" }
humantime = "2.1"
humantime-serde = "1.1"
itertools = "0.10"

View File

@@ -114,6 +114,18 @@ pub enum Error {
location: Location,
},
#[snafu(display(
"View plan columns changed from: {} to: {}",
origin_names,
actual_names
))]
ViewPlanColumnsChanged {
origin_names: String,
actual_names: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to find table partitions"))]
FindPartitions { source: partition::error::Error },
@@ -173,6 +185,14 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to project view columns"))]
ProjectViewColumns {
#[snafu(source)]
error: DataFusionError,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Table metadata manager error"))]
TableMetadataManager {
source: common_meta::error::Error,
@@ -208,6 +228,21 @@ pub enum Error {
},
}
impl Error {
pub fn should_fail(&self) -> bool {
use Error::*;
matches!(
self,
GetViewCache { .. }
| ViewInfoNotFound { .. }
| DecodePlan { .. }
| ViewPlanColumnsChanged { .. }
| ProjectViewColumns { .. }
)
}
}
pub type Result<T> = std::result::Result<T, Error>;
impl ErrorExt for Error {
@@ -220,6 +255,8 @@ impl ErrorExt for Error {
| Error::CacheNotFound { .. }
| Error::CastManager { .. } => StatusCode::Unexpected,
Error::ViewPlanColumnsChanged { .. } => StatusCode::InvalidArguments,
Error::ViewInfoNotFound { .. } => StatusCode::TableNotFound,
Error::SystemCatalog { .. } => StatusCode::StorageUnavailable,
@@ -245,7 +282,9 @@ impl ErrorExt for Error {
}
Error::QueryAccessDenied { .. } => StatusCode::AccessDenied,
Error::Datafusion { .. } => StatusCode::EngineExecuteQuery,
Error::ProjectViewColumns { .. } | Error::Datafusion { .. } => {
StatusCode::EngineExecuteQuery
}
Error::TableMetadataManager { source, .. } => source.status_code(),
Error::GetViewCache { source, .. } | Error::GetTableCache { source, .. } => {
source.status_code()

View File

@@ -17,21 +17,24 @@ use std::sync::Arc;
use bytes::Bytes;
use common_catalog::format_full_table_name;
use common_query::logical_plan::SubstraitPlanDecoderRef;
use common_query::logical_plan::{rename_logical_plan_columns, SubstraitPlanDecoderRef};
use datafusion::common::{ResolvedTableReference, TableReference};
use datafusion::datasource::view::ViewTable;
use datafusion::datasource::{provider_as_source, TableProvider};
use datafusion::logical_expr::TableSource;
use itertools::Itertools;
use session::context::QueryContext;
use snafu::{ensure, OptionExt, ResultExt};
use table::metadata::TableType;
use table::table::adapter::DfTableProviderAdapter;
mod dummy_catalog;
use dummy_catalog::DummyCatalogList;
use table::TableRef;
use crate::error::{
CastManagerSnafu, DatafusionSnafu, DecodePlanSnafu, GetViewCacheSnafu, QueryAccessDeniedSnafu,
Result, TableNotExistSnafu, ViewInfoNotFoundSnafu,
CastManagerSnafu, DatafusionSnafu, DecodePlanSnafu, GetViewCacheSnafu, ProjectViewColumnsSnafu,
QueryAccessDeniedSnafu, Result, TableNotExistSnafu, ViewInfoNotFoundSnafu,
ViewPlanColumnsChangedSnafu,
};
use crate::kvbackend::KvBackendCatalogManager;
use crate::CatalogManagerRef;
@@ -43,6 +46,7 @@ pub struct DfTableSourceProvider {
default_catalog: String,
default_schema: String,
plan_decoder: SubstraitPlanDecoderRef,
enable_ident_normalization: bool,
}
impl DfTableSourceProvider {
@@ -51,6 +55,7 @@ impl DfTableSourceProvider {
disallow_cross_catalog_query: bool,
query_ctx: &QueryContext,
plan_decoder: SubstraitPlanDecoderRef,
enable_ident_normalization: bool,
) -> Self {
Self {
catalog_manager,
@@ -59,6 +64,7 @@ impl DfTableSourceProvider {
default_catalog: query_ctx.current_catalog().to_owned(),
default_schema: query_ctx.current_schema(),
plan_decoder,
enable_ident_normalization,
}
}
@@ -108,32 +114,7 @@ impl DfTableSourceProvider {
})?;
let provider: Arc<dyn TableProvider> = if table.table_info().table_type == TableType::View {
let catalog_manager = self
.catalog_manager
.as_any()
.downcast_ref::<KvBackendCatalogManager>()
.context(CastManagerSnafu)?;
let view_info = catalog_manager
.view_info_cache()?
.get(table.table_info().ident.table_id)
.await
.context(GetViewCacheSnafu)?
.context(ViewInfoNotFoundSnafu {
name: &table.table_info().name,
})?;
// Build the catalog list provider for deserialization.
let catalog_list = Arc::new(DummyCatalogList::new(self.catalog_manager.clone()));
let logical_plan = self
.plan_decoder
.decode(Bytes::from(view_info.view_info.clone()), catalog_list, true)
.await
.context(DecodePlanSnafu {
name: &table.table_info().name,
})?;
Arc::new(ViewTable::try_new(logical_plan, None).context(DatafusionSnafu)?)
self.create_view_provider(&table).await?
} else {
Arc::new(DfTableProviderAdapter::new(table))
};
@@ -143,6 +124,80 @@ impl DfTableSourceProvider {
let _ = self.resolved_tables.insert(resolved_name, source.clone());
Ok(source)
}
async fn create_view_provider(&self, table: &TableRef) -> Result<Arc<dyn TableProvider>> {
let catalog_manager = self
.catalog_manager
.as_any()
.downcast_ref::<KvBackendCatalogManager>()
.context(CastManagerSnafu)?;
let view_info = catalog_manager
.view_info_cache()?
.get(table.table_info().ident.table_id)
.await
.context(GetViewCacheSnafu)?
.context(ViewInfoNotFoundSnafu {
name: &table.table_info().name,
})?;
// Build the catalog list provider for deserialization.
let catalog_list = Arc::new(DummyCatalogList::new(self.catalog_manager.clone()));
let logical_plan = self
.plan_decoder
.decode(Bytes::from(view_info.view_info.clone()), catalog_list, true)
.await
.context(DecodePlanSnafu {
name: &table.table_info().name,
})?;
let columns: Vec<_> = view_info.columns.iter().map(|c| c.as_str()).collect();
let original_plan_columns: Vec<_> =
view_info.plan_columns.iter().map(|c| c.as_str()).collect();
let plan_columns: Vec<_> = logical_plan
.schema()
.columns()
.into_iter()
.map(|c| c.name)
.collect();
// Only check columns number, because substrait doesn't include aliases currently.
// See https://github.com/apache/datafusion/issues/10815#issuecomment-2158666881
// and https://github.com/apache/datafusion/issues/6489
// TODO(dennis): check column names
ensure!(
original_plan_columns.len() == plan_columns.len(),
ViewPlanColumnsChangedSnafu {
origin_names: original_plan_columns.iter().join(","),
actual_names: plan_columns.iter().join(","),
}
);
// We have to do `columns` projection here, because
// substrait doesn't include aliases neither for tables nor for columns:
// https://github.com/apache/datafusion/issues/10815#issuecomment-2158666881
let logical_plan = if !columns.is_empty() {
rename_logical_plan_columns(
self.enable_ident_normalization,
logical_plan,
plan_columns
.iter()
.map(|c| c.as_str())
.zip(columns.into_iter())
.collect(),
)
.context(ProjectViewColumnsSnafu)?
} else {
logical_plan
};
Ok(Arc::new(
ViewTable::try_new(logical_plan, Some(view_info.definition.to_string()))
.context(DatafusionSnafu)?,
))
}
}
#[cfg(test)]
@@ -162,6 +217,7 @@ mod tests {
true,
query_ctx,
DummyDecoder::arc(),
true,
);
let table_ref = TableReference::bare("table_name");
@@ -277,12 +333,19 @@ mod tests {
let logical_plan = vec![1, 2, 3];
// Create view metadata
table_metadata_manager
.create_view_metadata(view_info.clone().into(), logical_plan, HashSet::new())
.create_view_metadata(
view_info.clone().into(),
logical_plan,
HashSet::new(),
vec!["a".to_string(), "b".to_string()],
vec!["id".to_string(), "name".to_string()],
"definition".to_string(),
)
.await
.unwrap();
let mut table_provider =
DfTableSourceProvider::new(catalog_manager, true, query_ctx, MockDecoder::arc());
DfTableSourceProvider::new(catalog_manager, true, query_ctx, MockDecoder::arc(), true);
// View not found
let table_ref = TableReference::bare("not_exists_view");
@@ -290,6 +353,12 @@ mod tests {
let table_ref = TableReference::bare(view_info.name);
let source = table_provider.resolve_table(table_ref).await.unwrap();
assert_eq!(*source.get_logical_plan().unwrap(), mock_plan());
assert_eq!(
r#"
Projection: person.id AS a, person.name AS b
Filter: person.id > Int32(500)
TableScan: person"#,
format!("\n{:?}", source.get_logical_plan().unwrap())
);
}
}

View File

@@ -63,8 +63,8 @@ fn invalidator<'a>(
ident: &'a CacheIdent,
) -> BoxFuture<'a, Result<()>> {
Box::pin(async move {
if let CacheIdent::TableId(table_id) = ident {
cache.invalidate(table_id).await
if let CacheIdent::TableId(view_id) = ident {
cache.invalidate(view_id).await
}
Ok(())
})
@@ -111,6 +111,7 @@ mod tests {
});
set
};
let definition = "CREATE VIEW test AS SELECT * FROM numbers";
task.view_info.ident.table_id = 1024;
table_metadata_manager
@@ -118,6 +119,9 @@ mod tests {
task.view_info.clone(),
task.create_view.logical_plan.clone(),
table_names,
vec!["a".to_string()],
vec!["number".to_string()],
definition.to_string(),
)
.await
.unwrap();
@@ -132,6 +136,9 @@ mod tests {
.map(|t| t.clone().into())
.collect::<HashSet<_>>()
);
assert_eq!(view_info.definition, task.create_view.definition);
assert_eq!(view_info.columns, task.create_view.columns);
assert_eq!(view_info.plan_columns, task.create_view.plan_columns);
assert!(cache.contains_key(&1024));
cache

View File

@@ -23,6 +23,7 @@ use crate::key::schema_name::SchemaNameKey;
use crate::key::table_info::TableInfoKey;
use crate::key::table_name::TableNameKey;
use crate::key::table_route::TableRouteKey;
use crate::key::view_info::ViewInfoKey;
use crate::key::MetaKey;
/// KvBackend cache invalidator
@@ -76,6 +77,9 @@ where
let key = TableRouteKey::new(*table_id);
self.invalidate_key(&key.to_bytes()).await;
let key = ViewInfoKey::new(*table_id);
self.invalidate_key(&key.to_bytes()).await;
}
CacheIdent::TableName(table_name) => {
let key: TableNameKey = table_name.into();

View File

@@ -197,9 +197,20 @@ impl CreateViewProcedure {
})?;
let new_logical_plan = self.data.task.raw_logical_plan().clone();
let table_names = self.data.task.table_names();
let columns = self.data.task.columns().clone();
let plan_columns = self.data.task.plan_columns().clone();
let new_view_definition = self.data.task.view_definition().to_string();
manager
.update_view_info(view_id, &current_view_info, new_logical_plan, table_names)
.update_view_info(
view_id,
&current_view_info,
new_logical_plan,
table_names,
columns,
plan_columns,
new_view_definition,
)
.await?;
info!("Updated view metadata for view {view_id}");
@@ -210,6 +221,9 @@ impl CreateViewProcedure {
raw_view_info,
self.data.task.raw_logical_plan().clone(),
self.data.task.table_names(),
self.data.task.columns().clone(),
self.data.task.plan_columns().clone(),
self.data.task.view_definition().to_string(),
)
.await?;

View File

@@ -69,6 +69,9 @@ pub(crate) fn test_create_view_task(name: &str) -> CreateViewTask {
create_if_not_exists: false,
logical_plan: vec![1, 2, 3],
table_names,
columns: vec!["a".to_string()],
plan_columns: vec!["number".to_string()],
definition: "CREATE VIEW test AS SELECT * FROM numbers".to_string(),
};
let view_info = RawTableInfo {
@@ -104,6 +107,9 @@ async fn test_on_prepare_view_exists_err() {
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();
@@ -128,6 +134,9 @@ async fn test_on_prepare_with_create_if_view_exists() {
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();
@@ -213,6 +222,8 @@ async fn test_replace_view_metadata() {
// Set `or_replce` to be `true` and try again
task.create_view.or_replace = true;
task.create_view.logical_plan = vec![4, 5, 6];
task.create_view.definition = "new_definition".to_string();
let mut procedure = CreateViewProcedure::new(cluster_id, task, ddl_context.clone());
procedure.on_prepare().await.unwrap();
let ctx = ProcedureContext {
@@ -233,6 +244,9 @@ async fn test_replace_view_metadata() {
.unwrap();
assert_eq!(current_view_info.view_info, vec![4, 5, 6]);
assert_eq!(current_view_info.definition, "new_definition");
assert_eq!(current_view_info.columns, vec!["a".to_string()]);
assert_eq!(current_view_info.plan_columns, vec!["number".to_string()]);
}
#[tokio::test]

View File

@@ -487,11 +487,21 @@ impl TableMetadataManager {
/// Creates metadata for view and returns an error if different metadata exists.
/// The caller MUST ensure it has the exclusive access to `TableNameKey`.
/// Parameters include:
/// - `view_info`: the encoded logical plan
/// - `table_names`: the resolved fully table names in logical plan
/// - `columns`: the view columns
/// - `plan_columns`: the original plan columns
/// - `definition`: The SQL to create the view
///
pub async fn create_view_metadata(
&self,
view_info: RawTableInfo,
raw_logical_plan: Vec<u8>,
table_names: HashSet<TableName>,
columns: Vec<String>,
plan_columns: Vec<String>,
definition: String,
) -> Result<()> {
let view_id = view_info.ident.table_id;
@@ -513,7 +523,13 @@ impl TableMetadataManager {
.build_create_txn(view_id, &table_info_value)?;
// Creates view info
let view_info_value = ViewInfoValue::new(raw_logical_plan, table_names);
let view_info_value = ViewInfoValue::new(
raw_logical_plan,
table_names,
columns,
plan_columns,
definition,
);
let (create_view_info_txn, on_create_view_info_failure) = self
.view_info_manager()
.build_create_txn(view_id, &view_info_value)?;
@@ -926,14 +942,33 @@ impl TableMetadataManager {
}
/// Updates view info and returns an error if different metadata exists.
/// Parameters include:
/// - `view_id`: the view id
/// - `current_view_info_value`: the current view info for CAS checking
/// - `new_view_info`: the encoded logical plan
/// - `table_names`: the resolved fully table names in logical plan
/// - `columns`: the view columns
/// - `plan_columns`: the original plan columns
/// - `definition`: The SQL to create the view
///
#[allow(clippy::too_many_arguments)]
pub async fn update_view_info(
&self,
view_id: TableId,
current_view_info_value: &DeserializedValueWithBytes<ViewInfoValue>,
new_view_info: Vec<u8>,
table_names: HashSet<TableName>,
columns: Vec<String>,
plan_columns: Vec<String>,
definition: String,
) -> Result<()> {
let new_view_info_value = current_view_info_value.update(new_view_info, table_names);
let new_view_info_value = current_view_info_value.update(
new_view_info,
table_names,
columns,
plan_columns,
definition,
);
// Updates view info.
let (update_view_info_txn, on_update_view_info_failure) = self
@@ -2006,12 +2041,21 @@ mod tests {
let view_id = view_info.ident.table_id;
let logical_plan: Vec<u8> = vec![1, 2, 3];
let columns = vec!["a".to_string()];
let plan_columns = vec!["number".to_string()];
let table_names = new_test_table_names();
let definition = "CREATE VIEW test AS SELECT * FROM numbers";
// Create metadata
table_metadata_manager
.create_view_metadata(view_info.clone(), logical_plan.clone(), table_names.clone())
.create_view_metadata(
view_info.clone(),
logical_plan.clone(),
table_names.clone(),
columns.clone(),
plan_columns.clone(),
definition.to_string(),
)
.await
.unwrap();
@@ -2026,6 +2070,9 @@ mod tests {
.into_inner();
assert_eq!(current_view_info.view_info, logical_plan);
assert_eq!(current_view_info.table_names, table_names);
assert_eq!(current_view_info.definition, definition);
assert_eq!(current_view_info.columns, columns);
assert_eq!(current_view_info.plan_columns, plan_columns);
// assert table info
let current_table_info = table_metadata_manager
.table_info_manager()
@@ -2052,10 +2099,16 @@ mod tests {
});
set
};
let new_columns = vec!["b".to_string()];
let new_plan_columns = vec!["number2".to_string()];
let new_definition = "CREATE VIEW test AS SELECT * FROM b_table join c_table";
let current_view_info_value = DeserializedValueWithBytes::from_inner(ViewInfoValue::new(
logical_plan.clone(),
table_names,
columns,
plan_columns,
definition.to_string(),
));
// should be ok.
table_metadata_manager
@@ -2064,6 +2117,9 @@ mod tests {
&current_view_info_value,
new_logical_plan.clone(),
new_table_names.clone(),
new_columns.clone(),
new_plan_columns.clone(),
new_definition.to_string(),
)
.await
.unwrap();
@@ -2074,6 +2130,9 @@ mod tests {
&current_view_info_value,
new_logical_plan.clone(),
new_table_names.clone(),
new_columns.clone(),
new_plan_columns.clone(),
new_definition.to_string(),
)
.await
.unwrap();
@@ -2088,11 +2147,20 @@ mod tests {
.into_inner();
assert_eq!(updated_view_info.view_info, new_logical_plan);
assert_eq!(updated_view_info.table_names, new_table_names);
assert_eq!(updated_view_info.definition, new_definition);
assert_eq!(updated_view_info.columns, new_columns);
assert_eq!(updated_view_info.plan_columns, new_plan_columns);
let wrong_view_info = logical_plan.clone();
let wrong_view_info_value = DeserializedValueWithBytes::from_inner(
current_view_info_value.update(wrong_view_info, new_table_names.clone()),
);
let wrong_definition = "wrong_definition";
let wrong_view_info_value =
DeserializedValueWithBytes::from_inner(current_view_info_value.update(
wrong_view_info,
new_table_names.clone(),
new_columns.clone(),
new_plan_columns.clone(),
wrong_definition.to_string(),
));
// if the current_view_info_value is wrong, it should return an error.
// The ABA problem.
assert!(table_metadata_manager
@@ -2101,6 +2169,9 @@ mod tests {
&wrong_view_info_value,
new_logical_plan.clone(),
new_table_names.clone(),
vec!["c".to_string()],
vec!["number3".to_string()],
wrong_definition.to_string(),
)
.await
.is_err());
@@ -2115,5 +2186,8 @@ mod tests {
.into_inner();
assert_eq!(current_view_info.view_info, new_logical_plan);
assert_eq!(current_view_info.table_names, new_table_names);
assert_eq!(current_view_info.definition, new_definition);
assert_eq!(current_view_info.columns, new_columns);
assert_eq!(current_view_info.plan_columns, new_plan_columns);
}
}

View File

@@ -82,18 +82,33 @@ impl<'a> MetaKey<'a, ViewInfoKey> for ViewInfoKey {
/// The VIEW info value that keeps the metadata.
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
pub struct ViewInfoValue {
/// The encoded logical plan
// The encoded logical plan
pub view_info: RawViewLogicalPlan,
/// The resolved fully table names in logical plan
// The resolved fully table names in logical plan
pub table_names: HashSet<TableName>,
// The view columns
pub columns: Vec<String>,
// The original plan columns
pub plan_columns: Vec<String>,
// The SQL to create the view
pub definition: String,
version: u64,
}
impl ViewInfoValue {
pub fn new(view_info: RawViewLogicalPlan, table_names: HashSet<TableName>) -> Self {
pub fn new(
view_info: RawViewLogicalPlan,
table_names: HashSet<TableName>,
columns: Vec<String>,
plan_columns: Vec<String>,
definition: String,
) -> Self {
Self {
view_info,
table_names,
columns,
plan_columns,
definition,
version: 0,
}
}
@@ -102,10 +117,16 @@ impl ViewInfoValue {
&self,
new_view_info: RawViewLogicalPlan,
table_names: HashSet<TableName>,
columns: Vec<String>,
plan_columns: Vec<String>,
definition: String,
) -> Self {
Self {
view_info: new_view_info,
table_names,
columns,
plan_columns,
definition,
version: self.version + 1,
}
}
@@ -286,6 +307,9 @@ mod tests {
view_info: vec![1, 2, 3],
version: 1,
table_names,
columns: vec!["a".to_string()],
plan_columns: vec!["number".to_string()],
definition: "CREATE VIEW test AS SELECT * FROM numbers".to_string(),
};
let serialized = value.try_as_raw_value().unwrap();
let deserialized = ViewInfoValue::try_from_raw_value(&serialized).unwrap();

View File

@@ -321,10 +321,17 @@ impl CreateViewTask {
}
}
/// Returns the encoded logical plan
pub fn raw_logical_plan(&self) -> &Vec<u8> {
&self.create_view.logical_plan
}
/// Returns the view definition in SQL
pub fn view_definition(&self) -> &str {
&self.create_view.definition
}
/// Returns the resolved table names in view's logical plan
pub fn table_names(&self) -> HashSet<TableName> {
self.create_view
.table_names
@@ -332,6 +339,16 @@ impl CreateViewTask {
.map(|t| t.clone().into())
.collect()
}
/// Returns the view's columns
pub fn columns(&self) -> &Vec<String> {
&self.create_view.columns
}
/// Returns the original logical plan's columns
pub fn plan_columns(&self) -> &Vec<String> {
&self.create_view.plan_columns
}
}
impl TryFrom<PbCreateViewTask> for CreateViewTask {

View File

@@ -20,7 +20,10 @@ mod udf;
use std::sync::Arc;
use datafusion::catalog::CatalogProviderList;
use datafusion::logical_expr::LogicalPlan;
use datafusion::error::Result as DatafusionResult;
use datafusion::logical_expr::{LogicalPlan, LogicalPlanBuilder};
use datafusion_common::Column;
use datafusion_expr::col;
use datatypes::prelude::ConcreteDataType;
pub use expr::build_filter_from_timestamp;
@@ -71,6 +74,39 @@ pub fn create_aggregate_function(
)
}
/// Rename columns by applying a new projection. Returns an error if the column to be
/// renamed does not exist. The `renames` parameter is a `Vector` with elements
/// in the form of `(old_name, new_name)`.
pub fn rename_logical_plan_columns(
enable_ident_normalization: bool,
plan: LogicalPlan,
renames: Vec<(&str, &str)>,
) -> DatafusionResult<LogicalPlan> {
let mut projection = Vec::with_capacity(renames.len());
for (old_name, new_name) in renames {
let old_column: Column = if enable_ident_normalization {
Column::from_qualified_name(old_name)
} else {
Column::from_qualified_name_ignore_case(old_name)
};
let (qualifier_rename, field_rename) =
match plan.schema().qualified_field_from_column(&old_column) {
Ok(qualifier_and_field) => qualifier_and_field,
Err(err) => return Err(err),
};
for (qualifier, field) in plan.schema().iter() {
if qualifier.eq(&qualifier_rename) && field.as_ref() == field_rename {
projection.push(col(Column::from((qualifier, field))).alias(new_name));
}
}
}
LogicalPlanBuilder::from(plan).project(projection)?.build()
}
/// The datafusion `[LogicalPlan]` decoder.
#[async_trait::async_trait]
pub trait SubstraitPlanDecoder {
@@ -95,12 +131,13 @@ mod tests {
use std::sync::Arc;
use datafusion_common::DFSchema;
use datafusion_expr::builder::LogicalTableSource;
use datafusion_expr::{
ColumnarValue as DfColumnarValue, ScalarUDF as DfScalarUDF,
lit, ColumnarValue as DfColumnarValue, ScalarUDF as DfScalarUDF,
TypeSignature as DfTypeSignature,
};
use datatypes::arrow::array::BooleanArray;
use datatypes::arrow::datatypes::DataType;
use datatypes::arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use datatypes::prelude::*;
use datatypes::vectors::{BooleanVector, VectorRef};
@@ -241,6 +278,40 @@ mod tests {
}
}
fn mock_plan() -> LogicalPlan {
let schema = Schema::new(vec![
Field::new("id", DataType::Int32, true),
Field::new("name", DataType::Utf8, true),
]);
let table_source = LogicalTableSource::new(SchemaRef::new(schema));
let projection = None;
let builder =
LogicalPlanBuilder::scan("person", Arc::new(table_source), projection).unwrap();
builder
.filter(col("id").gt(lit(500)))
.unwrap()
.build()
.unwrap()
}
#[test]
fn test_rename_logical_plan_columns() {
let plan = mock_plan();
let new_plan =
rename_logical_plan_columns(true, plan, vec![("id", "a"), ("name", "b")]).unwrap();
assert_eq!(
r#"
Projection: person.id AS a, person.name AS b
Filter: person.id > Int32(500)
TableScan: person"#,
format!("\n{:?}", new_plan)
);
}
#[test]
fn test_create_udaf() {
let creator = DummyAccumulatorCreator;

View File

@@ -455,6 +455,9 @@ pub fn check_permission(
Statement::ShowCreateFlow(stmt) => {
validate_param(&stmt.flow_name, query_ctx)?;
}
Statement::ShowCreateView(stmt) => {
validate_param(&stmt.view_name, query_ctx)?;
}
Statement::CreateExternalTable(stmt) => {
validate_param(&stmt.name, query_ctx)?;
}

View File

@@ -120,6 +120,13 @@ pub enum Error {
source: query::error::Error,
},
#[snafu(display("Failed to get schema from logical plan"))]
GetSchema {
#[snafu(implicit)]
location: Location,
source: query::error::Error,
},
#[snafu(display("Column datatype error"))]
ColumnDataType {
#[snafu(implicit)]
@@ -141,6 +148,15 @@ pub enum Error {
location: Location,
},
#[snafu(display("Expect {expected} columns for view {view_name}, but found {actual}"))]
ViewColumnsMismatch {
view_name: String,
expected: usize,
actual: usize,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to convert column default constraint, column: {}", column_name))]
ConvertColumnDefaultConstraint {
column_name: String,
@@ -205,6 +221,28 @@ pub enum Error {
source: catalog::error::Error,
},
#[snafu(display("Failed to find view info for: {}", view_name))]
FindViewInfo {
view_name: String,
#[snafu(implicit)]
location: Location,
source: common_meta::error::Error,
},
#[snafu(display("View info not found: {}", view_name))]
ViewInfoNotFound {
view_name: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("View not found: {}", view_name))]
ViewNotFound {
view_name: String,
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Failed to find table partition rule for table {}", table_name))]
FindTablePartitionRule {
table_name: String,
@@ -740,6 +778,7 @@ impl ErrorExt for Error {
| Error::InvalidTableName { .. }
| Error::InvalidViewName { .. }
| Error::InvalidExpr { .. }
| Error::ViewColumnsMismatch { .. }
| Error::InvalidViewStmt { .. }
| Error::ConvertIdentifier { .. }
| Error::InvalidPartition { .. } => StatusCode::InvalidArguments,
@@ -770,7 +809,9 @@ impl ErrorExt for Error {
| Error::CreateTableInfo { source, .. }
| Error::IntoVectors { source, .. } => source.status_code(),
Error::RequestInserts { source, .. } => source.status_code(),
Error::RequestInserts { source, .. } | Error::FindViewInfo { source, .. } => {
source.status_code()
}
Error::RequestRegion { source, .. } => source.status_code(),
Error::RequestDeletes { source, .. } => source.status_code(),
Error::SubstraitCodec { source, .. } => source.status_code(),
@@ -787,7 +828,10 @@ impl ErrorExt for Error {
Error::EncodeJson { .. } => StatusCode::Unexpected,
Error::TableNotFound { .. } => StatusCode::TableNotFound,
Error::ViewNotFound { .. }
| Error::ViewInfoNotFound { .. }
| Error::TableNotFound { .. } => StatusCode::TableNotFound,
Error::FlowNotFound { .. } => StatusCode::FlowNotFound,
Error::JoinTask { .. } => StatusCode::Internal,
@@ -804,6 +848,7 @@ impl ErrorExt for Error {
| Error::FindNewColumnsOnInsertion { source, .. } => source.status_code(),
Error::ExecuteStatement { source, .. }
| Error::GetSchema { source, .. }
| Error::ExtractTableNames { source, .. }
| Error::PlanStatement { source, .. }
| Error::ParseQuery { source, .. }

View File

@@ -498,6 +498,9 @@ pub fn to_create_view_expr(
stmt: CreateView,
logical_plan: Vec<u8>,
table_names: Vec<TableName>,
columns: Vec<String>,
plan_columns: Vec<String>,
definition: String,
query_ctx: QueryContextRef,
) -> Result<CreateViewExpr> {
let (catalog_name, schema_name, view_name) = table_idents_to_full_name(&stmt.name, &query_ctx)
@@ -512,6 +515,9 @@ pub fn to_create_view_expr(
create_if_not_exists: stmt.if_not_exists,
or_replace: stmt.or_replace,
table_names,
columns,
plan_columns,
definition,
};
Ok(expr)
@@ -801,11 +807,16 @@ mod tests {
let logical_plan = vec![1, 2, 3];
let table_names = new_test_table_names();
let columns = vec!["a".to_string()];
let plan_columns = vec!["number".to_string()];
let expr = to_create_view_expr(
stmt,
logical_plan.clone(),
table_names.clone(),
columns.clone(),
plan_columns.clone(),
sql.to_string(),
QueryContext::arc(),
)
.unwrap();
@@ -817,6 +828,9 @@ mod tests {
assert!(!expr.or_replace);
assert_eq!(logical_plan, expr.logical_plan);
assert_eq!(table_names, expr.table_names);
assert_eq!(sql, expr.definition);
assert_eq!(columns, expr.columns);
assert_eq!(plan_columns, expr.plan_columns);
}
#[test]
@@ -834,11 +848,16 @@ mod tests {
let logical_plan = vec![1, 2, 3];
let table_names = new_test_table_names();
let columns = vec!["a".to_string()];
let plan_columns = vec!["number".to_string()];
let expr = to_create_view_expr(
stmt,
logical_plan.clone(),
table_names.clone(),
columns.clone(),
plan_columns.clone(),
sql.to_string(),
QueryContext::arc(),
)
.unwrap();
@@ -850,5 +869,8 @@ mod tests {
assert!(expr.or_replace);
assert_eq!(logical_plan, expr.logical_plan);
assert_eq!(table_names, expr.table_names);
assert_eq!(sql, expr.definition);
assert_eq!(columns, expr.columns);
assert_eq!(plan_columns, expr.plan_columns);
}
}

View File

@@ -31,6 +31,7 @@ use common_meta::cache::TableRouteCacheRef;
use common_meta::cache_invalidator::CacheInvalidatorRef;
use common_meta::ddl::ProcedureExecutorRef;
use common_meta::key::flow::{FlowMetadataManager, FlowMetadataManagerRef};
use common_meta::key::view_info::{ViewInfoManager, ViewInfoManagerRef};
use common_meta::key::{TableMetadataManager, TableMetadataManagerRef};
use common_meta::kv_backend::KvBackendRef;
use common_query::Output;
@@ -45,6 +46,7 @@ use session::context::QueryContextRef;
use session::table_name::table_idents_to_full_name;
use snafu::{ensure, OptionExt, ResultExt};
use sql::statements::copy::{CopyDatabase, CopyDatabaseArgument, CopyTable, CopyTableArgument};
use sql::statements::set_variables::SetVariables;
use sql::statements::statement::Statement;
use sql::statements::OptionMap;
use sql::util::format_raw_object_name;
@@ -69,6 +71,7 @@ pub struct StatementExecutor {
procedure_executor: ProcedureExecutorRef,
table_metadata_manager: TableMetadataManagerRef,
flow_metadata_manager: FlowMetadataManagerRef,
view_info_manager: ViewInfoManagerRef,
partition_manager: PartitionRuleManagerRef,
cache_invalidator: CacheInvalidatorRef,
inserter: InserterRef,
@@ -92,6 +95,7 @@ impl StatementExecutor {
procedure_executor,
table_metadata_manager: Arc::new(TableMetadataManager::new(kv_backend.clone())),
flow_metadata_manager: Arc::new(FlowMetadataManager::new(kv_backend.clone())),
view_info_manager: Arc::new(ViewInfoManager::new(kv_backend.clone())),
partition_manager: Arc::new(PartitionRuleManager::new(kv_backend, table_route_cache)),
cache_invalidator,
inserter,
@@ -244,66 +248,9 @@ impl StatementExecutor {
self.show_create_table(table_name, table_ref, query_ctx)
.await
}
Statement::ShowCreateFlow(show) => {
let obj_name = &show.flow_name;
let (catalog_name, flow_name) = match &obj_name.0[..] {
[table] => (query_ctx.current_catalog().to_string(), table.value.clone()),
[catalog, table] => (catalog.value.clone(), table.value.clone()),
_ => {
return InvalidSqlSnafu {
err_msg: format!(
"expect flow name to be <catalog>.<flow_name> or <flow_name>, actual: {obj_name}",
),
}
.fail()
}
};
let flow_name_val = self
.flow_metadata_manager
.flow_name_manager()
.get(&catalog_name, &flow_name)
.await
.context(error::TableMetadataManagerSnafu)?
.context(error::FlowNotFoundSnafu {
flow_name: &flow_name,
})?;
let flow_val = self
.flow_metadata_manager
.flow_info_manager()
.get(flow_name_val.flow_id())
.await
.context(error::TableMetadataManagerSnafu)?
.context(error::FlowNotFoundSnafu {
flow_name: &flow_name,
})?;
self.show_create_flow(obj_name.clone(), flow_val, query_ctx)
.await
}
Statement::SetVariables(set_var) => {
let var_name = set_var.variable.to_string().to_uppercase();
match var_name.as_str() {
"TIMEZONE" | "TIME_ZONE" => set_timezone(set_var.value, query_ctx)?,
"BYTEA_OUTPUT" => set_bytea_output(set_var.value, query_ctx)?,
// Same as "bytea_output", we just ignore it here.
// Not harmful since it only relates to how date is viewed in client app's output.
// The tracked issue is https://github.com/GreptimeTeam/greptimedb/issues/3442.
"DATESTYLE" => set_datestyle(set_var.value, query_ctx)?,
"CLIENT_ENCODING" => validate_client_encoding(set_var)?,
_ => {
return NotSupportedSnafu {
feat: format!("Unsupported set variable {}", var_name),
}
.fail()
}
}
Ok(Output::new_with_affected_rows(0))
}
Statement::ShowCreateFlow(show) => self.show_create_flow(show, query_ctx).await,
Statement::ShowCreateView(show) => self.show_create_view(show, query_ctx).await,
Statement::SetVariables(set_var) => self.set_variables(set_var, query_ctx),
Statement::ShowVariables(show_variable) => self.show_variable(show_variable, query_ctx),
Statement::ShowColumns(show_columns) => {
self.show_columns(show_columns, query_ctx).await
@@ -329,6 +276,29 @@ impl StatementExecutor {
Ok(Output::new_with_record_batches(RecordBatches::empty()))
}
fn set_variables(&self, set_var: SetVariables, query_ctx: QueryContextRef) -> Result<Output> {
let var_name = set_var.variable.to_string().to_uppercase();
match var_name.as_str() {
"TIMEZONE" | "TIME_ZONE" => set_timezone(set_var.value, query_ctx)?,
"BYTEA_OUTPUT" => set_bytea_output(set_var.value, query_ctx)?,
// Same as "bytea_output", we just ignore it here.
// Not harmful since it only relates to how date is viewed in client app's output.
// The tracked issue is https://github.com/GreptimeTeam/greptimedb/issues/3442.
"DATESTYLE" => set_datestyle(set_var.value, query_ctx)?,
"CLIENT_ENCODING" => validate_client_encoding(set_var)?,
_ => {
return NotSupportedSnafu {
feat: format!("Unsupported set variable {}", var_name),
}
.fail()
}
}
Ok(Output::new_with_affected_rows(0))
}
pub async fn plan(
&self,
stmt: QueryStatement,

View File

@@ -400,8 +400,38 @@ impl StatementExecutor {
return InvalidViewStmtSnafu {}.fail();
}
};
// Save the definition for `show create view`.
let definition = create_view.to_string();
// Extract the table names from the origin plan
// Save the columns in plan, it may changed when the schemas of tables in plan
// are altered.
let plan_columns: Vec<_> = logical_plan
.schema()
.context(error::GetSchemaSnafu)?
.column_schemas()
.iter()
.map(|c| c.name.clone())
.collect();
let columns: Vec<_> = create_view
.columns
.iter()
.map(|ident| ident.to_string())
.collect();
// Validate columns
if !columns.is_empty() {
ensure!(
columns.len() == plan_columns.len(),
error::ViewColumnsMismatchSnafu {
view_name: create_view.name.to_string(),
expected: plan_columns.len(),
actual: columns.len(),
}
);
}
// Extract the table names from the original plan
// and rewrite them as fully qualified names.
let (table_names, plan) =
extract_and_rewrite_full_table_names(logical_plan.unwrap_df_plan(), ctx.clone())
@@ -423,6 +453,9 @@ impl StatementExecutor {
create_view,
encoded_plan.to_vec(),
table_names,
columns,
plan_columns,
definition,
ctx.clone(),
)?;
@@ -539,6 +572,12 @@ impl StatementExecutor {
})?;
info!("Successfully created view '{view_name}' with view id {view_id}");
view_info.ident.table_id = view_id;
let view_info = Arc::new(view_info.try_into().context(CreateTableInfoSnafu)?);
let table = DistTable::table(view_info);
// Invalidates local cache ASAP.
self.cache_invalidator
.invalidate(
@@ -551,12 +590,6 @@ impl StatementExecutor {
.await
.context(error::InvalidateTableCacheSnafu)?;
view_info.ident.table_id = view_id;
let view_info = Arc::new(view_info.try_into().context(CreateTableInfoSnafu)?);
let table = DistTable::table(view_info);
Ok(table)
}

View File

@@ -12,24 +12,28 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use common_meta::key::flow::flow_info::FlowInfoValue;
use common_error::ext::BoxedError;
use common_query::Output;
use common_telemetry::tracing;
use partition::manager::PartitionInfo;
use partition::partition::PartitionBound;
use session::context::QueryContextRef;
use snafu::ResultExt;
use session::table_name::table_idents_to_full_name;
use snafu::{OptionExt, ResultExt};
use sql::ast::Ident;
use sql::statements::create::Partitions;
use sql::statements::show::{
ShowColumns, ShowDatabases, ShowIndex, ShowKind, ShowTableStatus, ShowTables, ShowVariables,
ShowColumns, ShowCreateFlow, ShowCreateView, ShowDatabases, ShowIndex, ShowKind,
ShowTableStatus, ShowTables, ShowVariables,
};
use sqlparser::ast::ObjectName;
use table::metadata::TableType;
use table::table_name::TableName;
use table::TableRef;
use crate::error::{self, ExecuteStatementSnafu, Result};
use crate::error::{
self, CatalogSnafu, ExecuteStatementSnafu, ExternalSnafu, FindViewInfoSnafu, InvalidSqlSnafu,
Result, ViewInfoNotFoundSnafu, ViewNotFoundSnafu,
};
use crate::statement::StatementExecutor;
impl StatementExecutor {
@@ -119,13 +123,76 @@ impl StatementExecutor {
}
#[tracing::instrument(skip_all)]
pub async fn show_create_flow(
pub async fn show_create_view(
&self,
flow_name: ObjectName,
flow_val: FlowInfoValue,
show: ShowCreateView,
query_ctx: QueryContextRef,
) -> Result<Output> {
query::sql::show_create_flow(flow_name, flow_val, query_ctx)
let (catalog, schema, view) = table_idents_to_full_name(&show.view_name, &query_ctx)
.map_err(BoxedError::new)
.context(ExternalSnafu)?;
let table_ref = self
.catalog_manager
.table(&catalog, &schema, &view)
.await
.context(CatalogSnafu)?
.context(ViewNotFoundSnafu { view_name: &view })?;
let view_id = table_ref.table_info().ident.table_id;
let view_info = self
.view_info_manager
.get(view_id)
.await
.context(FindViewInfoSnafu { view_name: &view })?
.context(ViewInfoNotFoundSnafu { view_name: &view })?;
query::sql::show_create_view(show.view_name, &view_info.definition, query_ctx)
.context(error::ExecuteStatementSnafu)
}
#[tracing::instrument(skip_all)]
pub async fn show_create_flow(
&self,
show: ShowCreateFlow,
query_ctx: QueryContextRef,
) -> Result<Output> {
let obj_name = &show.flow_name;
let (catalog_name, flow_name) = match &obj_name.0[..] {
[table] => (query_ctx.current_catalog().to_string(), table.value.clone()),
[catalog, table] => (catalog.value.clone(), table.value.clone()),
_ => {
return InvalidSqlSnafu {
err_msg: format!(
"expect flow name to be <catalog>.<flow_name> or <flow_name>, actual: {obj_name}",
),
}
.fail()
}
};
let flow_name_val = self
.flow_metadata_manager
.flow_name_manager()
.get(&catalog_name, &flow_name)
.await
.context(error::TableMetadataManagerSnafu)?
.context(error::FlowNotFoundSnafu {
flow_name: &flow_name,
})?;
let flow_val = self
.flow_metadata_manager
.flow_info_manager()
.get(flow_name_val.flow_id())
.await
.context(error::TableMetadataManagerSnafu)?
.context(error::FlowNotFoundSnafu {
flow_name: &flow_name,
})?;
query::sql::show_create_flow(obj_name.clone(), flow_val, query_ctx)
.context(error::ExecuteStatementSnafu)
}

View File

@@ -64,6 +64,10 @@ impl DfContextProviderAdapter {
engine_state.disallow_cross_catalog_query(),
query_ctx.as_ref(),
Arc::new(DefaultPlanDecoder::new(session_state.clone(), &query_ctx)?),
session_state
.config_options()
.sql_parser
.enable_ident_normalization,
);
let tables = resolve_tables(table_names, &mut table_provider).await?;
@@ -91,10 +95,18 @@ async fn resolve_tables(
if let Entry::Vacant(v) = tables.entry(resolved_name.to_string()) {
// Try our best to resolve the tables here, but we don't return an error if table is not found,
// because the table name may be a temporary name of CTE or view, they can't be found until plan
// because the table name may be a temporary name of CTE, they can't be found until plan
// execution.
if let Ok(table) = table_provider.resolve_table(table_name).await {
let _ = v.insert(table);
match table_provider.resolve_table(table_name).await {
Ok(table) => {
let _ = v.insert(table);
}
Err(e) if e.should_fail() => {
return Err(e).context(CatalogSnafu);
}
_ => {
// ignore
}
}
}
}

View File

@@ -73,6 +73,10 @@ impl DfLogicalPlanner {
self.session_state.clone(),
&query_ctx,
)?),
self.session_state
.config_options()
.sql_parser
.enable_ident_normalization,
);
let context_provider = DfContextProviderAdapter::try_new(
@@ -148,6 +152,10 @@ impl DfLogicalPlanner {
self.session_state.clone(),
&query_ctx,
)?),
self.session_state
.config_options()
.sql_parser
.enable_ident_normalization,
);
PromPlanner::stmt_to_plan(table_provider, stmt, &self.session_state)
.await

View File

@@ -2337,6 +2337,7 @@ mod test {
false,
QueryContext::arc().as_ref(),
DummyDecoder::arc(),
false,
)
}
@@ -3176,6 +3177,7 @@ mod test {
false,
QueryContext::arc().as_ref(),
DummyDecoder::arc(),
true,
),
EvalStmt {
expr: parser::parse("metrics{tag = \"1\"}").unwrap(),
@@ -3205,6 +3207,7 @@ mod test {
false,
QueryContext::arc().as_ref(),
DummyDecoder::arc(),
true,
),
EvalStmt {
expr: parser::parse("avg_over_time(metrics{tag = \"1\"}[5s])").unwrap(),

View File

@@ -52,10 +52,11 @@ pub use show_create_table::create_table_stmt;
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::Ident;
use sql::parser::ParserContext;
use sql::statements::create::{CreateFlow, Partitions};
use sql::statements::create::{CreateFlow, CreateView, Partitions};
use sql::statements::show::{
ShowColumns, ShowDatabases, ShowIndex, ShowKind, ShowTableStatus, ShowTables, ShowVariables,
};
use sql::statements::statement::Statement;
use sqlparser::ast::ObjectName;
use table::requests::{FILE_TABLE_LOCATION_KEY, FILE_TABLE_PATTERN_KEY};
use table::TableRef;
@@ -146,6 +147,13 @@ static SHOW_CREATE_FLOW_OUTPUT_SCHEMA: Lazy<Arc<Schema>> = Lazy::new(|| {
]))
});
static SHOW_CREATE_VIEW_OUTPUT_SCHEMA: Lazy<Arc<Schema>> = Lazy::new(|| {
Arc::new(Schema::new(vec![
ColumnSchema::new("View", ConcreteDataType::string_datatype(), false),
ColumnSchema::new("Create View", ConcreteDataType::string_datatype(), false),
]))
});
fn null() -> Expr {
lit(ScalarValue::Null)
}
@@ -683,6 +691,40 @@ pub fn show_create_table(
Ok(Output::new_with_record_batches(records))
}
pub fn show_create_view(
view_name: ObjectName,
definition: &str,
query_ctx: QueryContextRef,
) -> Result<Output> {
let mut parser_ctx =
ParserContext::new(query_ctx.sql_dialect(), definition).context(error::SqlSnafu)?;
let Statement::CreateView(create_view) =
parser_ctx.parse_statement().context(error::SqlSnafu)?
else {
// MUST be `CreateView` statement.
unreachable!();
};
let stmt = CreateView {
name: view_name.clone(),
columns: create_view.columns,
query: create_view.query,
or_replace: create_view.or_replace,
if_not_exists: create_view.if_not_exists,
};
let sql = format!("{}", stmt);
let columns = vec![
Arc::new(StringVector::from(vec![view_name.to_string()])) as _,
Arc::new(StringVector::from(vec![sql])) as _,
];
let records = RecordBatches::try_from_columns(SHOW_CREATE_VIEW_OUTPUT_SCHEMA.clone(), columns)
.context(error::CreateRecordBatchSnafu)?;
Ok(Output::new_with_record_batches(records))
}
pub fn show_create_flow(
flow_name: ObjectName,
flow_val: FlowInfoValue,

View File

@@ -24,7 +24,7 @@ use sqlparser::ast::{ColumnOption, ColumnOptionDef, DataType, Expr, KeyOrIndexDi
use sqlparser::dialect::keywords::Keyword;
use sqlparser::keywords::ALL_KEYWORDS;
use sqlparser::parser::IsOptional::Mandatory;
use sqlparser::parser::Parser;
use sqlparser::parser::{Parser, ParserError};
use sqlparser::tokenizer::{Token, TokenWithLocation, Word};
use table::requests::validate_table_option;
@@ -125,6 +125,8 @@ impl<'a> ParserContext<'a> {
let if_not_exists = self.parse_if_not_exist()?;
let view_name = self.intern_parse_table_name()?;
let columns = self.parse_view_columns()?;
self.parser
.expect_keyword(Keyword::AS)
.context(SyntaxSnafu)?;
@@ -133,12 +135,36 @@ impl<'a> ParserContext<'a> {
Ok(Statement::CreateView(CreateView {
name: view_name,
columns,
or_replace,
query: Box::new(query),
if_not_exists,
}))
}
fn parse_view_columns(&mut self) -> Result<Vec<Ident>> {
let mut columns = vec![];
if !self.parser.consume_token(&Token::LParen) || self.parser.consume_token(&Token::RParen) {
return Ok(columns);
}
loop {
let name = self.parse_column_name().context(SyntaxSnafu)?;
columns.push(name);
let comma = self.parser.consume_token(&Token::Comma);
if self.parser.consume_token(&Token::RParen) {
// allow a trailing comma, even though it's not in standard
break;
} else if !comma {
return self.expected("',' or ')' after column name", self.parser.peek_token());
}
}
Ok(columns)
}
fn parse_create_external_table(&mut self) -> Result<Statement> {
let _ = self.parser.next_token();
self.parser
@@ -547,10 +573,26 @@ impl<'a> ParserContext<'a> {
Ok(())
}
/// Parse the column name and check if it's valid.
fn parse_column_name(&mut self) -> std::result::Result<Ident, ParserError> {
let name = self.parser.parse_identifier(false)?;
if name.quote_style.is_none() &&
// "ALL_KEYWORDS" are sorted.
ALL_KEYWORDS.binary_search(&name.value.to_uppercase().as_str()).is_ok()
{
return Err(ParserError::ParserError(format!(
"Cannot use keyword '{}' as column name. Hint: add quotes to the name.",
&name.value
)));
}
Ok(name)
}
pub fn parse_column_def(&mut self) -> Result<Column> {
let name = self.parse_column_name().context(SyntaxSnafu)?;
let parser = &mut self.parser;
let name = parser.parse_identifier(false).context(SyntaxSnafu)?;
ensure!(
!(name.quote_style.is_none() &&
// "ALL_KEYWORDS" are sorted.
@@ -2009,4 +2051,85 @@ CREATE TABLE log (
.to_string()
.contains("invalid FULLTEXT option"));
}
#[test]
fn test_parse_create_view_with_columns() {
let sql = "CREATE VIEW test () AS SELECT * FROM NUMBERS";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
.unwrap();
match &result[0] {
Statement::CreateView(c) => {
assert_eq!(c.to_string(), "CREATE VIEW test AS SELECT * FROM NUMBERS");
assert!(!c.or_replace);
assert!(!c.if_not_exists);
assert_eq!("test", c.name.to_string());
}
_ => unreachable!(),
}
assert_eq!(
"CREATE VIEW test AS SELECT * FROM NUMBERS",
result[0].to_string()
);
let sql = "CREATE VIEW test (n1) AS SELECT * FROM NUMBERS";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
.unwrap();
match &result[0] {
Statement::CreateView(c) => {
assert_eq!(c.to_string(), sql);
assert!(!c.or_replace);
assert!(!c.if_not_exists);
assert_eq!("test", c.name.to_string());
}
_ => unreachable!(),
}
assert_eq!(sql, result[0].to_string());
let sql = "CREATE VIEW test (n1, n2) AS SELECT * FROM NUMBERS";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default())
.unwrap();
match &result[0] {
Statement::CreateView(c) => {
assert_eq!(c.to_string(), sql);
assert!(!c.or_replace);
assert!(!c.if_not_exists);
assert_eq!("test", c.name.to_string());
}
_ => unreachable!(),
}
assert_eq!(sql, result[0].to_string());
// Some invalid syntax cases
let sql = "CREATE VIEW test (n1 AS select * from demo";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
assert!(result.is_err());
let sql = "CREATE VIEW test (n1, AS select * from demo";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
assert!(result.is_err());
let sql = "CREATE VIEW test n1,n2) AS select * from demo";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
assert!(result.is_err());
let sql = "CREATE VIEW test (1) AS select * from demo";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
assert!(result.is_err());
// keyword
let sql = "CREATE VIEW test (n1, select) AS select * from demo";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
assert!(result.is_err());
}
}

View File

@@ -21,8 +21,8 @@ use crate::error::{
};
use crate::parser::ParserContext;
use crate::statements::show::{
ShowColumns, ShowCreateFlow, ShowCreateTable, ShowDatabases, ShowIndex, ShowKind, ShowStatus,
ShowTableStatus, ShowTables, ShowVariables,
ShowColumns, ShowCreateFlow, ShowCreateTable, ShowCreateView, ShowDatabases, ShowIndex,
ShowKind, ShowStatus, ShowTableStatus, ShowTables, ShowVariables,
};
use crate::statements::statement::Statement;
@@ -74,6 +74,8 @@ impl<'a> ParserContext<'a> {
self.parse_show_create_table()
} else if self.consume_token("FLOW") {
self.parse_show_create_flow()
} else if self.consume_token("VIEW") {
self.parse_show_create_view()
} else {
self.unsupported(self.peek_token_as_string())
}
@@ -141,6 +143,24 @@ impl<'a> ParserContext<'a> {
Ok(Statement::ShowCreateFlow(ShowCreateFlow { flow_name }))
}
fn parse_show_create_view(&mut self) -> Result<Statement> {
let raw_view_name = self
.parse_object_name()
.with_context(|_| error::UnexpectedSnafu {
sql: self.sql,
expected: "a view name",
actual: self.peek_token_as_string(),
})?;
let view_name = Self::canonicalize_object_name(raw_view_name);
ensure!(
!view_name.0.is_empty(),
InvalidTableNameSnafu {
name: view_name.to_string(),
}
);
Ok(Statement::ShowCreateView(ShowCreateView { view_name }))
}
fn parse_show_table_name(&mut self) -> Result<String> {
self.parser.next_token();
let table_name = self
@@ -906,4 +926,20 @@ mod tests {
assert!(matches!(stmt.kind, ShowKind::Where(_)));
assert_eq!(sql, stmt.to_string());
}
#[test]
pub fn test_show_create_view() {
let sql = "SHOW CREATE VIEW test";
let result =
ParserContext::create_with_dialect(sql, &GreptimeDbDialect {}, ParseOptions::default());
let stmts = result.unwrap();
assert_eq!(1, stmts.len());
assert_eq!(
stmts[0],
Statement::ShowCreateView(ShowCreateView {
view_name: ObjectName(vec![Ident::new("test")]),
})
);
assert_eq!(sql, stmts[0].to_string());
}
}

View File

@@ -385,6 +385,8 @@ impl Display for CreateFlow {
pub struct CreateView {
/// View name
pub name: ObjectName,
/// An optional list of names to be used for columns of the view
pub columns: Vec<Ident>,
/// The clause after `As` that defines the VIEW.
/// Can only be either [Statement::Query] or [Statement::Tql].
pub query: Box<Statement>,
@@ -405,6 +407,9 @@ impl Display for CreateView {
write!(f, "IF NOT EXISTS ")?;
}
write!(f, "{} ", &self.name)?;
if !self.columns.is_empty() {
write!(f, "({}) ", format_list_comma!(self.columns))?;
}
write!(f, "AS {}", &self.query)
}
}

View File

@@ -187,6 +187,19 @@ impl Display for ShowCreateFlow {
}
}
/// SQL structure for `SHOW CREATE VIEW`.
#[derive(Debug, Clone, PartialEq, Eq, Visit, VisitMut)]
pub struct ShowCreateView {
pub view_name: ObjectName,
}
impl Display for ShowCreateView {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let view_name = &self.view_name;
write!(f, "SHOW CREATE VIEW {view_name}")
}
}
/// SQL structure for `SHOW VARIABLES xxx`.
#[derive(Debug, Clone, PartialEq, Eq, Visit, VisitMut)]
pub struct ShowVariables {

View File

@@ -31,8 +31,8 @@ use crate::statements::insert::Insert;
use crate::statements::query::Query;
use crate::statements::set_variables::SetVariables;
use crate::statements::show::{
ShowColumns, ShowCreateFlow, ShowCreateTable, ShowDatabases, ShowIndex, ShowKind, ShowStatus,
ShowTableStatus, ShowTables, ShowVariables,
ShowColumns, ShowCreateFlow, ShowCreateTable, ShowCreateView, ShowDatabases, ShowIndex,
ShowKind, ShowStatus, ShowTableStatus, ShowTables, ShowVariables,
};
use crate::statements::tql::Tql;
use crate::statements::truncate::TruncateTable;
@@ -85,6 +85,8 @@ pub enum Statement {
ShowCreateTable(ShowCreateTable),
// SHOW CREATE FLOW
ShowCreateFlow(ShowCreateFlow),
// SHOW CREATE VIEW
ShowCreateView(ShowCreateView),
// SHOW STATUS
ShowStatus(ShowStatus),
// DESCRIBE TABLE
@@ -126,6 +128,7 @@ impl Display for Statement {
Statement::ShowIndex(s) => s.fmt(f),
Statement::ShowCreateTable(s) => s.fmt(f),
Statement::ShowCreateFlow(s) => s.fmt(f),
Statement::ShowCreateView(s) => s.fmt(f),
Statement::ShowStatus(s) => s.fmt(f),
Statement::DescribeTable(s) => s.fmt(f),
Statement::Explain(s) => s.fmt(f),

View File

@@ -0,0 +1,248 @@
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
INSERT INTO t1 VALUES (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9), (10, 10);
Affected Rows: 10
CREATE VIEW IF NOT EXISTS v1 (a) AS SELECT * FROM t1;
Error: 1004(InvalidArguments), Expect 2 columns for view v1, but found 1
CREATE VIEW IF NOT EXISTS v1 (a) AS SELECT n FROM t1;
Affected Rows: 0
SHOW CREATE VIEW v1;
+------+------------------------------------------------------+
| View | Create View |
+------+------------------------------------------------------+
| v1 | CREATE VIEW IF NOT EXISTS v1 (a) AS SELECT n FROM t1 |
+------+------------------------------------------------------+
SELECT * FROM v1;
+----+
| a |
+----+
| 1 |
| 2 |
| 3 |
| 4 |
| 5 |
| 6 |
| 7 |
| 8 |
| 9 |
| 10 |
+----+
SELECT a FROM v1;
+----+
| a |
+----+
| 1 |
| 2 |
| 3 |
| 4 |
| 5 |
| 6 |
| 7 |
| 8 |
| 9 |
| 10 |
+----+
SELECT n FROM v1;
Error: 3000(PlanQuery), Failed to plan SQL: No field named n. Valid fields are v1.a.
CREATE OR REPLACE VIEW v1 (a, b) AS SELECT n, n+1 FROM t1;
Affected Rows: 0
SHOW CREATE VIEW v1;
+------+-------------------------------------------------------------+
| View | Create View |
+------+-------------------------------------------------------------+
| v1 | CREATE OR REPLACE VIEW v1 (a, b) AS SELECT n, n + 1 FROM t1 |
+------+-------------------------------------------------------------+
SELECT * FROM v1;
+----+----+
| a | b |
+----+----+
| 1 | 2 |
| 2 | 3 |
| 3 | 4 |
| 4 | 5 |
| 5 | 6 |
| 6 | 7 |
| 7 | 8 |
| 8 | 9 |
| 9 | 10 |
| 10 | 11 |
+----+----+
SELECT * FROM v1 WHERE a > 5;
+----+----+
| a | b |
+----+----+
| 6 | 7 |
| 7 | 8 |
| 8 | 9 |
| 9 | 10 |
| 10 | 11 |
+----+----+
SELECT * FROM v1 WHERE b > 5;
+----+----+
| a | b |
+----+----+
| 5 | 6 |
| 6 | 7 |
| 7 | 8 |
| 8 | 9 |
| 9 | 10 |
| 10 | 11 |
+----+----+
SELECT a FROM v1;
+----+
| a |
+----+
| 1 |
| 2 |
| 3 |
| 4 |
| 5 |
| 6 |
| 7 |
| 8 |
| 9 |
| 10 |
+----+
SELECT b FROM v1;
+----+
| b |
+----+
| 2 |
| 3 |
| 4 |
| 5 |
| 6 |
| 7 |
| 8 |
| 9 |
| 10 |
| 11 |
+----+
SELECT a,b FROM v1;
+----+----+
| a | b |
+----+----+
| 1 | 2 |
| 2 | 3 |
| 3 | 4 |
| 4 | 5 |
| 5 | 6 |
| 6 | 7 |
| 7 | 8 |
| 8 | 9 |
| 9 | 10 |
| 10 | 11 |
+----+----+
SELECT n FROM v1;
Error: 3000(PlanQuery), Failed to plan SQL: No field named n. Valid fields are v1.a, v1.b.
SELECT * FROM v1 WHERE n > 5;
Error: 3000(PlanQuery), Failed to plan SQL: No field named n. Valid fields are v1.a, v1.b.
-- test view after altering table t1 --
CREATE OR REPLACE VIEW v1 AS SELECT n, ts FROM t1 LIMIT 5;
Affected Rows: 0
SELECT * FROM v1;
+---+-------------------------+
| n | ts |
+---+-------------------------+
| 1 | 1970-01-01T00:00:00.001 |
| 2 | 1970-01-01T00:00:00.002 |
| 3 | 1970-01-01T00:00:00.003 |
| 4 | 1970-01-01T00:00:00.004 |
| 5 | 1970-01-01T00:00:00.005 |
+---+-------------------------+
ALTER TABLE t1 ADD COLUMN s STRING DEFAULT '';
Affected Rows: 0
SELECT * FROM v1;
+---+-------------------------+
| n | ts |
+---+-------------------------+
| 1 | 1970-01-01T00:00:00.001 |
| 2 | 1970-01-01T00:00:00.002 |
| 3 | 1970-01-01T00:00:00.003 |
| 4 | 1970-01-01T00:00:00.004 |
| 5 | 1970-01-01T00:00:00.005 |
+---+-------------------------+
ALTER TABLE t1 DROP COLUMN n;
Affected Rows: 0
-- FIXME(dennis): The result looks weird,
-- Looks like substrait referes to columns only by their relative indices, so thats name-independent.
-- Limit: skip=0, fetch=5
-- Projection: greptime.public.t1.ts, greptime.public.t1.s
-- MergeScan [is_placeholder=false]
-- Limit: skip=0, fetch=5
-- MergeScan [is_placeholder=false]
-- See https://github.com/apache/datafusion/issues/6489
SELECT * FROM v1;
+-------------------------+---+
| ts | s |
+-------------------------+---+
| 1970-01-01T00:00:00.001 | |
| 1970-01-01T00:00:00.002 | |
| 1970-01-01T00:00:00.003 | |
| 1970-01-01T00:00:00.004 | |
| 1970-01-01T00:00:00.005 | |
+-------------------------+---+
USE public;
Affected Rows: 0
DROP DATABASE schema_for_view_test;
Affected Rows: 0

View File

@@ -0,0 +1,64 @@
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);
CREATE VIEW IF NOT EXISTS v1 (a) AS SELECT * FROM t1;
CREATE VIEW IF NOT EXISTS v1 (a) AS SELECT n FROM t1;
SHOW CREATE VIEW v1;
SELECT * FROM v1;
SELECT a FROM v1;
SELECT n FROM v1;
CREATE OR REPLACE VIEW v1 (a, b) AS SELECT n, n+1 FROM t1;
SHOW CREATE VIEW v1;
SELECT * FROM v1;
SELECT * FROM v1 WHERE a > 5;
SELECT * FROM v1 WHERE b > 5;
SELECT a FROM v1;
SELECT b FROM v1;
SELECT a,b FROM v1;
SELECT n FROM v1;
SELECT * FROM v1 WHERE n > 5;
-- test view after altering table t1 --
CREATE OR REPLACE VIEW v1 AS SELECT n, ts FROM t1 LIMIT 5;
SELECT * FROM v1;
ALTER TABLE t1 ADD COLUMN s STRING DEFAULT '';
SELECT * FROM v1;
ALTER TABLE t1 DROP COLUMN n;
-- FIXME(dennis): The result looks weird,
-- Looks like substrait referes to columns only by their relative indices, so thats name-independent.
-- Limit: skip=0, fetch=5
-- Projection: greptime.public.t1.ts, greptime.public.t1.s
-- MergeScan [is_placeholder=false]
-- Limit: skip=0, fetch=5
-- MergeScan [is_placeholder=false]
-- See https://github.com/apache/datafusion/issues/6489
SELECT * FROM v1;
USE public;
DROP DATABASE schema_for_view_test;

View File

@@ -0,0 +1,136 @@
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
INSERT INTO t1 VALUES (41, "hello", 1), (42, "world", 2), (43, "greptime", 3);
Affected Rows: 3
CREATE VIEW v1 AS SELECT a, b FROM t1;
Affected Rows: 0
SELECT * FROM v1;
+----+----------+
| a | b |
+----+----------+
| 41 | hello |
| 42 | world |
| 43 | greptime |
+----+----------+
SELECT a FROM v1;
+----+
| a |
+----+
| 41 |
| 42 |
| 43 |
+----+
INSERT INTO t1 VALUES (44, "greptimedb", 4);
Affected Rows: 1
SELECT * FROM v1;
+----+------------+
| a | b |
+----+------------+
| 41 | hello |
| 42 | world |
| 43 | greptime |
| 44 | greptimedb |
+----+------------+
SHOW CREATE VIEW v1;
+------+---------------------------------------+
| View | Create View |
+------+---------------------------------------+
| v1 | CREATE VIEW v1 AS SELECT a, b FROM t1 |
+------+---------------------------------------+
CREATE OR REPLACE VIEW v1 AS SELECT a, b, c FROM t1 WHERE a > 43;
Affected Rows: 0
SHOW CREATE VIEW v1;
+------+------------------------------------------------------------------+
| View | Create View |
+------+------------------------------------------------------------------+
| v1 | CREATE OR REPLACE VIEW v1 AS SELECT a, b, c FROM t1 WHERE a > 43 |
+------+------------------------------------------------------------------+
SELECT * FROM v1;
+----+------------+-------------------------+
| a | b | c |
+----+------------+-------------------------+
| 44 | greptimedb | 1970-01-01T00:00:00.004 |
+----+------------+-------------------------+
--- if not exists, so it doesn't change at all ---
CREATE VIEW IF NOT EXISTS v1 AS SELECT c FROM t1;
Affected Rows: 0
SHOW CREATE VIEW v1;
+------+------------------------------------------------------------------+
| View | Create View |
+------+------------------------------------------------------------------+
| v1 | CREATE OR REPLACE VIEW v1 AS SELECT a, b, c FROM t1 WHERE a > 43 |
+------+------------------------------------------------------------------+
SELECT * FROM v1;
+----+------------+-------------------------+
| a | b | c |
+----+------------+-------------------------+
| 44 | greptimedb | 1970-01-01T00:00:00.004 |
+----+------------+-------------------------+
--- if not exists with replace, so it changes ---
CREATE OR REPLACE VIEW IF NOT EXISTS v1 AS SELECT c FROM t1;
Affected Rows: 0
SHOW CREATE VIEW v1;
+------+-------------------------------------------------------------+
| View | Create View |
+------+-------------------------------------------------------------+
| v1 | CREATE OR REPLACE VIEW IF NOT EXISTS v1 AS SELECT c FROM t1 |
+------+-------------------------------------------------------------+
SELECT * FROM v1;
+-------------------------+
| c |
+-------------------------+
| 1970-01-01T00:00:00.001 |
| 1970-01-01T00:00:00.002 |
| 1970-01-01T00:00:00.003 |
| 1970-01-01T00:00:00.004 |
+-------------------------+
USE public;
Affected Rows: 0
DROP DATABASE schema_for_view_test;
Affected Rows: 0

View File

@@ -0,0 +1,44 @@
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);
CREATE VIEW v1 AS SELECT a, b FROM t1;
SELECT * FROM v1;
SELECT a FROM v1;
INSERT INTO t1 VALUES (44, "greptimedb", 4);
SELECT * FROM v1;
SHOW CREATE VIEW v1;
CREATE OR REPLACE VIEW v1 AS SELECT a, b, c FROM t1 WHERE a > 43;
SHOW CREATE VIEW v1;
SELECT * FROM v1;
--- if not exists, so it doesn't change at all ---
CREATE VIEW IF NOT EXISTS v1 AS SELECT c FROM t1;
SHOW CREATE VIEW v1;
SELECT * FROM v1;
--- if not exists with replace, so it changes ---
CREATE OR REPLACE VIEW IF NOT EXISTS v1 AS SELECT c FROM t1;
SHOW CREATE VIEW v1;
SELECT * FROM v1;
USE public;
DROP DATABASE schema_for_view_test;

View File

@@ -30,20 +30,28 @@ SELECT * FROM v1;
| 1970-01-01T00:00:00.042 |
+-------------------------+
-- CREATE VIEW v1 AS SELECT 'whatever'; --
-- FIXME(dennis): Substrait doesn't support alias in projection --
-- https://github.com/apache/datafusion/issues/6489 --
SELECT j FROM v1 WHERE j > 41;
Error: 3000(PlanQuery), Failed to plan SQL: No field named j. Valid fields are v1.i.
-- FIXME(dennis):: name alias in view, not supported yet --
--SELECT x FROM v1 t1(x) WHERE x > 41 --
SELECT x FROM v1 t1(x) WHERE x > 41;
+-------------------------+
| x |
+-------------------------+
| 1970-01-01T00:00:00.042 |
+-------------------------+
-- FIXME(dennis): DROP VIEW not supported yet--
-- DROP VIEW v1 --
-- SELECT j FROM v1 WHERE j > 41 --
-- CREATE VIEW v1 AS SELECT 'whatever'; --
-- SELECT * FROM v1; --
-- CREATE OR REPLACE VIEW v1 AS SELECT 42; --
-- substrait can't process such query currently
-- CREATE VIEW v1 AS SELECT 'whatever';--
-- SELECT * FROM v1; --
-- substrait can't process such query currently
--CREATE OR REPLACE VIEW v1 AS SELECT 42;--
--SELECT * FROM v1;--
INSERT INTO v1 VALUES (1);
Error: 1004(InvalidArguments), Invalid SQL, error: column count mismatch, columns: 0, values: 1

View File

@@ -14,27 +14,22 @@ FROM t1 WHERE i < 43;
SELECT * FROM v1;
-- CREATE VIEW v1 AS SELECT 'whatever'; --
-- FIXME(dennis): Substrait doesn't support alias in projection --
-- https://github.com/apache/datafusion/issues/6489 --
SELECT j FROM v1 WHERE j > 41;
-- FIXME(dennis):: name alias in view, not supported yet --
--SELECT x FROM v1 t1(x) WHERE x > 41 --
SELECT x FROM v1 t1(x) WHERE x > 41;
-- FIXME(dennis): DROP VIEW not supported yet--
-- DROP VIEW v1 --
-- SELECT j FROM v1 WHERE j > 41 --
-- CREATE VIEW v1 AS SELECT 'whatever'; --
-- substrait can't process such query currently
-- CREATE VIEW v1 AS SELECT 'whatever';--
-- SELECT * FROM v1; --
-- CREATE OR REPLACE VIEW v1 AS SELECT 42; --
-- SELECT * FROM v1; --
-- substrait can't process such query currently
--CREATE OR REPLACE VIEW v1 AS SELECT 42;--
--SELECT * FROM v1;--
INSERT INTO v1 VALUES (1);