mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #47725 from ClickHouse/46741_Parameterized_view_support_subquery
Support for subquery in parameterized views
This commit is contained in:
commit
464b166e91
@ -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)
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -36,3 +36,5 @@ ERROR
|
||||
20
|
||||
10
|
||||
10
|
||||
10
|
||||
10
|
||||
|
@ -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"
|
||||
|
Loading…
Reference in New Issue
Block a user