Merge pull request #47725 from ClickHouse/46741_Parameterized_view_support_subquery

Support for subquery in parameterized views
This commit is contained in:
SmitaRKulkarni 2023-03-22 09:05:26 +01:00 committed by GitHub
commit 464b166e91
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 41 additions and 30 deletions

View File

@ -415,7 +415,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
RewriteCountDistinctFunctionVisitor(data_rewrite_countdistinct).visit(query_ptr);
}
JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols);
JoinedTables joined_tables(getSubqueryContext(context), getSelectQuery(), options.with_all_cols, options_.is_create_parameterized_view);
bool got_storage_from_query = false;
if (!has_input && !storage)

View File

@ -262,12 +262,12 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_,
{
if (!context_->hasQueryContext())
{
SelectQueryOptions options;
if (is_subquery)
return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock();
else if (is_create_parameterized_view)
return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().createParameterizedView().analyze()).getSampleBlock();
else
return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();
options = options.subquery();
if (is_create_parameterized_view)
options = options.createParameterizedView();
return InterpreterSelectWithUnionQuery(query_ptr_, context_, std::move(options.analyze())).getSampleBlock();
}
auto & cache = context_->getSampleBlockCache();
@ -278,21 +278,12 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_,
return cache[key];
}
SelectQueryOptions options;
if (is_subquery)
{
return cache[key]
= InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock();
}
else if (is_create_parameterized_view)
{
return cache[key]
= InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().createParameterizedView().analyze())
.getSampleBlock();
}
else
{
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock();
}
options = options.subquery();
if (is_create_parameterized_view)
options = options.createParameterizedView();
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr_, context_, std::move(options.analyze())).getSampleBlock();
}

View File

@ -173,13 +173,14 @@ using RenameQualifiedIdentifiersVisitor = InDepthNodeVisitor<RenameQualifiedIden
}
JoinedTables::JoinedTables(ContextPtr context_, const ASTSelectQuery & select_query_, bool include_all_columns_)
JoinedTables::JoinedTables(ContextPtr context_, const ASTSelectQuery & select_query_, bool include_all_columns_, bool is_create_parameterized_view_)
: context(context_)
, table_expressions(getTableExpressions(select_query_))
, include_all_columns(include_all_columns_)
, left_table_expression(extractTableExpression(select_query_, 0))
, left_db_and_table(getDatabaseAndTable(select_query_, 0))
, select_query(select_query_)
, is_create_parameterized_view(is_create_parameterized_view_)
{}
bool JoinedTables::isLeftTableSubquery() const
@ -239,7 +240,7 @@ bool JoinedTables::resolveTables()
const auto & settings = context->getSettingsRef();
bool include_alias_cols = include_all_columns || settings.asterisk_include_alias_columns;
bool include_materialized_cols = include_all_columns || settings.asterisk_include_materialized_columns;
tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context, include_alias_cols, include_materialized_cols);
tables_with_columns = getDatabaseAndTablesWithColumns(table_expressions, context, include_alias_cols, include_materialized_cols, is_create_parameterized_view);
if (tables_with_columns.size() != table_expressions.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected tables count");

View File

@ -22,7 +22,7 @@ using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
class JoinedTables
{
public:
JoinedTables(ContextPtr context, const ASTSelectQuery & select_query_, bool include_all_columns_ = false);
JoinedTables(ContextPtr context, const ASTSelectQuery & select_query_, bool include_all_columns_ = false, bool is_create_parameterized_view_ = false);
void reset(const ASTSelectQuery & select_query);
@ -53,6 +53,7 @@ private:
ASTPtr left_table_expression;
std::optional<DatabaseAndTableWithAlias> left_db_and_table;
const ASTSelectQuery & select_query;
const bool is_create_parameterized_view;
};
}

View File

