mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Merge pull request #2881 from zhang2014/feature/support_optimize_predicate_for_view
ISSUES-863 support optimize predicate for view
This commit is contained in:
commit
ef22af2e49
@ -1,11 +1,14 @@
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
|
||||
#include <Storages/StorageView.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
|
||||
#include <DataStreams/MaterializingBlockInputStream.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -13,6 +16,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -31,14 +35,26 @@ StorageView::StorageView(
|
||||
|
||||
BlockInputStreams StorageView::read(
|
||||
const Names & column_names,
|
||||
const SelectQueryInfo & /*query_info*/,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
const size_t /*max_block_size*/,
|
||||
const unsigned /*num_streams*/)
|
||||
{
|
||||
checkQueryProcessingStage(processed_stage, context);
|
||||
BlockInputStreams res = InterpreterSelectWithUnionQuery(inner_query, context, column_names).executeWithMultipleStreams();
|
||||
|
||||
BlockInputStreams res;
|
||||
|
||||
if (context.getSettings().enable_optimize_predicate_expression)
|
||||
{
|
||||
replaceTableNameWithSubquery(typeid_cast<ASTSelectQuery *>(query_info.query.get()), inner_query);
|
||||
auto res_io = InterpreterSelectQuery(query_info.query, context, column_names, processed_stage).execute();
|
||||
|
||||
res.emplace_back(res_io.in);
|
||||
return res;
|
||||
}
|
||||
|
||||
res = InterpreterSelectWithUnionQuery(inner_query, context, column_names).executeWithMultipleStreams();
|
||||
|
||||
/// It's expected that the columns read from storage are not constant.
|
||||
/// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery.
|
||||
@ -48,6 +64,23 @@ BlockInputStreams StorageView::read(
|
||||
return res;
|
||||
}
|
||||
|
||||
void StorageView::replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery)
|
||||
{
|
||||
ASTTablesInSelectQueryElement * select_element = static_cast<ASTTablesInSelectQueryElement *>(select_query->tables->children[0].get());
|
||||
|
||||
if (!select_element->table_expression)
|
||||
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ASTTableExpression * table_expression = static_cast<ASTTableExpression *>(select_element->table_expression.get());
|
||||
|
||||
if (!table_expression->database_and_table_name)
|
||||
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
table_expression->database_and_table_name = {};
|
||||
table_expression->subquery = std::make_shared<ASTSubquery>();
|
||||
table_expression->subquery->children.push_back(subquery->clone());
|
||||
}
|
||||
|
||||
|
||||
void registerStorageView(StorageFactory & factory)
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -38,6 +39,8 @@ private:
|
||||
String table_name;
|
||||
ASTPtr inner_query;
|
||||
|
||||
void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery);
|
||||
|
||||
protected:
|
||||
StorageView(
|
||||
const String & table_name_,
|
||||
|
@ -20,5 +20,6 @@
|
||||
2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1
|
||||
2000-01-01 1 test string 1 1
|
||||
1 2000-01-01 2000-01-01 1 test string 1 1
|
||||
2000-01-01 1 test string 1 1
|
||||
-------Push to having expression, need check.-------
|
||||
-------Compatibility test-------
|
||||
|
@ -1,8 +1,10 @@
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
DROP TABLE IF EXISTS test.test;
|
||||
DROP TABLE IF EXISTS test.test_view;
|
||||
|
||||
CREATE TABLE test.test(date Date, id Int8, name String, value Int64) ENGINE = MergeTree(date, (id, date), 8192);
|
||||
CREATE VIEW test.test_view AS SELECT * FROM test.test;
|
||||
|
||||
INSERT INTO test.test VALUES('2000-01-01', 1, 'test string 1', 1);
|
||||
INSERT INTO test.test VALUES('2000-01-01', 2, 'test string 2', 2);
|
||||
@ -50,6 +52,9 @@ SELECT * FROM (SELECT * FROM (SELECT * FROM test.test) ANY LEFT JOIN (SELECT * F
|
||||
-- Optimize predicate expression with join query and qualified
|
||||
SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test) AS b USING date WHERE b.id = 1;
|
||||
|
||||
-- Optimize predicate expression with view
|
||||
SELECT * FROM test.test_view WHERE id = 1;
|
||||
|
||||
SELECT '-------Push to having expression, need check.-------';
|
||||
SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1; -- { serverError 277 }
|
||||
|
||||
@ -57,3 +62,4 @@ SELECT '-------Compatibility test-------';
|
||||
SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test) AS b USING date WHERE b.date = toDate('2000-01-01'); -- {serverError 47}
|
||||
|
||||
DROP TABLE IF EXISTS test.test;
|
||||
DROP TABLE IF EXISTS test.test_view;
|
||||
|
Loading…
Reference in New Issue
Block a user