diff --git a/packages/cubejs-backend-native/Cargo.lock b/packages/cubejs-backend-native/Cargo.lock index 426ccc912292d..253fec2eeef0e 100644 --- a/packages/cubejs-backend-native/Cargo.lock +++ b/packages/cubejs-backend-native/Cargo.lock @@ -1936,6 +1936,7 @@ dependencies = [ "async-trait", "byteorder", "bytes 0.5.6", + "chrono", "log", "thiserror", "tokio", diff --git a/rust/cubesql/.gitignore b/rust/cubesql/.gitignore index 654f0364dc1ee..5a09063345259 100644 --- a/rust/cubesql/.gitignore +++ b/rust/cubesql/.gitignore @@ -6,3 +6,4 @@ upstream dist node_modules .vscode +/cubesql/egraph-debug diff --git a/rust/cubesql/cubesql/src/compile/engine/df/scan.rs b/rust/cubesql/cubesql/src/compile/engine/df/scan.rs index b2b1318bd0d2e..43382c2af3353 100644 --- a/rust/cubesql/cubesql/src/compile/engine/df/scan.rs +++ b/rust/cubesql/cubesql/src/compile/engine/df/scan.rs @@ -7,7 +7,7 @@ use std::{ }; use async_trait::async_trait; -use cubeclient::models::{V1LoadRequestQuery, V1LoadResult}; +use cubeclient::models::{V1LoadRequestQuery, V1LoadResult, V1LoadResultAnnotation}; use datafusion::{ arrow::{ array::{ArrayRef, BooleanBuilder, Float64Builder, Int64Builder, StringBuilder}, @@ -34,12 +34,20 @@ use chrono::{TimeZone, Utc}; use datafusion::{ arrow::{array::TimestampNanosecondBuilder, datatypes::TimeUnit}, execution::context::TaskContext, + scalar::ScalarValue, }; +use serde_json::json; + +#[derive(Debug, Clone, Eq, PartialEq)] +pub enum MemberField { + Member(String), + Literal(ScalarValue), +} #[derive(Debug, Clone)] pub struct CubeScanNode { pub schema: DFSchemaRef, - pub member_fields: Vec>, + pub member_fields: Vec, pub request: V1LoadRequestQuery, pub auth_context: Arc, } @@ -47,7 +55,7 @@ pub struct CubeScanNode { impl CubeScanNode { pub fn new( schema: DFSchemaRef, - member_fields: Vec>, + member_fields: Vec, request: V1LoadRequestQuery, auth_context: Arc, ) -> Self { @@ -144,7 +152,7 @@ impl ExtensionPlanner for CubeScanExtensionPlanner { struct CubeScanExecutionPlan { // Options from logical node schema: SchemaRef, - member_fields: Vec>, + member_fields: Vec, request: V1LoadRequestQuery, auth_context: Arc, // Shared references which will be injected by extension planner @@ -153,6 +161,50 @@ struct CubeScanExecutionPlan { meta_fields: TransportServiceMetaFields, } +macro_rules! build_column { + ($data_type:expr, $builder_ty:ty, $response:expr, $field_name:expr, { $($builder_block:tt)* }, { $($scalar_block:tt)* }) => {{ + let mut builder = <$builder_ty>::new($response.data.len()); + + match $field_name { + MemberField::Member(field_name) => { + for row in $response.data.iter() { + let value = row.as_object().unwrap().get(field_name).ok_or( + DataFusionError::Internal( + "Unexpected response from Cube.js, rows are not objects" + .to_string(), + ), + )?; + match (&value, &mut builder) { + (serde_json::Value::Null, builder) => builder.append_null()?, + $($builder_block)* + (v, _) => { + return Err(DataFusionError::Execution(format!( + "Unable to map value {:?} to {:?}", + v, + $data_type + ))); + } + }; + } + } + MemberField::Literal(value) => { + match (value, &mut builder) { + $($scalar_block)* + (v, _) => { + return Err(DataFusionError::Execution(format!( + "Unable to map value {:?} to {:?}", + v, + $data_type + ))); + } + } + } + }; + + Arc::new(builder.finish()) as ArrayRef + }} +} + impl CubeScanExecutionPlan { // This methods transform response from Cube.js to RecordBatch which stores // schema and array of columns. @@ -163,199 +215,123 @@ impl CubeScanExecutionPlan { let field_name = &self.member_fields[i]; let column = match schema_field.data_type() { DataType::Utf8 => { - let mut builder = StringBuilder::new(100); - - if let Some(field_name) = field_name { - for row in response.data.iter() { - let value = row.as_object().unwrap().get(field_name).ok_or( - DataFusionError::Internal( - "Unexpected response from Cube.js, rows are not objects" - .to_string(), - ), - )?; - match &value { - serde_json::Value::Null => builder.append_null()?, - serde_json::Value::String(v) => builder.append_value(v)?, - serde_json::Value::Bool(v) => { - builder.append_value(if *v { "true" } else { "false" })? - } - serde_json::Value::Number(v) => { - builder.append_value(v.to_string())? - } - v => { - log::error!( - "Unable to map value {:?} to DataType::Utf8 (returning null)", - v - ); - - builder.append_null()? - } - }; + build_column!( + DataType::Utf8, + StringBuilder, + response, + field_name, + { + (serde_json::Value::String(v), builder) => builder.append_value(v)?, + (serde_json::Value::Bool(v), builder) => builder.append_value(if *v { "true" } else { "false" })?, + (serde_json::Value::Number(v), builder) => builder.append_value(v.to_string())?, + }, + { + (ScalarValue::Utf8(v), builder) => builder.append_option(v.as_ref())?, } - } else { - builder.append_null()?; - } - - Arc::new(builder.finish()) as ArrayRef + ) } DataType::Int64 => { - let mut builder = Int64Builder::new(100); - - for row in response.data.iter() { - if let Some(field_name) = field_name { - let value = row.as_object().unwrap().get(field_name).ok_or( - DataFusionError::Internal( - "Unexpected response from Cube.js, rows are not objects" - .to_string(), - ), - )?; - match &value { - serde_json::Value::Null => builder.append_null()?, - serde_json::Value::Number(number) => match number.as_i64() { - Some(v) => builder.append_value(v)?, - None => builder.append_null()?, - }, - serde_json::Value::String(s) => match s.parse::() { - Ok(v) => builder.append_value(v)?, - Err(error) => { - warn!( - "Unable to parse value as i64: {}", - error.to_string() - ); - - builder.append_null()? - } - }, - v => { - log::error!( - "Unable to map value {:?} to DataType::Int64 (returning null)", - v + build_column!( + DataType::Int64, + Int64Builder, + response, + field_name, + { + (serde_json::Value::Number(number), builder) => match number.as_i64() { + Some(v) => builder.append_value(v)?, + None => builder.append_null()?, + }, + (serde_json::Value::String(s), builder) => match s.parse::() { + Ok(v) => builder.append_value(v)?, + Err(error) => { + warn!( + "Unable to parse value as i64: {}", + error.to_string() ); builder.append_null()? } - }; - } else { - builder.append_null()?; + }, + }, + { + (ScalarValue::Int64(v), builder) => builder.append_option(v.clone())?, } - } - - Arc::new(builder.finish()) as ArrayRef + ) } DataType::Float64 => { - let mut builder = Float64Builder::new(100); - - for row in response.data.iter() { - if let Some(field_name) = field_name { - let value = row.as_object().unwrap().get(field_name).ok_or( - DataFusionError::Internal( - "Unexpected response from Cube.js, rows are not objects" - .to_string(), - ), - )?; - match &value { - serde_json::Value::Null => builder.append_null()?, - serde_json::Value::Number(number) => match number.as_f64() { - Some(v) => builder.append_value(v)?, - None => builder.append_null()?, - }, - serde_json::Value::String(s) => match s.parse::() { - Ok(v) => builder.append_value(v)?, - Err(error) => { - warn!( - "Unable to parse value as f64: {}", - error.to_string() - ); - - builder.append_null()? - } - }, - v => { - log::error!("Unable to map value {:?} to DataType::Float64 (returning null)", v); + build_column!( + DataType::Float64, + Float64Builder, + response, + field_name, + { + (serde_json::Value::Number(number), builder) => match number.as_f64() { + Some(v) => builder.append_value(v)?, + None => builder.append_null()?, + }, + (serde_json::Value::String(s), builder) => match s.parse::() { + Ok(v) => builder.append_value(v)?, + Err(error) => { + warn!( + "Unable to parse value as f64: {}", + error.to_string() + ); builder.append_null()? } - }; - } else { - builder.append_null()?; + }, + }, + { + (ScalarValue::Float64(v), builder) => builder.append_option(v.clone())?, } - } - - Arc::new(builder.finish()) as ArrayRef + ) } DataType::Boolean => { - let mut builder = BooleanBuilder::new(100); - - for row in response.data.iter() { - if let Some(field_name) = field_name { - let value = row.as_object().unwrap().get(field_name).ok_or( - DataFusionError::Internal( - "Unexpected response from Cube.js, rows are not objects" - .to_string(), - ), - )?; - match &value { - serde_json::Value::Null => builder.append_null()?, - serde_json::Value::Bool(v) => builder.append_value(*v)?, - // Cube allows to mark a type as boolean, but it doesn't guarantee that the user will return a boolean type - serde_json::Value::String(v) => match v.as_str() { - "true" | "1" => builder.append_value(true)?, - "false" | "0" => builder.append_value(false)?, - _ => { - log::error!("Unable to map value {:?} to DataType::Boolean (returning null)", v); - - builder.append_null()? - } - }, - v => { + build_column!( + DataType::Boolean, + BooleanBuilder, + response, + field_name, + { + (serde_json::Value::Bool(v), builder) => builder.append_value(*v)?, + (serde_json::Value::String(v), builder) => match v.as_str() { + "true" | "1" => builder.append_value(true)?, + "false" | "0" => builder.append_value(false)?, + _ => { log::error!("Unable to map value {:?} to DataType::Boolean (returning null)", v); builder.append_null()? } - }; - } else { - builder.append_null()?; + }, + }, + { + (ScalarValue::Boolean(v), builder) => builder.append_option(v.clone())?, } - } - - Arc::new(builder.finish()) as ArrayRef + ) } DataType::Timestamp(TimeUnit::Nanosecond, None) => { - let mut builder = TimestampNanosecondBuilder::new(response.data.len()); - - for row in response.data.iter() { - if let Some(field_name) = field_name { - let value = row.as_object().unwrap().get(field_name).ok_or( - DataFusionError::Internal( - "Unexpected response from Cube.js, rows are not objects" - .to_string(), - ), - )?; - match &value { - serde_json::Value::Null => builder.append_null()?, - serde_json::Value::String(s) => { - let timestamp = Utc - .datetime_from_str(s.as_str(), "%Y-%m-%dT%H:%M:%S.%f") - .map_err(|e| { - DataFusionError::Execution(format!( - "Can't parse timestamp: '{}': {}", - s, e - )) - })?; - builder.append_value(timestamp.timestamp_nanos())?; - } - v => { - log::error!("Unable to map value {:?} to DataType::Timestamp(TimeUnit::Nanosecond, None) (returning null)", v); - - builder.append_null()? - } - }; - } else { - builder.append_null()?; + build_column!( + DataType::Timestamp(TimeUnit::Nanosecond, None), + TimestampNanosecondBuilder, + response, + field_name, + { + (serde_json::Value::String(s), builder) => { + let timestamp = Utc + .datetime_from_str(s.as_str(), "%Y-%m-%dT%H:%M:%S.%f") + .map_err(|e| { + DataFusionError::Execution(format!( + "Can't parse timestamp: '{}': {}", + s, e + )) + })?; + builder.append_value(timestamp.timestamp_nanos())?; + }, + }, + { + (ScalarValue::TimestampNanosecond(v, None), builder) => builder.append_option(v.clone())?, } - } - - Arc::new(builder.finish()) as ArrayRef + ) } t => { return Err(DataFusionError::NotImplemented(format!( @@ -410,23 +386,55 @@ impl ExecutionPlan for CubeScanExecutionPlan { _partition: usize, _context: Arc, ) -> Result { - let result = self - .transport - .load( - self.request.clone(), - self.auth_context.clone(), - self.meta_fields.clone(), + let no_members_query = self.request.measures.as_ref().map(|v| v.len()).unwrap_or(0) == 0 + && self + .request + .dimensions + .as_ref() + .map(|v| v.len()) + .unwrap_or(0) + == 0 + && self + .request + .time_dimensions + .as_ref() + .map(|v| v.iter().filter(|d| d.granularity.is_some()).count()) + .unwrap_or(0) + == 0; + let result = if no_members_query { + let limit = self.request.limit.unwrap_or(1); + let mut data = Vec::new(); + for _ in 0..limit { + data.push(serde_json::Value::Null) + } + V1LoadResult::new( + V1LoadResultAnnotation { + measures: json!(Vec::::new()), + dimensions: json!(Vec::::new()), + segments: json!(Vec::::new()), + time_dimensions: json!(Vec::::new()), + }, + data, ) - .await; - - let mut response = result.map_err(|err| DataFusionError::Execution(err.to_string()))?; - - let result = if let Some(data) = response.results.pop() { - data } else { - return Err(DataFusionError::Execution(format!( - "Unable to extract result from Cube.js response", - ))); + let result = self + .transport + .load( + self.request.clone(), + self.auth_context.clone(), + self.meta_fields.clone(), + ) + .await; + + let mut response = result.map_err(|err| DataFusionError::Execution(err.to_string()))?; + + if let Some(data) = response.results.pop() { + data + } else { + return Err(DataFusionError::Execution(format!( + "Unable to extract result from Cube.js response", + ))); + } }; Ok(Box::pin(CubeScanMemoryStream::new( @@ -587,11 +595,14 @@ mod tests { member_fields: schema .fields() .iter() - .map(|f| Some(f.name().to_string())) + .map(|f| MemberField::Member(f.name().to_string())) .collect(), request: V1LoadRequestQuery { - measures: None, - dimensions: None, + measures: Some(vec![ + "KibanaSampleDataEcommerce.count".to_string(), + "KibanaSampleDataEcommerce.maxPrice".to_string(), + ]), + dimensions: Some(vec!["KibanaSampleDataEcommerce.isBool".to_string()]), segments: None, time_dimensions: None, order: None, diff --git a/rust/cubesql/cubesql/src/compile/mod.rs b/rust/cubesql/cubesql/src/compile/mod.rs index 0d56c868f8560..b92cbd1adfc29 100644 --- a/rust/cubesql/cubesql/src/compile/mod.rs +++ b/rust/cubesql/cubesql/src/compile/mod.rs @@ -62,8 +62,11 @@ use self::{ use crate::{ compile::{ builder::QueryBuilder, - engine::udf::{ - create_pg_is_other_temp_schema, create_pg_my_temp_schema, create_session_user_udf, + engine::{ + df::scan::MemberField, + udf::{ + create_pg_is_other_temp_schema, create_pg_my_temp_schema, create_session_user_udf, + }, }, rewrite::converter::LogicalPlanToLanguageConverter, }, @@ -1447,13 +1450,6 @@ struct QueryPlanner { session_manager: Arc, } -lazy_static! { - static ref METABASE_WORKAROUND: regex::Regex = regex::Regex::new( - r#"SELECT true AS "_" FROM "public"\."(?P.*)" WHERE 1 <> 1 LIMIT 0"# - ) - .unwrap(); -} - impl QueryPlanner { pub fn new( state: Arc, @@ -1475,24 +1471,6 @@ impl QueryPlanner { stmt: &ast::Statement, q: &Box, ) -> CompilationResult { - // Metabase - if let Some(c) = METABASE_WORKAROUND.captures(&stmt.to_string()) { - let tblname = c.name("tblname").unwrap().as_str(); - if self.meta.find_cube_with_name(tblname).is_some() { - return Ok(QueryPlan::MetaTabular( - StatusFlags::empty(), - Box::new(dataframe::DataFrame::new( - vec![dataframe::Column::new( - "_".to_string(), - ColumnType::Int8, - ColumnFlags::empty(), - )], - vec![], - )), - )); - } - } - // TODO move CUBESQL_REWRITE_ENGINE env to config let rewrite_engine = env::var("CUBESQL_REWRITE_ENGINE") .ok() @@ -1691,7 +1669,7 @@ impl QueryPlanner { schema .fields() .iter() - .map(|f| Some(f.name().to_string())) + .map(|f| MemberField::Member(f.name().to_string())) .collect(), query.request, // @todo Remove after split! @@ -3986,7 +3964,9 @@ ORDER BY \"COUNT(count)\" DESC" ); assert_eq!( &cube_scan.member_fields, - &vec![Some("KibanaSampleDataEcommerce.count".to_string())] + &vec![MemberField::Member( + "KibanaSampleDataEcommerce.count".to_string() + )] ); } @@ -5617,7 +5597,7 @@ ORDER BY \"COUNT(count)\" DESC" QueryPlan::DataFusionSelect(flags, plan, ctx) => { let df = DFDataFrame::new(ctx.state, &plan); let batches = df.collect().await?; - let frame = batch_to_dataframe(&batches)?; + let frame = batch_to_dataframe(&df.schema().into(), &batches)?; return Ok((frame.print(), flags)); } @@ -5698,14 +5678,14 @@ ORDER BY \"COUNT(count)\" DESC" #[tokio::test] async fn test_information_schema_stats_for_columns() -> Result<(), CubeError> { // This query is used by metabase for introspection - assert_eq!( + insta::assert_snapshot!( + "test_information_schema_stats_for_columns", execute_query(" SELECT A.TABLE_SCHEMA TABLE_CAT, NULL TABLE_SCHEM, A.TABLE_NAME, A.COLUMN_NAME, B.SEQ_IN_INDEX KEY_SEQ, B.INDEX_NAME PK_NAME FROM INFORMATION_SCHEMA.COLUMNS A, INFORMATION_SCHEMA.STATISTICS B WHERE A.COLUMN_KEY in ('PRI','pri') AND B.INDEX_NAME='PRIMARY' AND (ISNULL(database()) OR (A.TABLE_SCHEMA = database())) AND (ISNULL(database()) OR (B.TABLE_SCHEMA = database())) AND A.TABLE_NAME = 'OutlierFingerprints' AND B.TABLE_NAME = 'OutlierFingerprints' AND A.TABLE_SCHEMA = B.TABLE_SCHEMA AND A.TABLE_NAME = B.TABLE_NAME AND A.COLUMN_NAME = B.COLUMN_NAME - ORDER BY A.COLUMN_NAME".to_string(), DatabaseProtocol::MySQL).await?, - "++\n++\n++" + ORDER BY A.COLUMN_NAME".to_string(), DatabaseProtocol::MySQL).await? ); Ok(()) @@ -5986,13 +5966,13 @@ ORDER BY \"COUNT(count)\" DESC" ); // Negative test, we dont define this variable - assert_eq!( + insta::assert_snapshot!( + "show_variables_like_aurora", execute_query( "show variables like 'aurora_version';".to_string(), DatabaseProtocol::MySQL ) - .await?, - "++\n++\n++" + .await? ); // All variables @@ -6134,7 +6114,8 @@ ORDER BY \"COUNT(count)\" DESC" #[tokio::test] async fn test_tableau() -> Result<(), CubeError> { - assert_eq!( + insta::assert_snapshot!( + "tableau_table_name_column_name_query", execute_query( "SELECT `table_name`, `column_name` FROM `information_schema`.`columns` @@ -6142,8 +6123,7 @@ ORDER BY \"COUNT(count)\" DESC" .to_string(), DatabaseProtocol::MySQL ) - .await?, - "++\n++\n++" + .await? ); insta::assert_snapshot!( @@ -9522,4 +9502,28 @@ ORDER BY \"COUNT(count)\" DESC" } ) } + + #[test] + fn metabase_limit_0() { + init_logger(); + + let logical_plan = convert_select_to_query_plan( + "SELECT true AS \"_\" FROM \"public\".\"KibanaSampleDataEcommerce\" WHERE 1 <> 1 LIMIT 0".to_string(), + DatabaseProtocol::PostgreSQL + ).as_logical_plan(); + + assert_eq!( + logical_plan.find_cube_scan().request, + V1LoadRequestQuery { + measures: Some(vec![]), + dimensions: Some(vec![]), + segments: Some(vec![]), + time_dimensions: None, + order: None, + limit: Some(1), + offset: None, + filters: None + } + ) + } } diff --git a/rust/cubesql/cubesql/src/compile/rewrite/converter.rs b/rust/cubesql/cubesql/src/compile/rewrite/converter.rs index 1cf55c9e432ee..5d4bc81eb599f 100644 --- a/rust/cubesql/cubesql/src/compile/rewrite/converter.rs +++ b/rust/cubesql/cubesql/src/compile/rewrite/converter.rs @@ -1,6 +1,9 @@ use crate::{ compile::{ - engine::{df::scan::CubeScanNode, provider::CubeContext}, + engine::{ + df::scan::{CubeScanNode, MemberField}, + provider::CubeContext, + }, rewrite::{ analysis::LogicalPlanAnalysis, rewriter::Rewriter, AggregateFunctionExprDistinct, AggregateFunctionExprFun, AggregateUDFExprFun, AliasExprAlias, AnyExprOp, @@ -8,12 +11,12 @@ use crate::{ CubeScanLimit, CubeScanTableName, DimensionName, EmptyRelationProduceOneRow, FilterMemberMember, FilterMemberOp, FilterMemberValues, FilterOpOp, InListExprNegated, JoinJoinConstraint, JoinJoinType, JoinLeftOn, JoinRightOn, LimitN, LiteralExprValue, - LogicalPlanLanguage, MeasureName, MemberErrorError, OrderAsc, OrderMember, - OuterColumnExprColumn, OuterColumnExprDataType, ProjectionAlias, ScalarFunctionExprFun, - ScalarUDFExprFun, ScalarVariableExprDataType, ScalarVariableExprVariable, - SegmentMemberMember, SortExprAsc, SortExprNullsFirst, TableScanLimit, - TableScanProjection, TableScanSourceTableName, TableScanTableName, TableUDFExprFun, - TimeDimensionDateRange, TimeDimensionGranularity, TimeDimensionName, + LiteralMemberValue, LogicalPlanLanguage, MeasureName, MemberErrorError, OrderAsc, + OrderMember, OuterColumnExprColumn, OuterColumnExprDataType, ProjectionAlias, + ScalarFunctionExprFun, ScalarUDFExprFun, ScalarVariableExprDataType, + ScalarVariableExprVariable, SegmentMemberMember, SortExprAsc, SortExprNullsFirst, + TableScanLimit, TableScanProjection, TableScanSourceTableName, TableScanTableName, + TableUDFExprFun, TimeDimensionDateRange, TimeDimensionGranularity, TimeDimensionName, TryCastExprDataType, UnionAlias, WindowFunctionExprFun, WindowFunctionExprWindowFrame, }, }, @@ -33,6 +36,7 @@ use datafusion::{ LogicalPlanBuilder, TableScan, Union, }, physical_plan::planner::DefaultPhysicalPlanner, + scalar::ScalarValue, sql::planner::ContextProvider, }; use egg::{EGraph, Id, RecExpr}; @@ -1081,7 +1085,7 @@ impl LanguageToLogicalPlanConverter { // TODO actually nullable. Just to fit tests false, ), - Some(measure.to_string()), + MemberField::Member(measure.to_string()), )); } LogicalPlanLanguage::TimeDimension(params) => { @@ -1120,7 +1124,10 @@ impl LanguageToLogicalPlanConverter { // TODO actually nullable. Just to fit tests false, ), - Some(format!("{}.{}", dimension, granularity)), + MemberField::Member(format!( + "{}.{}", + dimension, granularity + )), )); } } @@ -1146,7 +1153,7 @@ impl LanguageToLogicalPlanConverter { // TODO actually nullable. Just to fit tests false, ), - Some(dimension), + MemberField::Member(dimension), )); } LogicalPlanLanguage::Segment(params) => { @@ -1160,7 +1167,22 @@ impl LanguageToLogicalPlanConverter { // TODO actually nullable. Just to fit tests false, ), - None, + MemberField::Literal(ScalarValue::Boolean(None)), + )); + } + LogicalPlanLanguage::LiteralMember(params) => { + let value = + match_data_node!(node_by_id, params[0], LiteralMemberValue); + let expr = self.to_expr(params[1])?; + fields.push(( + DFField::new( + Some(&table_name), + &expr_name(&expr)?, + value.get_datatype(), + // TODO actually nullable. Just to fit tests + false, + ), + MemberField::Literal(value), )); } LogicalPlanLanguage::MemberError(params) => { @@ -1321,10 +1343,7 @@ impl LanguageToLogicalPlanConverter { ]) } - if query_measures.len() == 0 - && query_dimensions.len() == 0 - && query_time_dimensions.len() == 0 - { + if fields.len() == 0 { return Err(CubeError::internal( "Can't detect Cube query and it may be not supported yet" .to_string(), diff --git a/rust/cubesql/cubesql/src/compile/rewrite/cost.rs b/rust/cubesql/cubesql/src/compile/rewrite/cost.rs index 923b010646733..59afe5880983f 100644 --- a/rust/cubesql/cubesql/src/compile/rewrite/cost.rs +++ b/rust/cubesql/cubesql/src/compile/rewrite/cost.rs @@ -72,6 +72,7 @@ impl CostFunction for BestCubePlan { let cube_members = match enode { LogicalPlanLanguage::Measure(_) => 1, LogicalPlanLanguage::Dimension(_) => 1, + LogicalPlanLanguage::LiteralMember(_) => 1, LogicalPlanLanguage::TimeDimensionGranularity(TimeDimensionGranularity(Some(_))) => 1, LogicalPlanLanguage::MemberError(_) => 1, _ => 0, diff --git a/rust/cubesql/cubesql/src/compile/rewrite/mod.rs b/rust/cubesql/cubesql/src/compile/rewrite/mod.rs index ca468e07ed406..d7e68bb1f81f8 100644 --- a/rust/cubesql/cubesql/src/compile/rewrite/mod.rs +++ b/rust/cubesql/cubesql/src/compile/rewrite/mod.rs @@ -228,6 +228,10 @@ crate::plan_to_language! { name: String, expr: Arc, }, + LiteralMember { + value: ScalarValue, + expr: Arc, + }, Order { member: String, asc: bool, @@ -800,6 +804,10 @@ fn segment_expr(name: impl Display, expr: impl Display) -> String { format!("(Segment {} {})", name, expr) } +fn literal_member(value: impl Display, expr: impl Display) -> String { + format!("(LiteralMember {} {})", value, expr) +} + fn time_dimension_expr( name: impl Display, granularity: impl Display, diff --git a/rust/cubesql/cubesql/src/compile/rewrite/rules/members.rs b/rust/cubesql/cubesql/src/compile/rewrite/rules/members.rs index 2efea6e82f7a1..efe875ac38ad7 100644 --- a/rust/cubesql/cubesql/src/compile/rewrite/rules/members.rs +++ b/rust/cubesql/cubesql/src/compile/rewrite/rules/members.rs @@ -9,20 +9,21 @@ use crate::{ column_name_to_member_vec, cube_scan, cube_scan_filters_empty_tail, cube_scan_members, cube_scan_members_empty_tail, cube_scan_order_empty_tail, dimension_expr, expr_column_name, expr_column_name_with_relation, fun_expr, limit, - list_concat_pushdown_replacer, list_concat_pushup_replacer, literal_expr, measure_expr, - member_pushdown_replacer, member_replacer, original_expr_name, projection, - projection_expr, projection_expr_empty_tail, referenced_columns, rewrite, + list_concat_pushdown_replacer, list_concat_pushup_replacer, literal_expr, + literal_member, measure_expr, member_pushdown_replacer, member_replacer, + original_expr_name, projection, projection_expr, projection_expr_empty_tail, + referenced_columns, rewrite, rewriter::RewriteRules, rules::{replacer_push_down_node, replacer_push_down_node_substitute_rules}, segment_expr, table_scan, time_dimension_expr, transforming_chain_rewrite, transforming_rewrite, udaf_expr, AggregateFunctionExprDistinct, AggregateFunctionExprFun, AliasExprAlias, ColumnExprColumn, CubeScanAliases, CubeScanLimit, CubeScanTableName, DimensionName, LimitN, LiteralExprValue, - LogicalPlanLanguage, MeasureName, MemberErrorError, MemberErrorPriority, - MemberPushdownReplacerTableName, MemberPushdownReplacerTargetTableName, - ProjectionAlias, SegmentName, TableScanSourceTableName, TableScanTableName, - TimeDimensionDateRange, TimeDimensionGranularity, TimeDimensionName, - WithColumnRelation, + LiteralMemberValue, LogicalPlanLanguage, MeasureName, MemberErrorError, + MemberErrorPriority, MemberPushdownReplacerTableName, + MemberPushdownReplacerTargetTableName, ProjectionAlias, SegmentName, + TableScanSourceTableName, TableScanTableName, TimeDimensionDateRange, + TimeDimensionGranularity, TimeDimensionName, WithColumnRelation, }, }, transport::{V1CubeMetaDimensionExt, V1CubeMetaMeasureExt}, @@ -153,6 +154,24 @@ impl RewriteRules for MemberRules { "?member".to_string(), self.transform_projection_member("?source_table_name", "?column", None, "?member"), ), + transforming_rewrite( + "literal-member", + member_replacer(literal_expr("?value"), "?source_table_name"), + literal_member("?literal_member_value", literal_expr("?value")), + self.transform_literal_member("?value", "?literal_member_value"), + ), + transforming_rewrite( + "literal-member-alias", + member_replacer( + alias_expr(literal_expr("?value"), "?alias"), + "?source_table_name", + ), + literal_member( + "?literal_member_value", + alias_expr(literal_expr("?value"), "?alias"), + ), + self.transform_literal_member("?value", "?literal_member_value"), + ), transforming_chain_rewrite( "date-trunc", member_replacer("?original_expr", "?source_table_name"), @@ -598,6 +617,9 @@ impl MemberRules { rules.extend(member_column_pushdown("time-dimension", |column| { time_dimension_expr("?name", "?granularity", "?date_range", column) })); + rules.extend(member_column_pushdown("literal", |column| { + literal_member("?value", column) + })); fn list_concat_terminal( name: &str, @@ -1070,6 +1092,25 @@ impl MemberRules { } } + fn transform_literal_member( + &self, + literal_value_var: &'static str, + literal_member_value_var: &'static str, + ) -> impl Fn(&mut EGraph, &mut Subst) -> bool { + let literal_value_var = var!(literal_value_var); + let literal_member_value_var = var!(literal_member_value_var); + move |egraph, subst| { + for value in var_iter!(egraph[subst[literal_value_var]], LiteralExprValue).cloned() { + let literal_member_value = egraph.add(LogicalPlanLanguage::LiteralMemberValue( + LiteralMemberValue(value), + )); + subst.insert(literal_member_value_var, literal_member_value); + return true; + } + false + } + } + fn transform_projection_member( &self, cube_var: &'static str, diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_fkey_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_fkey_query.snap index 37326fa5af6b8..4d677596460fa 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_fkey_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_fkey_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n SELECT\n nspname as Schema,\n cl.relname as Table,\n clr.relname as RefTableName,\n conname as Name,\n conkey as ColumnIndexes,\n confkey as ColumnRefIndexes\n FROM pg_constraint\n INNER JOIN pg_namespace ON connamespace = pg_namespace.oid\n INNER JOIN pg_class cl ON conrelid = cl.oid\n INNER JOIN pg_class clr ON confrelid = clr.oid\n WHERE\n contype = 'f' AND\n conname like E'sample\\\\_fkey' AND\n nspname like E'public' AND\n cl.relname like E'KibanaSampleDataEcommerce'\n order by 1\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++--------+-------+--------------+------+---------------+------------------+ +| Schema | Table | RefTableName | Name | ColumnIndexes | ColumnRefIndexes | ++--------+-------+--------------+------+---------------+------------------+ ++--------+-------+--------------+------+---------------+------------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_pg_constraint_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_pg_constraint_query.snap index 27a5f0257f67d..d7b898f63aac9 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_pg_constraint_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_pg_constraint_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n SELECT\n a.conname as Name,\n ns.nspname as Schema,\n mycl.relname as Table,\n b.conname as ReferencedKey,\n frns.nspname as ReferencedSchema,\n frcl.relname as ReferencedTable,\n a.oid as Oid,\n a.conkey as ColumnIndexes,\n a.confkey as ForeignColumnIndexes,\n a.confupdtype as UpdateActionCode,\n a.confdeltype as DeleteActionCode,\n a.confmatchtype as ForeignKeyMatchType,\n a.condeferrable as IsDeferrable,\n a.condeferred as Iscondeferred\n FROM pg_constraint a\n inner join pg_constraint b on (\n a.confrelid = b.conrelid AND\n a.confkey = b.conkey\n )\n INNER JOIN pg_namespace ns ON a.connamespace = ns.oid\n INNER JOIN pg_class mycl ON a.conrelid = mycl.oid\n LEFT OUTER JOIN pg_class frcl ON a.confrelid = frcl.oid\n INNER JOIN pg_namespace frns ON frcl.relnamespace = frns.oid\n WHERE\n a.contype = 'f' AND\n (\n b.contype = 'p' OR\n b.contype = 'u'\n ) AND\n a.oid::varchar like '%' AND\n a.conname like '%' AND\n ns.nspname like E'public' AND\n mycl.relname like E'KibanaSampleDataEcommerce' AND\n frns.nspname like '%' AND\n frcl.relname like '%'\n ORDER BY 1\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++------+--------+-------+---------------+------------------+-----------------+-----+---------------+----------------------+------------------+------------------+---------------------+--------------+---------------+ +| Name | Schema | Table | ReferencedKey | ReferencedSchema | ReferencedTable | Oid | ColumnIndexes | ForeignColumnIndexes | UpdateActionCode | DeleteActionCode | ForeignKeyMatchType | IsDeferrable | Iscondeferred | ++------+--------+-------+---------------+------------------+-----------------+-----+---------------+----------------------+------------------+------------------+---------------------+--------------+---------------+ ++------+--------+-------+---------------+------------------+-----------------+-----+---------------+----------------------+------------------+------------------+---------------------+--------------+---------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_typname_big_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_typname_big_query.snap index a0e2b2f4be573..d255919973d38 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_typname_big_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__excel_typname_big_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n SELECT\n typname as name,\n n.nspname as Schema,\n pg_get_userbyid(typowner) as Definer,\n typlen as Length,\n t.oid as oid,\n typbyval as IsReferenceType,\n case\n when typtype = 'b' then 'base'\n when typtype = 'd' then 'domain'\n when typtype = 'c' then 'composite'\n when typtype = 'd' then 'pseudo'\n end as Type,\n case\n when typalign = 'c' then 'char'\n when typalign = 's' then 'short'\n when typalign = 'i' then 'int'\n else 'double'\n end as alignment,\n case\n when typstorage = 'p' then 'plain'\n when typstorage = 'e' then 'secondary'\n when typstorage = 'm' then 'compressed inline'\n else 'secondary or compressed inline'\n end as ValueStorage,\n typdefault as DefaultValue,\n description as comment\n FROM pg_type t\n LEFT OUTER JOIN\n pg_description des ON des.objoid = t.oid,\n pg_namespace n\n WHERE\n t.typnamespace = n.oid and\n t.oid::varchar like E'1033' and\n typname like E'%' and\n n.nspname like E'%' and\n pg_get_userbyid(typowner)::varchar like E'%' and\n typtype::varchar like E'c'\n ORDER BY name\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++------+--------+---------+--------+-----+-----------------+------+-----------+--------------+--------------+---------+ +| name | Schema | Definer | Length | oid | IsReferenceType | Type | alignment | ValueStorage | DefaultValue | comment | ++------+--------+---------+--------+-----+-----------------+------+-----------+--------------+--------------+---------+ ++------+--------+---------+--------+-----+-----------------+------+-----------+--------------+--------------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase-3.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase-3.snap index 120e29b55781a..f23a4113f5dbf 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase-3.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase-3.snap @@ -1,9 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 4325 -expression: "execute_query(\"SELECT\n KCU.REFERENCED_TABLE_SCHEMA PKTABLE_CAT,\n NULL PKTABLE_SCHEM,\n KCU.REFERENCED_TABLE_NAME PKTABLE_NAME,\n KCU.REFERENCED_COLUMN_NAME PKCOLUMN_NAME,\n KCU.TABLE_SCHEMA FKTABLE_CAT,\n NULL FKTABLE_SCHEM,\n KCU.TABLE_NAME FKTABLE_NAME,\n KCU.COLUMN_NAME FKCOLUMN_NAME,\n KCU.POSITION_IN_UNIQUE_CONSTRAINT KEY_SEQ,\n CASE update_rule WHEN 'RESTRICT' THEN 1 WHEN 'NO ACTION' THEN 3 WHEN 'CASCADE' THEN 0 WHEN 'SET NULL' THEN 2 WHEN 'SET DEFAULT' THEN 4 END UPDATE_RULE,\n CASE DELETE_RULE WHEN 'RESTRICT' THEN 1 WHEN 'NO ACTION' THEN 3 WHEN 'CASCADE' THEN 0 WHEN 'SET NULL' THEN 2 WHEN 'SET DEFAULT' THEN 4 END DELETE_RULE,\n RC.CONSTRAINT_NAME FK_NAME,\n NULL PK_NAME,\n 7 DEFERRABILITY\n FROM INFORMATION_SCHEMA.KEY_COLUMN_USAGE KCU\n INNER JOIN INFORMATION_SCHEMA.REFERENTIAL_CONSTRAINTS RC ON KCU.CONSTRAINT_SCHEMA = RC.CONSTRAINT_SCHEMA AND KCU.CONSTRAINT_NAME = RC.CONSTRAINT_NAME\n WHERE (ISNULL(database()) OR (KCU.TABLE_SCHEMA = database())) AND KCU.TABLE_NAME = 'SlackMessages' ORDER BY PKTABLE_CAT, PKTABLE_SCHEM, PKTABLE_NAME, KEY_SEQ\n \".to_string()).await?" - +expression: "execute_query(\"SELECT\n KCU.REFERENCED_TABLE_SCHEMA PKTABLE_CAT,\n NULL PKTABLE_SCHEM,\n KCU.REFERENCED_TABLE_NAME PKTABLE_NAME,\n KCU.REFERENCED_COLUMN_NAME PKCOLUMN_NAME,\n KCU.TABLE_SCHEMA FKTABLE_CAT,\n NULL FKTABLE_SCHEM,\n KCU.TABLE_NAME FKTABLE_NAME,\n KCU.COLUMN_NAME FKCOLUMN_NAME,\n KCU.POSITION_IN_UNIQUE_CONSTRAINT KEY_SEQ,\n CASE update_rule WHEN 'RESTRICT' THEN 1 WHEN 'NO ACTION' THEN 3 WHEN 'CASCADE' THEN 0 WHEN 'SET NULL' THEN 2 WHEN 'SET DEFAULT' THEN 4 END UPDATE_RULE,\n CASE DELETE_RULE WHEN 'RESTRICT' THEN 1 WHEN 'NO ACTION' THEN 3 WHEN 'CASCADE' THEN 0 WHEN 'SET NULL' THEN 2 WHEN 'SET DEFAULT' THEN 4 END DELETE_RULE,\n RC.CONSTRAINT_NAME FK_NAME,\n NULL PK_NAME,\n 7 DEFERRABILITY\n FROM INFORMATION_SCHEMA.KEY_COLUMN_USAGE KCU\n INNER JOIN INFORMATION_SCHEMA.REFERENTIAL_CONSTRAINTS RC ON KCU.CONSTRAINT_SCHEMA = RC.CONSTRAINT_SCHEMA AND KCU.CONSTRAINT_NAME = RC.CONSTRAINT_NAME\n WHERE (ISNULL(database()) OR (KCU.TABLE_SCHEMA = database())) AND KCU.TABLE_NAME = 'SlackMessages' ORDER BY PKTABLE_CAT, PKTABLE_SCHEM, PKTABLE_NAME, KEY_SEQ\n \".to_string(),\n DatabaseProtocol::MySQL).await?" --- -++ -++ -++ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ +| PKTABLE_CAT | PKTABLE_SCHEM | PKTABLE_NAME | PKCOLUMN_NAME | FKTABLE_CAT | FKTABLE_SCHEM | FKTABLE_NAME | FKCOLUMN_NAME | KEY_SEQ | UPDATE_RULE | DELETE_RULE | FK_NAME | PK_NAME | DEFERRABILITY | ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_pktable_cat_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_pktable_cat_query.snap index 49344d7295fec..1a774d97f4bd3 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_pktable_cat_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_pktable_cat_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n SELECT NULL::text AS pktable_cat,\n pkn.nspname AS pktable_schem,\n pkc.relname AS pktable_name,\n pka.attname AS pkcolumn_name,\n NULL::text AS fktable_cat,\n fkn.nspname AS fktable_schem,\n fkc.relname AS fktable_name,\n fka.attname AS fkcolumn_name,\n pos.n AS key_seq,\n CASE con.confupdtype\n WHEN 'c' THEN 0\n WHEN 'n' THEN 2\n WHEN 'd' THEN 4\n WHEN 'r' THEN 1\n WHEN 'p' THEN 1\n WHEN 'a' THEN 3\n ELSE NULL\n END AS update_rule,\n CASE con.confdeltype\n WHEN 'c' THEN 0\n WHEN 'n' THEN 2\n WHEN 'd' THEN 4\n WHEN 'r' THEN 1\n WHEN 'p' THEN 1\n WHEN 'a' THEN 3\n ELSE NULL\n END AS delete_rule,\n con.conname AS fk_name,\n pkic.relname AS pk_name,\n CASE\n WHEN con.condeferrable AND con.condeferred THEN 5\n WHEN con.condeferrable THEN 6\n ELSE 7\n END AS deferrability\n FROM pg_catalog.pg_namespace pkn,\n pg_catalog.pg_class pkc,\n pg_catalog.pg_attribute pka,\n pg_catalog.pg_namespace fkn,\n pg_catalog.pg_class fkc,\n pg_catalog.pg_attribute fka,\n pg_catalog.pg_constraint con,\n pg_catalog.generate_series(1, 32) pos(n),\n pg_catalog.pg_class pkic\n WHERE pkn.oid = pkc.relnamespace\n AND pkc.oid = pka.attrelid\n AND pka.attnum = con.confkey[pos.n]\n AND con.confrelid = pkc.oid\n AND fkn.oid = fkc.relnamespace\n AND fkc.oid = fka.attrelid\n AND fka.attnum = con.conkey[pos.n]\n AND con.conrelid = fkc.oid\n AND con.contype = 'f'\n AND (pkic.relkind = 'i' OR pkic.relkind = 'I')\n AND pkic.oid = con.conindid\n AND fkn.nspname = 'public'\n AND fkc.relname = 'actor'\n ORDER BY pkn.nspname, pkc.relname, con.conname, pos.n;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ +| pktable_cat | pktable_schem | pktable_name | pkcolumn_name | fktable_cat | fktable_schem | fktable_name | fkcolumn_name | key_seq | update_rule | delete_rule | fk_name | pk_name | deferrability | ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_table_cat_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_table_cat_query.snap index dd1312c4e3313..0737948ab4187 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_table_cat_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__metabase_table_cat_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n SELECT result.table_cat,\n result.table_schem,\n result.table_name,\n result.column_name,\n result.key_seq,\n result.pk_name\n FROM (\n SELECT NULL AS table_cat,\n n.nspname AS table_schem,\n ct.relname AS table_name,\n a.attname AS column_name,\n (information_schema._pg_expandarray(i.indkey)).n as key_seq,\n ci.relname AS pk_name,\n information_schema._pg_expandarray(i.indkey) AS keys,\n a.attnum AS a_attnum\n FROM pg_catalog.pg_class ct\n JOIN pg_catalog.pg_attribute a ON(ct.oid = a.attrelid)\n JOIN pg_catalog.pg_namespace n ON (ct.relnamespace = n.oid)\n JOIN pg_catalog.pg_index i ON (a.attrelid = i.indrelid)\n JOIN pg_catalog.pg_class ci ON (ci.oid = i.indexrelid)\n WHERE true AND ct.relname = 'actor' AND i.indisprimary) result\n WHERE result.a_attnum = (result.keys).x\n ORDER BY result.table_name, result.pk_name, result.key_seq;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-----------+-------------+------------+-------------+---------+---------+ +| table_cat | table_schem | table_name | column_name | key_seq | pk_name | ++-----------+-------------+------------+-------------+---------+---------+ ++-----------+-------------+------------+-------------+---------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__pgcatalog_pgdepend_postgres.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__pgcatalog_pgdepend_postgres.snap index 12a14fd240024..da0ac0ed3e6cd 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__pgcatalog_pgdepend_postgres.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__pgcatalog_pgdepend_postgres.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"SELECT * FROM pg_catalog.pg_depend ORDER BY refclassid ASC, refobjid ASC\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++---------+-------+----------+------------+----------+-------------+---------+ +| classid | objid | objsubid | refclassid | refobjid | refobjsubid | deptype | ++---------+-------+----------+------------+----------+-------------+---------+ ++---------+-------+----------+------------+----------+-------------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_composite_types.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_composite_types.snap index 069d4b24b76e9..2fc0e265f9fff 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_composite_types.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_composite_types.snap @@ -1,8 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 7136 expression: "execute_query(\"/*** Load field definitions for (free-standing) composite types ***/\n SELECT typ.oid, att.attname, att.atttypid\n FROM pg_type AS typ\n JOIN pg_namespace AS ns ON (ns.oid = typ.typnamespace)\n JOIN pg_class AS cls ON (cls.oid = typ.typrelid)\n JOIN pg_attribute AS att ON (att.attrelid = typ.typrelid)\n WHERE\n (typ.typtype = 'c' AND cls.relkind='c') AND\n attnum > 0 AND /* Don't load system attributes */\n NOT attisdropped\n ORDER BY typ.oid, att.attnum\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-----+---------+----------+ +| oid | attname | atttypid | ++-----+---------+----------+ ++-----+---------+----------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_enums.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_enums.snap index 471e0fa3cb707..f4f989f703995 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_enums.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_enums.snap @@ -1,8 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 7156 expression: "execute_query(\"/*** Load enum fields ***/\n SELECT pg_type.oid, enumlabel\n FROM pg_enum\n JOIN pg_type ON pg_type.oid=enumtypid\n ORDER BY oid, enumsortorder\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-----+-----------+ +| oid | enumlabel | ++-----+-----------+ ++-----+-----------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_from_subquery.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_from_subquery.snap index 5bdc71dbcd351..78c83c882f30c 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_from_subquery.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_from_subquery.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n select\n pkcol.COLUMN_NAME as PK_COLUMN_NAME,\n fkcol.TABLE_SCHEMA AS FK_TABLE_SCHEMA,\n fkcol.TABLE_NAME AS FK_TABLE_NAME,\n fkcol.COLUMN_NAME as FK_COLUMN_NAME,\n fkcol.ORDINAL_POSITION as ORDINAL,\n fkcon.CONSTRAINT_SCHEMA || '_' || fkcol.TABLE_NAME || '_' || 'users' || '_' || fkcon.CONSTRAINT_NAME as FK_NAME\n from\n (select distinct constraint_catalog, constraint_schema, unique_constraint_schema, constraint_name, unique_constraint_name\n from INFORMATION_SCHEMA.REFERENTIAL_CONSTRAINTS) fkcon\n inner join\n INFORMATION_SCHEMA.KEY_COLUMN_USAGE fkcol\n on fkcon.CONSTRAINT_SCHEMA = fkcol.CONSTRAINT_SCHEMA\n and fkcon.CONSTRAINT_NAME = fkcol.CONSTRAINT_NAME\n inner join\n INFORMATION_SCHEMA.KEY_COLUMN_USAGE pkcol\n on fkcon.UNIQUE_CONSTRAINT_SCHEMA = pkcol.CONSTRAINT_SCHEMA\n and fkcon.UNIQUE_CONSTRAINT_NAME = pkcol.CONSTRAINT_NAME\n where pkcol.TABLE_SCHEMA = 'public' and pkcol.TABLE_NAME = 'users'\n and pkcol.ORDINAL_POSITION = fkcol.ORDINAL_POSITION\n order by FK_NAME, fkcol.ORDINAL_POSITION\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++----------------+-----------------+---------------+----------------+---------+---------+ +| PK_COLUMN_NAME | FK_TABLE_SCHEMA | FK_TABLE_NAME | FK_COLUMN_NAME | ORDINAL | FK_NAME | ++----------------+-----------------+---------------+----------------+---------+---------+ ++----------------+-----------------+---------------+----------------+---------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_uppercase_alias.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_uppercase_alias.snap index c84033ab2c782..007487c1625c8 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_uppercase_alias.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__powerbi_uppercase_alias.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"\n select\n i.CONSTRAINT_SCHEMA || '_' || i.CONSTRAINT_NAME as INDEX_NAME,\n ii.COLUMN_NAME,\n ii.ORDINAL_POSITION,\n case\n when i.CONSTRAINT_TYPE = 'PRIMARY KEY' then 'Y'\n else 'N'\n end as PRIMARY_KEY\n from INFORMATION_SCHEMA.table_constraints i\n inner join INFORMATION_SCHEMA.key_column_usage ii on\n i.CONSTRAINT_SCHEMA = ii.CONSTRAINT_SCHEMA and\n i.CONSTRAINT_NAME = ii.CONSTRAINT_NAME and\n i.TABLE_SCHEMA = ii.TABLE_SCHEMA and\n i.TABLE_NAME = ii.TABLE_NAME\n where\n i.TABLE_SCHEMA = 'public' and\n i.TABLE_NAME = 'KibanaSampleDataEcommerce' and\n i.CONSTRAINT_TYPE in ('PRIMARY KEY', 'UNIQUE')\n order by\n i.CONSTRAINT_SCHEMA || '_' || i.CONSTRAINT_NAME,\n ii.TABLE_SCHEMA,\n ii.TABLE_NAME,\n ii.ORDINAL_POSITION\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++------------+-------------+------------------+-------------+ +| INDEX_NAME | COLUMN_NAME | ORDINAL_POSITION | PRIMARY_KEY | ++------------+-------------+------------------+-------------+ ++------------+-------------+------------------+-------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__show_variables_like_aurora.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__show_variables_like_aurora.snap new file mode 100644 index 0000000000000..0410e043eee6e --- /dev/null +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__show_variables_like_aurora.snap @@ -0,0 +1,8 @@ +--- +source: cubesql/src/compile/mod.rs +expression: "execute_query(\"show variables like 'aurora_version';\".to_string(),\n DatabaseProtocol::MySQL).await?" +--- ++---------------+-------+ +| Variable_name | Value | ++---------------+-------+ ++---------------+-------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__sigma_computing_array_subquery_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__sigma_computing_array_subquery_query.snap index d57be4b589d67..8a13a51fdf3b8 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__sigma_computing_array_subquery_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__sigma_computing_array_subquery_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(r#\"\n select\n cl.relname as \"source_table\",\n array(\n select (\n select attname::text\n from pg_attribute\n where\n attrelid = con.conrelid and\n attnum = con.conkey[i]\n )\n from generate_series(array_lower(con.conkey, 1), array_upper(con.conkey, 1)) i\n ) as \"source_keys\",\n (\n select nspname\n from pg_namespace ns2\n join pg_class cl2 on ns2.oid = cl2.relnamespace\n where cl2.oid = con.confrelid\n ) as \"target_schema\",\n (\n select relname\n from pg_class\n where oid = con.confrelid\n ) as \"target_table\",\n array(\n select (\n select attname::text\n from pg_attribute\n where\n attrelid = con.confrelid and\n attnum = con.confkey[i]\n )\n from generate_series(array_lower(con.confkey, 1), array_upper(con.confkey, 1)) i\n ) as \"target_keys\"\n from pg_class cl\n join pg_namespace ns on cl.relnamespace = ns.oid\n join pg_constraint con on con.conrelid = cl.oid\n where\n ns.nspname = 'public' and\n cl.relname >= 'A' and\n cl.relname <= 'z' and\n con.contype = 'f'\n order by\n \"source_table\",\n con.conname\n ;\n \"#.to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++--------------+-------------+---------------+--------------+-------------+ +| source_table | source_keys | target_schema | target_table | target_keys | ++--------------+-------------+---------------+--------------+-------------+ ++--------------+-------------+---------------+--------------+-------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__subquery_with_same_name_excel.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__subquery_with_same_name_excel.snap index 1edcd58d1b056..bd07e21b668de 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__subquery_with_same_name_excel.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__subquery_with_same_name_excel.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"SELECT oid, (SELECT oid FROM pg_type WHERE typname like 'geography') as dd FROM pg_type WHERE typname like 'geometry'\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-----+----+ +| oid | dd | ++-----+----+ ++-----+----+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_attname_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_attname_query.snap index adf5492ac6f52..26f6501328c38 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_attname_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_attname_query.snap @@ -1,8 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 5903 expression: "execute_query(r#\"SELECT a.attname\n FROM pg_attribute a JOIN (\n SELECT unnest(ix.indkey) attnum,\n generate_subscripts(ix.indkey, 1) ord\n FROM pg_index ix\n WHERE ix.indrelid = 13449 AND ix.indisprimary\n ) k ON a.attnum=k.attnum\n WHERE a.attrelid = 13449\n ORDER BY k.ord\n \"#.to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++---------+ +| attname | ++---------+ ++---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_conname_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_conname_query.snap index 3680b6d5c5331..7b2da5c2e8daa 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_conname_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_conname_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(r#\"SELECT r.conname,\n pg_catalog.pg_get_constraintdef(r.oid, true) as condef,\n n.nspname as conschema\n FROM pg_catalog.pg_constraint r,\n pg_namespace n,\n pg_class c\n WHERE r.conrelid = 13449 AND\n r.contype = 'f' AND\n c.oid = confrelid AND\n n.oid = c.relnamespace\n ORDER BY 1\n \"#.to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++---------+--------+-----------+ +| conname | condef | conschema | ++---------+--------+-----------+ ++---------+--------+-----------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_visible_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_visible_query.snap index 3a44669a311e6..1e28c08335ef1 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_visible_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__superset_visible_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(r#\"\n SELECT\n t.typname as \"name\",\n pg_catalog.pg_type_is_visible(t.oid) as \"visible\",\n n.nspname as \"schema\",\n e.enumlabel as \"label\"\n FROM pg_catalog.pg_type t\n LEFT JOIN pg_catalog.pg_namespace n ON n.oid = t.typnamespace\n LEFT JOIN pg_catalog.pg_enum e ON t.oid = e.enumtypid\n WHERE t.typtype = 'e'\n ORDER BY\n \"schema\",\n \"name\",\n e.oid\n ;\n \"#.to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++------+---------+--------+-------+ +| name | visible | schema | label | ++------+---------+--------+-------+ ++------+---------+--------+-------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_constraints.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_constraints.snap index 01a4741fa0945..5af3b2c83bd6a 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_constraints.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_constraints.snap @@ -1,8 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 7694 expression: "execute_query(\"select\t'test'::name as PKTABLE_CAT,\n n2.nspname as PKTABLE_SCHEM,\n c2.relname as PKTABLE_NAME,\n a2.attname as PKCOLUMN_NAME,\n 'test'::name as FKTABLE_CAT,\n n1.nspname as FKTABLE_SCHEM,\n c1.relname as FKTABLE_NAME,\n a1.attname as FKCOLUMN_NAME,\n i::int2 as KEY_SEQ,\n case ref.confupdtype\n when 'c' then 0::int2\n when 'n' then 2::int2\n when 'd' then 4::int2\n when 'r' then 1::int2\n else 3::int2\n end as UPDATE_RULE,\n case ref.confdeltype\n when 'c' then 0::int2\n when 'n' then 2::int2\n when 'd' then 4::int2\n when 'r' then 1::int2\n else 3::int2\n end as DELETE_RULE,\n ref.conname as FK_NAME,\n cn.conname as PK_NAME,\n case\n when ref.condeferrable then\n case\n when ref.condeferred then 5::int2\n else 6::int2\n end\n else 7::int2\n end as DEFERRABLITY\n from\n ((((((( (select cn.oid, conrelid, conkey, confrelid, confkey,\n generate_series(array_lower(conkey, 1), array_upper(conkey, 1)) as i,\n confupdtype, confdeltype, conname,\n condeferrable, condeferred\n from pg_catalog.pg_constraint cn,\n pg_catalog.pg_class c,\n pg_catalog.pg_namespace n\n where contype = 'f'\n and conrelid = c.oid\n and relname = 'KibanaSampleDataEcommerce'\n and n.oid = c.relnamespace\n and n.nspname = 'public'\n ) ref\n inner join pg_catalog.pg_class c1\n on c1.oid = ref.conrelid)\n inner join pg_catalog.pg_namespace n1\n on n1.oid = c1.relnamespace)\n inner join pg_catalog.pg_attribute a1\n on a1.attrelid = c1.oid\n and a1.attnum = conkey[i])\n inner join pg_catalog.pg_class c2\n on c2.oid = ref.confrelid)\n inner join pg_catalog.pg_namespace n2\n on n2.oid = c2.relnamespace)\n inner join pg_catalog.pg_attribute a2\n on a2.attrelid = c2.oid\n and a2.attnum = confkey[i])\n left outer join pg_catalog.pg_constraint cn\n on cn.conrelid = ref.confrelid\n and cn.contype = 'p')\n order by ref.oid, ref.i;\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+--------------+ +| PKTABLE_CAT | PKTABLE_SCHEM | PKTABLE_NAME | PKCOLUMN_NAME | FKTABLE_CAT | FKTABLE_SCHEM | FKTABLE_NAME | FKCOLUMN_NAME | KEY_SEQ | UPDATE_RULE | DELETE_RULE | FK_NAME | PK_NAME | DEFERRABLITY | ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+--------------+ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+--------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_indexes.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_indexes.snap index b620098926f64..a36e9818f6b5c 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_indexes.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_indexes.snap @@ -1,8 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 7929 expression: "execute_query(\"SELECT\n ta.attname,\n ia.attnum,\n ic.relname,\n n.nspname,\n tc.relname\n FROM\n pg_catalog.pg_attribute ta,\n pg_catalog.pg_attribute ia,\n pg_catalog.pg_class tc,\n pg_catalog.pg_index i,\n pg_catalog.pg_namespace n,\n pg_catalog.pg_class ic\n WHERE\n tc.relname = 'KibanaSampleDataEcommerce'\n AND n.nspname = 'public'\n AND tc.oid = i.indrelid\n AND n.oid = tc.relnamespace\n AND i.indisprimary = 't'\n AND ia.attrelid = i.indexrelid\n AND ta.attrelid = i.indrelid\n AND ta.attnum = i.indkey [ia.attnum-1]\n AND (NOT ta.attisdropped)\n AND (NOT ia.attisdropped)\n AND ic.oid = i.indexrelid\n ORDER BY\n ia.attnum;\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++---------+--------+---------+---------+---------+ +| attname | attnum | relname | nspname | relname | ++---------+--------+---------+---------+---------+ ++---------+--------+---------+---------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_pkeys.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_pkeys.snap index 9e3570abc9dec..a36e9818f6b5c 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_pkeys.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_pkeys.snap @@ -1,8 +1,8 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 7965 expression: "execute_query(\"SELECT\n ta.attname,\n ia.attnum,\n ic.relname,\n n.nspname,\n tc.relname\n FROM\n pg_catalog.pg_attribute ta,\n pg_catalog.pg_attribute ia,\n pg_catalog.pg_class tc,\n pg_catalog.pg_index i,\n pg_catalog.pg_namespace n,\n pg_catalog.pg_class ic\n WHERE\n tc.relname = 'KibanaSampleDataEcommerce'\n AND n.nspname = 'public'\n AND tc.oid = i.indrelid\n AND n.oid = tc.relnamespace\n AND i.indisprimary = 't'\n AND ia.attrelid = i.indexrelid\n AND ta.attrelid = i.indrelid\n AND ta.attnum = i.indkey [ia.attnum-1]\n AND (NOT ta.attisdropped)\n AND (NOT ia.attisdropped)\n AND ic.oid = i.indexrelid\n ORDER BY\n ia.attnum;\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++---------+--------+---------+---------+---------+ +| attname | attnum | relname | nspname | relname | ++---------+--------+---------+---------+---------+ ++---------+--------+---------+---------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_table_columns.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_table_columns.snap index 50404d42a78af..d83b3180ce909 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_table_columns.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_desktop_table_columns.snap @@ -1,19 +1,18 @@ --- source: cubesql/src/compile/mod.rs -assertion_line: 7879 expression: "execute_query(\"select\n n.nspname,\n c.relname,\n a.attname,\n a.atttypid,\n t.typname,\n a.attnum,\n a.attlen,\n a.atttypmod,\n a.attnotnull,\n c.relhasrules,\n c.relkind,\n c.oid,\n pg_get_expr(d.adbin, d.adrelid),\n case\n t.typtype\n when 'd' then t.typbasetype\n else 0\n end,\n t.typtypmod,\n c.relhasoids\n from\n (\n (\n (\n pg_catalog.pg_class c\n inner join pg_catalog.pg_namespace n on n.oid = c.relnamespace\n and c.oid = 18000\n )\n inner join pg_catalog.pg_attribute a on (not a.attisdropped)\n and a.attnum > 0\n and a.attrelid = c.oid\n )\n inner join pg_catalog.pg_type t on t.oid = a.atttypid\n )\n /* Attention, We have hack for on a.atthasdef */\n left outer join pg_attrdef d on a.atthasdef and d.adrelid = a.attrelid and d.adnum = a.attnum\n order by\n n.nspname,\n c.relname,\n attnum;\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -+---------+---------------------------+--------------------+----------+-----------+--------+--------+-----------+------------+-------------+---------+-------+--------------------------------+----------------------------------------------------------------------+-----------+------------+ -| nspname | relname | attname | atttypid | typname | attnum | attlen | atttypmod | attnotnull | relhasrules | relkind | oid | pg_get_expr(d.adbin,d.adrelid) | CASE #t.typtype WHEN Utf8("d") THEN #t.typbasetype ELSE Int64(0) END | typtypmod | relhasoids | -+---------+---------------------------+--------------------+----------+-----------+--------+--------+-----------+------------+-------------+---------+-------+--------------------------------+----------------------------------------------------------------------+-----------+------------+ -| public | KibanaSampleDataEcommerce | count | 20 | int8 | 1 | 8 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | maxPrice | 1700 | numeric | 2 | -1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | minPrice | 1700 | numeric | 3 | -1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | avgPrice | 1700 | numeric | 4 | -1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | order_date | 1114 | timestamp | 5 | 8 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | customer_gender | 25 | text | 6 | -1 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | taxful_total_price | 1700 | numeric | 7 | -1 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | has_subscription | 16 | bool | 8 | 1 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | is_male | 16 | bool | 9 | 1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | -| public | KibanaSampleDataEcommerce | is_female | 16 | bool | 10 | 1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | -+---------+---------------------------+--------------------+----------+-----------+--------+--------+-----------+------------+-------------+---------+-------+--------------------------------+----------------------------------------------------------------------+-----------+------------+ ++---------+---------------------------+--------------------+----------+-----------+--------+--------+-----------+------------+-------------+---------+-------+--------------------------------+--------------------------------------------------------------------+-----------+------------+ +| nspname | relname | attname | atttypid | typname | attnum | attlen | atttypmod | attnotnull | relhasrules | relkind | oid | pg_get_expr(d.adbin,d.adrelid) | CASE t.typtype WHEN Utf8("d") THEN t.typbasetype ELSE Int64(0) END | typtypmod | relhasoids | ++---------+---------------------------+--------------------+----------+-----------+--------+--------+-----------+------------+-------------+---------+-------+--------------------------------+--------------------------------------------------------------------+-----------+------------+ +| public | KibanaSampleDataEcommerce | count | 20 | int8 | 1 | 8 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | maxPrice | 1700 | numeric | 2 | -1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | minPrice | 1700 | numeric | 3 | -1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | avgPrice | 1700 | numeric | 4 | -1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | order_date | 1114 | timestamp | 5 | 8 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | customer_gender | 25 | text | 6 | -1 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | taxful_total_price | 1700 | numeric | 7 | -1 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | has_subscription | 16 | bool | 8 | 1 | -1 | false | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | is_male | 16 | bool | 9 | 1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | +| public | KibanaSampleDataEcommerce | is_female | 16 | bool | 10 | 1 | -1 | true | false | r | 18000 | NULL | 0 | -1 | false | ++---------+---------------------------+--------------------+----------+-----------+--------+--------+-----------+------------+-------------+---------+-------+--------------------------------+--------------------------------------------------------------------+-----------+------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_get_expr_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_get_expr_query.snap index 5deddc397c173..a8a238c1d11b5 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_get_expr_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_get_expr_query.snap @@ -2,6 +2,7 @@ source: cubesql/src/compile/mod.rs expression: "execute_query(\"SELECT c.oid, a.attnum, a.attname, c.relname, n.nspname, a.attnotnull OR ( t.typtype = 'd' AND t.typnotnull ), a.attidentity != '' OR pg_catalog.Pg_get_expr(d.adbin, d.adrelid) LIKE '%nextval(%'\n FROM pg_catalog.pg_class c\n JOIN pg_catalog.pg_namespace n\n ON ( c.relnamespace = n.oid )\n JOIN pg_catalog.pg_attribute a\n ON ( c.oid = a.attrelid )\n JOIN pg_catalog.pg_type t\n ON ( a.atttypid = t.oid )\n LEFT JOIN pg_catalog.pg_attrdef d\n ON ( d.adrelid = a.attrelid AND d.adnum = a.attnum )\n JOIN (SELECT 2615 AS oid, 2 AS attnum UNION ALL SELECT 1259, 2 UNION ALL SELECT 2609, 4) vals\n ON ( c.oid = vals.oid AND a.attnum = vals.attnum );\".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-----+--------+---------+---------+---------+--------------------------------------------------------+-------------------------------------------------------------------------------------+ +| oid | attnum | attname | relname | nspname | a.attnotnull OR t.typtype = Utf8("d") AND t.typnotnull | a.attidentity != Utf8("") OR pg_get_expr(d.adbin,d.adrelid) LIKE Utf8("%nextval(%") | ++-----+--------+---------+---------+---------+--------------------------------------------------------+-------------------------------------------------------------------------------------+ ++-----+--------+---------+---------+---------+--------------------------------------------------------+-------------------------------------------------------------------------------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_null_text_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_null_text_query.snap index 1c63471385cd2..1c8e01752afd2 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_null_text_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_null_text_query.snap @@ -1,7 +1,8 @@ --- source: cubesql/src/compile/mod.rs -expression: "execute_query(\"\n SELECT\n NULL::text AS PKTABLE_CAT,\n pkn.nspname AS PKTABLE_SCHEM,\n pkc.relname AS PKTABLE_NAME,\n pka.attname AS PKCOLUMN_NAME,\n NULL::text AS FKTABLE_CAT,\n fkn.nspname AS FKTABLE_SCHEM,\n fkc.relname AS FKTABLE_NAME,\n fka.attname AS FKCOLUMN_NAME,\n /*!TODO pos.n AS KEY_SEQ, */\n CASE con.confupdtype\n WHEN 'c' THEN 0\n WHEN 'n' THEN 2\n WHEN 'd' THEN 4\n WHEN 'r' THEN 1\n WHEN 'p' THEN 1\n WHEN 'a' THEN 3\n ELSE NULL\n END AS UPDATE_RULE,\n CASE con.confdeltype\n WHEN 'c' THEN 0\n WHEN 'n' THEN 2\n WHEN 'd' THEN 4\n WHEN 'r' THEN 1\n WHEN 'p' THEN 1\n WHEN 'a' THEN 3\n ELSE NULL\n END AS DELETE_RULE,\n con.conname AS FK_NAME,\n pkic.relname AS PK_NAME,\n CASE\n WHEN con.condeferrable AND con.condeferred THEN 5\n WHEN con.condeferrable THEN 6\n ELSE 7\n END AS DEFERRABILITY\n FROM\n pg_catalog.pg_namespace pkn,\n pg_catalog.pg_class pkc,\n pg_catalog.pg_attribute pka,\n pg_catalog.pg_namespace fkn,\n pg_catalog.pg_class fkc,\n pg_catalog.pg_attribute fka,\n pg_catalog.pg_constraint con,\n /*!TODO pg_catalog.generate_series(1, 32) pos(n), */\n pg_catalog.pg_class pkic\n WHERE\n pkn.oid = pkc.relnamespace AND\n pkc.oid = pka.attrelid AND\n /*!TODO pka.attnum = con.confkey[pos.n] AND */\n con.confrelid = pkc.oid AND\n fkn.oid = fkc.relnamespace AND\n fkc.oid = fka.attrelid AND\n /*!TODO fka.attnum = con.conkey[pos.n] AND */\n con.conrelid = fkc.oid AND\n con.contype = 'f' AND\n (pkic.relkind = 'i' OR pkic.relkind = 'I') AND\n pkic.oid = con.conindid AND\n fkn.nspname = 'public' AND\n fkc.relname = 'payment'\n ORDER BY\n pkn.nspname,\n pkc.relname,\n con.conname/*!TODO ,\n pos.n */\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" +expression: "execute_query(\"\n SELECT\n NULL::text AS PKTABLE_CAT,\n pkn.nspname AS PKTABLE_SCHEM,\n pkc.relname AS PKTABLE_NAME,\n pka.attname AS PKCOLUMN_NAME,\n NULL::text AS FKTABLE_CAT,\n fkn.nspname AS FKTABLE_SCHEM,\n fkc.relname AS FKTABLE_NAME,\n fka.attname AS FKCOLUMN_NAME,\n pos.n AS KEY_SEQ,\n CASE con.confupdtype\n WHEN 'c' THEN 0\n WHEN 'n' THEN 2\n WHEN 'd' THEN 4\n WHEN 'r' THEN 1\n WHEN 'p' THEN 1\n WHEN 'a' THEN 3\n ELSE NULL\n END AS UPDATE_RULE,\n CASE con.confdeltype\n WHEN 'c' THEN 0\n WHEN 'n' THEN 2\n WHEN 'd' THEN 4\n WHEN 'r' THEN 1\n WHEN 'p' THEN 1\n WHEN 'a' THEN 3\n ELSE NULL\n END AS DELETE_RULE,\n con.conname AS FK_NAME,\n pkic.relname AS PK_NAME,\n CASE\n WHEN con.condeferrable AND con.condeferred THEN 5\n WHEN con.condeferrable THEN 6\n ELSE 7\n END AS DEFERRABILITY\n FROM\n pg_catalog.pg_namespace pkn,\n pg_catalog.pg_class pkc,\n pg_catalog.pg_attribute pka,\n pg_catalog.pg_namespace fkn,\n pg_catalog.pg_class fkc,\n pg_catalog.pg_attribute fka,\n pg_catalog.pg_constraint con,\n pg_catalog.generate_series(1, 32) pos(n),\n pg_catalog.pg_class pkic\n WHERE\n pkn.oid = pkc.relnamespace AND\n pkc.oid = pka.attrelid AND\n pka.attnum = con.confkey[pos.n] AND\n con.confrelid = pkc.oid AND\n fkn.oid = fkc.relnamespace AND\n fkc.oid = fka.attrelid AND\n fka.attnum = con.conkey[pos.n] AND\n con.conrelid = fkc.oid AND\n con.contype = 'f' AND\n (pkic.relkind = 'i' OR pkic.relkind = 'I') AND\n pkic.oid = con.conindid AND\n fkn.nspname = 'public' AND\n fkc.relname = 'payment'\n ORDER BY\n pkn.nspname,\n pkc.relname,\n con.conname,\n pos.n\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ +| PKTABLE_CAT | PKTABLE_SCHEM | PKTABLE_NAME | PKCOLUMN_NAME | FKTABLE_CAT | FKTABLE_SCHEM | FKTABLE_NAME | FKCOLUMN_NAME | KEY_SEQ | UPDATE_RULE | DELETE_RULE | FK_NAME | PK_NAME | DEFERRABILITY | ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ ++-------------+---------------+--------------+---------------+-------------+---------------+--------------+---------------+---------+-------------+-------------+---------+---------+---------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_cat_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_cat_query.snap index 75638ba3073a3..adf0317c28e02 100644 --- a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_cat_query.snap +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_cat_query.snap @@ -1,7 +1,8 @@ --- source: cubesql/src/compile/mod.rs -expression: "execute_query(\"\n SELECT\n result.TABLE_CAT,\n result.TABLE_SCHEM,\n result.TABLE_NAME,\n result.COLUMN_NAME,\n /*!TODO result.KEY_SEQ, */\n result.PK_NAME\n FROM\n (\n SELECT\n NULL AS TABLE_CAT,\n n.nspname AS TABLE_SCHEM,\n ct.relname AS TABLE_NAME,\n a.attname AS COLUMN_NAME,\n /*!TODO (information_schema._pg_expandarray(i.indkey)).n AS KEY_SEQ, */\n ci.relname AS PK_NAME,\n /*!TODO information_schema._pg_expandarray(i.indkey) AS KEYS, */\n a.attnum AS A_ATTNUM\n FROM pg_catalog.pg_class ct\n JOIN pg_catalog.pg_attribute a ON (ct.oid = a.attrelid)\n JOIN pg_catalog.pg_namespace n ON (ct.relnamespace = n.oid)\n JOIN pg_catalog.pg_index i ON (a.attrelid = i.indrelid)\n JOIN pg_catalog.pg_class ci ON (ci.oid = i.indexrelid)\n WHERE\n true AND\n n.nspname = 'public' AND\n ct.relname = 'payment' AND\n i.indisprimary\n ) result\n /*!TODO where result.A_ATTNUM = (result.KEYS).x */\n ORDER BY\n result.table_name,\n result.pk_name/*!TODO ,\n result.key_seq */\n ;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" +expression: "execute_query(\"\n SELECT\n result.TABLE_CAT,\n result.TABLE_SCHEM,\n result.TABLE_NAME,\n result.COLUMN_NAME,\n result.KEY_SEQ,\n result.PK_NAME\n FROM\n (\n SELECT\n NULL AS TABLE_CAT,\n n.nspname AS TABLE_SCHEM,\n ct.relname AS TABLE_NAME,\n a.attname AS COLUMN_NAME,\n (information_schema._pg_expandarray(i.indkey)).n AS KEY_SEQ,\n ci.relname AS PK_NAME,\n information_schema._pg_expandarray(i.indkey) AS KEYS,\n a.attnum AS A_ATTNUM\n FROM pg_catalog.pg_class ct\n JOIN pg_catalog.pg_attribute a ON (ct.oid = a.attrelid)\n JOIN pg_catalog.pg_namespace n ON (ct.relnamespace = n.oid)\n JOIN pg_catalog.pg_index i ON (a.attrelid = i.indrelid)\n JOIN pg_catalog.pg_class ci ON (ci.oid = i.indexrelid)\n WHERE\n true AND\n n.nspname = 'public' AND\n ct.relname = 'payment' AND\n i.indisprimary\n ) result\n where result.A_ATTNUM = (result.KEYS).x\n ORDER BY\n result.table_name,\n result.pk_name,\n result.key_seq;\n \".to_string(),\n DatabaseProtocol::PostgreSQL).await?" --- -++ -++ -++ ++-----------+-------------+------------+-------------+---------+---------+ +| TABLE_CAT | TABLE_SCHEM | TABLE_NAME | COLUMN_NAME | KEY_SEQ | PK_NAME | ++-----------+-------------+------------+-------------+---------+---------+ ++-----------+-------------+------------+-------------+---------+---------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_name_column_name_query.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_name_column_name_query.snap new file mode 100644 index 0000000000000..244d6c7014d6f --- /dev/null +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__tableau_table_name_column_name_query.snap @@ -0,0 +1,8 @@ +--- +source: cubesql/src/compile/mod.rs +expression: "execute_query(\"SELECT `table_name`, `column_name`\n FROM `information_schema`.`columns`\n WHERE `data_type`='enum' AND `table_schema`='db'\".to_string(),\n DatabaseProtocol::MySQL).await?" +--- ++------------+-------------+ +| TABLE_NAME | COLUMN_NAME | ++------------+-------------+ ++------------+-------------+ diff --git a/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__test_information_schema_stats_for_columns.snap b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__test_information_schema_stats_for_columns.snap new file mode 100644 index 0000000000000..242c333a23055 --- /dev/null +++ b/rust/cubesql/cubesql/src/compile/snapshots/cubesql__compile__tests__test_information_schema_stats_for_columns.snap @@ -0,0 +1,8 @@ +--- +source: cubesql/src/compile/mod.rs +expression: "execute_query(\"\n SELECT\n A.TABLE_SCHEMA TABLE_CAT, NULL TABLE_SCHEM, A.TABLE_NAME, A.COLUMN_NAME, B.SEQ_IN_INDEX KEY_SEQ, B.INDEX_NAME PK_NAME\n FROM INFORMATION_SCHEMA.COLUMNS A, INFORMATION_SCHEMA.STATISTICS B\n WHERE A.COLUMN_KEY in ('PRI','pri') AND B.INDEX_NAME='PRIMARY' AND (ISNULL(database()) OR (A.TABLE_SCHEMA = database())) AND (ISNULL(database()) OR (B.TABLE_SCHEMA = database())) AND A.TABLE_NAME = 'OutlierFingerprints' AND B.TABLE_NAME = 'OutlierFingerprints' AND A.TABLE_SCHEMA = B.TABLE_SCHEMA AND A.TABLE_NAME = B.TABLE_NAME AND A.COLUMN_NAME = B.COLUMN_NAME\n ORDER BY A.COLUMN_NAME\".to_string(),\n DatabaseProtocol::MySQL).await?" +--- ++-----------+-------------+------------+-------------+---------+---------+ +| TABLE_CAT | TABLE_SCHEM | TABLE_NAME | COLUMN_NAME | KEY_SEQ | PK_NAME | ++-----------+-------------+------------+-------------+---------+---------+ ++-----------+-------------+------------+-------------+---------+---------+ diff --git a/rust/cubesql/cubesql/src/sql/dataframe.rs b/rust/cubesql/cubesql/src/sql/dataframe.rs index 286bbcb53e63c..3d078389f9d5b 100644 --- a/rust/cubesql/cubesql/src/sql/dataframe.rs +++ b/rust/cubesql/cubesql/src/sql/dataframe.rs @@ -16,7 +16,7 @@ use datafusion::arrow::{ ListArray, StringArray, TimestampMicrosecondArray, TimestampNanosecondArray, UInt16Array, UInt32Array, UInt64Array, UInt8Array, }, - datatypes::{DataType, IntervalUnit, TimeUnit}, + datatypes::{DataType, IntervalUnit, Schema, TimeUnit}, record_batch::RecordBatch, temporal_conversions, }; @@ -388,21 +388,21 @@ pub fn arrow_to_column_type(arrow_type: DataType) -> Result) -> Result { +pub fn batch_to_dataframe( + schema: &Schema, + batches: &Vec, +) -> Result { let mut cols = vec![]; let mut all_rows = vec![]; + for (_i, field) in schema.fields().iter().enumerate() { + cols.push(Column::new( + field.name().clone(), + arrow_to_column_type(field.data_type().clone())?, + ColumnFlags::empty(), + )); + } for batch in batches.iter() { - if cols.is_empty() { - let schema = batch.schema().clone(); - for (_i, field) in schema.fields().iter().enumerate() { - cols.push(Column::new( - field.name().clone(), - arrow_to_column_type(field.data_type().clone())?, - ColumnFlags::empty(), - )); - } - } if batch.num_rows() == 0 { continue; } diff --git a/rust/cubesql/cubesql/src/sql/mysql/service.rs b/rust/cubesql/cubesql/src/sql/mysql/service.rs index 9bd3f293ec6fa..13dc6a0bc64fc 100644 --- a/rust/cubesql/cubesql/src/sql/mysql/service.rs +++ b/rust/cubesql/cubesql/src/sql/mysql/service.rs @@ -231,7 +231,7 @@ impl MySqlConnection { &plan, ); let batches = df.collect().await?; - let response = batch_to_dataframe(&batches)?; + let response = batch_to_dataframe(&df.schema().into(), &batches)?; return Ok(QueryResponse::ResultSet(status, Box::new(response))) } diff --git a/rust/cubesql/cubesql/src/sql/postgres/extended.rs b/rust/cubesql/cubesql/src/sql/postgres/extended.rs index 2990a3066c968..e74b3c28bc58d 100644 --- a/rust/cubesql/cubesql/src/sql/postgres/extended.rs +++ b/rust/cubesql/cubesql/src/sql/postgres/extended.rs @@ -245,7 +245,7 @@ impl Portal { }; // TODO: Split doesn't split batches, it copy the part, lets dont convert whole batch to dataframe - let frame = batch_to_dataframe(&vec![batch_for_write])?; + let frame = batch_to_dataframe(batch_for_write.schema().as_ref(), &vec![batch_for_write])?; self.write_dataframe_to_writer(writer, frame, rows_to_read)?; Ok(unused)