@ -73,18 +73,21 @@ ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number
return nullptr;
}
/// The parameter is_create_parameterized_view is used in getSampleBlock of the subquery.
/// If it is set to true, then query parameters are allowed in the subquery, and that expression is not evaluated.
static NamesAndTypesList getColumnsFromTableExpression(
const ASTTableExpression & table_expression,
ContextPtr context,
NamesAndTypesList & materialized,
NamesAndTypesList & aliases,
NamesAndTypesList & virtuals)
NamesAndTypesList & virtuals,
bool is_create_parameterized_view)
{
NamesAndTypesList names_and_type_list;
if (table_expression.subquery)
{
const auto & subquery = table_expression.subquery->children.at(0);
names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context, true).getNamesAndTypesList();
names_and_type_list = InterpreterSelectWithUnionQuery::getSampleBlock(subquery, context, true, is_create_parameterized_view).getNamesAndTypesList();
}
else if (table_expression.table_function)
{
@ -117,7 +120,8 @@ TablesWithColumns getDatabaseAndTablesWithColumns(
const ASTTableExprConstPtrs & table_expressions,
ContextPtr context,
bool include_alias_cols,
bool include_materialized_cols)
bool include_materialized_cols,
bool is_create_parameterized_view)
{
TablesWithColumns tables_with_columns;
@ -129,7 +133,7 @@ TablesWithColumns getDatabaseAndTablesWithColumns(
NamesAndTypesList aliases;
NamesAndTypesList virtuals;
NamesAndTypesList names_and_types = getColumnsFromTableExpression(
*table_expression, context, materialized, aliases, virtuals);
*table_expression, context, materialized, aliases, virtuals, is_create_parameterized_view);
removeDuplicateColumns(names_and_types);

View File

@ -20,7 +20,9 @@ const ASTTableExpression * getTableExpression(const ASTSelectQuery & select, siz
ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number);
/// The parameter is_create_parameterized_view is used in getSampleBlock of the subquery. It is forwarded to getColumnsFromTableExpression.
/// If it is set to true, then query parameters are allowed in the subquery, and that expression is not evaluated.
TablesWithColumns getDatabaseAndTablesWithColumns(
const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols);
const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols, bool is_create_parameterized_view = false);
}

View File

@ -36,3 +36,5 @@ ERROR
20
10
10
10
10

View File

@ -16,13 +16,15 @@ $CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv7"
$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv8"
$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv9"
$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv10"
$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv11"
$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_pv12"
$CLICKHOUSE_CLIENT -q "DROP VIEW IF EXISTS test_02428_v1"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02428_Catalog"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}.Catalog"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS ${CLICKHOUSE_TEST_UNIQUE_NAME}"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02428_Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = Memory"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02428_Catalog (Name String, Price UInt64, Quantity UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parameterized_view', 'r1') ORDER BY Name"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Pen', 10, 3)"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book', 50, 2)"
@ -71,7 +73,7 @@ $CLICKHOUSE_CLIENT -q "SELECT Price FROM \`${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1\`(
$CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book2', 30, 8)"
$CLICKHOUSE_CLIENT -q "INSERT INTO test_02428_Catalog VALUES ('Book3', 30, 8)"
$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv5 AS SELECT Price FROM test_02428_Catalog WHERE {price:UInt64} HAVING Quantity in (SELECT {quantity:UInt64}) LIMIT {limit:UInt64}"
$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv5 AS SELECT Price FROM test_02428_Catalog WHERE Price={price:UInt64} HAVING Quantity in (SELECT {quantity:UInt64}) LIMIT {limit:UInt64}"
$CLICKHOUSE_CLIENT -q "SELECT Price FROM test_02428_pv5(price=30, quantity=8, limit=1)"
$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv6 AS SELECT Price+{price:UInt64} FROM test_02428_Catalog GROUP BY Price+{price:UInt64} ORDER BY Price+{price:UInt64}"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv6(price=10)"
@ -87,6 +89,12 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv9(quantity=3)"
$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv10 AS SELECT Price FROM test_02428_Catalog WHERE Price={Pri:UInt64} ORDER BY Price"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv10(Pri=10)"
$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv11 AS SELECT * from ( SELECT Price FROM test_02428_Catalog WHERE Price={price:UInt64} )"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv11(price=10)"
$CLICKHOUSE_CLIENT -q "CREATE VIEW test_02428_pv12 AS SELECT * from ( SELECT Price FROM test_02428_Catalog WHERE Price IN (SELECT number FROM numbers({price:UInt64})) )"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02428_pv12(price=11)"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv1"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv2"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv3"
@ -96,6 +104,8 @@ $CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv7"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv8"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv9"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv10"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv11"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_pv12"
$CLICKHOUSE_CLIENT -q "DROP VIEW test_02428_v1"
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02428_Catalog"
$CLICKHOUSE_CLIENT -q "DROP TABLE ${CLICKHOUSE_TEST_UNIQUE_NAME}.pv1"