mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 12:22:12 +00:00
Use settings from global context in StorageView
This commit is contained in:
parent
85bcf8a1dc
commit
80a001ab0c
@ -29,64 +29,12 @@ namespace ErrorCodes
|
|||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
namespace
|
|
||||||
{
|
|
||||||
|
|
||||||
void addSettingsChanges(ASTPtr ast, const Settings & settings)
|
|
||||||
{
|
|
||||||
auto * settings_ast = ast->as<ASTSetQuery>();
|
|
||||||
if (!settings_ast)
|
|
||||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "ASTSetQuery expected");
|
|
||||||
|
|
||||||
settings_ast->is_standalone = false;
|
|
||||||
if (settings_ast->changes.tryGet("join_use_nulls") == nullptr)
|
|
||||||
settings_ast->changes.emplace_back("join_use_nulls", Field(settings.join_use_nulls));
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Save to AST settings from context that affects view behaviour.
|
|
||||||
void saveSettingsToAst(ASTSelectWithUnionQuery * select, const Settings & settings)
|
|
||||||
{
|
|
||||||
/// Check SETTINGS section on the top level
|
|
||||||
if (select->settings_ast)
|
|
||||||
{
|
|
||||||
addSettingsChanges(select->settings_ast, settings);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// We cannot add SETTINGS on the top level because it will clash with section from inner SELECT
|
|
||||||
/// and will got query: SELECT ... SETTINGS ... SETTINGS ...
|
|
||||||
|
|
||||||
/// Process every select in ast and add SETTINGS section to each
|
|
||||||
for (const auto & child : select->list_of_selects->children)
|
|
||||||
{
|
|
||||||
auto * child_select = child->as<ASTSelectQuery>();
|
|
||||||
if (!child_select)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
ASTPtr ast_set_query = child_select->settings();
|
|
||||||
if (ast_set_query)
|
|
||||||
{
|
|
||||||
/// Modify existing SETTINGS section
|
|
||||||
addSettingsChanges(ast_set_query, settings);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// Add SETTINGS section to query
|
|
||||||
ast_set_query = std::make_shared<ASTSetQuery>();
|
|
||||||
addSettingsChanges(ast_set_query, settings);
|
|
||||||
child_select->setExpression(ASTSelectQuery::Expression::SETTINGS, std::move(ast_set_query));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
StorageView::StorageView(
|
StorageView::StorageView(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
const ASTCreateQuery & query,
|
const ASTCreateQuery & query,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const String & comment,
|
const String & comment)
|
||||||
const Settings & settings)
|
|
||||||
: IStorage(table_id_)
|
: IStorage(table_id_)
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata storage_metadata;
|
StorageInMemoryMetadata storage_metadata;
|
||||||
@ -95,8 +43,6 @@ StorageView::StorageView(
|
|||||||
|
|
||||||
if (!query.select)
|
if (!query.select)
|
||||||
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
||||||
|
|
||||||
saveSettingsToAst(query.select, settings);
|
|
||||||
SelectQueryDescription description;
|
SelectQueryDescription description;
|
||||||
|
|
||||||
description.inner_query = query.select->ptr();
|
description.inner_query = query.select->ptr();
|
||||||
@ -140,7 +86,12 @@ void StorageView::read(
|
|||||||
current_inner_query = query_info.view_query->clone();
|
current_inner_query = query_info.view_query->clone();
|
||||||
}
|
}
|
||||||
|
|
||||||
InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names);
|
auto modified_context = Context::createCopy(context);
|
||||||
|
/// Use settings from global context,
|
||||||
|
/// because difference between settings set on VIEW creation and query execution can break queries
|
||||||
|
modified_context->setSettings(context->getGlobalContext()->getSettingsRef());
|
||||||
|
|
||||||
|
InterpreterSelectWithUnionQuery interpreter(current_inner_query, modified_context, {}, column_names);
|
||||||
interpreter.buildQueryPlan(query_plan);
|
interpreter.buildQueryPlan(query_plan);
|
||||||
|
|
||||||
/// It's expected that the columns read from storage are not constant.
|
/// It's expected that the columns read from storage are not constant.
|
||||||
@ -228,7 +179,7 @@ void registerStorageView(StorageFactory & factory)
|
|||||||
if (args.query.storage)
|
if (args.query.storage)
|
||||||
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
||||||
|
|
||||||
return StorageView::create(args.table_id, args.query, args.columns, args.comment, args.getLocalContext()->getSettingsRef());
|
return StorageView::create(args.table_id, args.query, args.columns, args.comment);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -53,8 +53,7 @@ protected:
|
|||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
const ASTCreateQuery & query,
|
const ASTCreateQuery & query,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const String & comment,
|
const String & comment);
|
||||||
const Settings & settings);
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -42,7 +42,7 @@ StoragePtr TableFunctionView::executeImpl(
|
|||||||
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
||||||
{
|
{
|
||||||
auto columns = getActualTableStructure(context);
|
auto columns = getActualTableStructure(context);
|
||||||
auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns, String{}, context->getSettingsRef());
|
auto res = StorageView::create(StorageID(getDatabaseName(), table_name), create, columns, "");
|
||||||
res->startup();
|
res->startup();
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
@ -1 +1 @@
|
|||||||
CREATE VIEW default.test_view_00599\n(\n `id` UInt64\n) AS\nSELECT *\nFROM default.test_00599\nWHERE id = (\n SELECT 1\n)\nSETTINGS join_use_nulls = 0
|
CREATE VIEW default.test_view_00599\n(\n `id` UInt64\n) AS\nSELECT *\nFROM default.test_00599\nWHERE id = (\n SELECT 1\n)
|
||||||
|
@ -1,2 +1,2 @@
|
|||||||
CREATE VIEW default.t\n(\n `number` UInt64\n) AS\nSELECT number\nFROM system.numbers\nSETTINGS join_use_nulls = 0
|
CREATE VIEW default.t\n(\n `number` UInt64\n) AS\nSELECT number\nFROM system.numbers
|
||||||
CREATE VIEW default.t\n(\n `next_number` UInt64\n) AS\nSELECT number + 1 AS next_number\nFROM system.numbers\nSETTINGS join_use_nulls = 0
|
CREATE VIEW default.t\n(\n `next_number` UInt64\n) AS\nSELECT number + 1 AS next_number\nFROM system.numbers
|
||||||
|
@ -8,7 +8,6 @@ FROM
|
|||||||
SELECT *
|
SELECT *
|
||||||
FROM default.test
|
FROM default.test
|
||||||
HAVING id = 1
|
HAVING id = 1
|
||||||
SETTINGS join_use_nulls = 0
|
|
||||||
) AS test_view
|
) AS test_view
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
SELECT
|
SELECT
|
||||||
@ -21,7 +20,6 @@ FROM
|
|||||||
SELECT *
|
SELECT *
|
||||||
FROM default.test
|
FROM default.test
|
||||||
HAVING id = 2
|
HAVING id = 2
|
||||||
SETTINGS join_use_nulls = 0
|
|
||||||
) AS test_view
|
) AS test_view
|
||||||
WHERE id = 2
|
WHERE id = 2
|
||||||
SELECT id
|
SELECT id
|
||||||
@ -30,7 +28,6 @@ FROM
|
|||||||
SELECT *
|
SELECT *
|
||||||
FROM default.test
|
FROM default.test
|
||||||
HAVING id = 1
|
HAVING id = 1
|
||||||
SETTINGS join_use_nulls = 0
|
|
||||||
) AS test_view
|
) AS test_view
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
SELECT id
|
SELECT id
|
||||||
@ -39,6 +36,5 @@ FROM
|
|||||||
SELECT *
|
SELECT *
|
||||||
FROM default.test
|
FROM default.test
|
||||||
HAVING id = 1
|
HAVING id = 1
|
||||||
SETTINGS join_use_nulls = 0
|
|
||||||
) AS s
|
) AS s
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
CREATE VIEW test_1602.v\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl\nSETTINGS join_use_nulls = 0
|
CREATE VIEW test_1602.v\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||||
CREATE MATERIALIZED VIEW test_1602.vv\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n)\nENGINE = MergeTree\nPARTITION BY toYYYYMM(EventDate)\nORDER BY (CounterID, EventDate, intHash32(UserID))\nSETTINGS index_granularity = 8192 AS\nSELECT *\nFROM test_1602.tbl
|
CREATE MATERIALIZED VIEW test_1602.vv\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n)\nENGINE = MergeTree\nPARTITION BY toYYYYMM(EventDate)\nORDER BY (CounterID, EventDate, intHash32(UserID))\nSETTINGS index_granularity = 8192 AS\nSELECT *\nFROM test_1602.tbl
|
||||||
CREATE LIVE VIEW test_1602.vvv\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
CREATE LIVE VIEW test_1602.vvv\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||||
CREATE VIEW test_1602.VIEW\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl\nSETTINGS join_use_nulls = 0
|
CREATE VIEW test_1602.VIEW\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||||
CREATE VIEW test_1602.DATABASE\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl\nSETTINGS join_use_nulls = 0
|
CREATE VIEW test_1602.DATABASE\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||||
CREATE VIEW test_1602.DICTIONARY\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl\nSETTINGS join_use_nulls = 0
|
CREATE VIEW test_1602.DICTIONARY\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||||
CREATE VIEW test_1602.TABLE\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl\nSETTINGS join_use_nulls = 0
|
CREATE VIEW test_1602.TABLE\n(\n `EventDate` DateTime,\n `CounterID` UInt32,\n `UserID` UInt32\n) AS\nSELECT *\nFROM test_1602.tbl
|
||||||
|
@ -1,48 +1,34 @@
|
|||||||
SELECT
|
join_use_nulls = 1
|
||||||
a,
|
-
|
||||||
b,
|
|
||||||
c
|
|
||||||
FROM
|
|
||||||
(
|
|
||||||
SELECT *
|
|
||||||
FROM
|
|
||||||
(
|
|
||||||
SELECT
|
|
||||||
number + 1 AS a,
|
|
||||||
number + 11 AS b
|
|
||||||
FROM numbers(2)
|
|
||||||
) AS t1
|
|
||||||
FULL OUTER JOIN
|
|
||||||
(
|
|
||||||
SELECT
|
|
||||||
number + 2 AS a,
|
|
||||||
number + 22 AS c
|
|
||||||
FROM numbers(2)
|
|
||||||
) AS t2 USING (a)
|
|
||||||
ORDER BY a ASC
|
|
||||||
SETTINGS max_block_size = 666, join_use_nulls = 0
|
|
||||||
) AS view_no_nulls
|
|
||||||
1 11 0
|
1 11 0
|
||||||
2 12 22
|
2 12 22
|
||||||
3 0 23
|
3 0 23
|
||||||
|
-
|
||||||
1 11 0
|
1 11 0
|
||||||
2 12 22
|
2 12 22
|
||||||
3 0 23
|
3 0 23
|
||||||
|
-
|
||||||
1 11 \N
|
1 11 \N
|
||||||
2 12 22
|
2 12 22
|
||||||
3 \N 23
|
3 \N 23
|
||||||
1 11 \N
|
-
|
||||||
2 12 22
|
|
||||||
3 \N 23
|
|
||||||
1 11 0
|
1 11 0
|
||||||
2 12 22
|
2 12 22
|
||||||
3 0 23
|
3 0 23
|
||||||
|
join_use_nulls = 0
|
||||||
|
-
|
||||||
1 11 0
|
1 11 0
|
||||||
2 12 22
|
2 12 22
|
||||||
3 0 23
|
3 0 23
|
||||||
|
-
|
||||||
|
1 11 0
|
||||||
|
2 12 22
|
||||||
|
3 0 23
|
||||||
|
-
|
||||||
1 11 \N
|
1 11 \N
|
||||||
2 12 22
|
2 12 22
|
||||||
3 \N 23
|
3 \N 23
|
||||||
1 11 \N
|
-
|
||||||
|
1 11 0
|
||||||
2 12 22
|
2 12 22
|
||||||
3 \N 23
|
3 0 23
|
||||||
|
@ -8,17 +8,13 @@ SET join_use_nulls = 0;
|
|||||||
CREATE OR REPLACE VIEW view_no_nulls AS
|
CREATE OR REPLACE VIEW view_no_nulls AS
|
||||||
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
|
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
|
||||||
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
|
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
|
||||||
USING a ORDER BY a
|
USING a ORDER BY a;
|
||||||
SETTINGS max_block_size = 666;
|
|
||||||
|
|
||||||
-- check that max_block_size not rewriten
|
|
||||||
EXPLAIN SYNTAX SELECT * FROM view_no_nulls;
|
|
||||||
|
|
||||||
CREATE OR REPLACE VIEW view_nulls_set AS
|
CREATE OR REPLACE VIEW view_nulls_set AS
|
||||||
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
|
SELECT * FROM ( SELECT number + 1 AS a, number + 11 AS b FROM numbers(2) ) AS t1
|
||||||
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
|
FULL JOIN ( SELECT number + 2 AS a, number + 22 AS c FROM numbers(2) ) AS t2
|
||||||
USING a ORDER BY a
|
USING a ORDER BY a
|
||||||
SETTINGS join_use_nulls = 1, max_block_size = 666;
|
SETTINGS join_use_nulls = 1;
|
||||||
|
|
||||||
SET join_use_nulls = 1;
|
SET join_use_nulls = 1;
|
||||||
|
|
||||||
@ -35,17 +31,29 @@ SETTINGS join_use_nulls = 0;
|
|||||||
|
|
||||||
SET join_use_nulls = 1;
|
SET join_use_nulls = 1;
|
||||||
|
|
||||||
SELECT * from view_no_nulls;
|
SELECT 'join_use_nulls = 1';
|
||||||
SELECT * from view_no_nulls_set;
|
|
||||||
SELECT * from view_nulls_set;
|
SELECT '-';
|
||||||
SELECT * from view_nulls;
|
SELECT * FROM view_no_nulls;
|
||||||
|
SELECT '-';
|
||||||
|
SELECT * FROM view_no_nulls_set;
|
||||||
|
SELECT '-';
|
||||||
|
SELECT * FROM view_nulls_set;
|
||||||
|
SELECT '-';
|
||||||
|
SELECT * FROM view_nulls;
|
||||||
|
|
||||||
SET join_use_nulls = 0;
|
SET join_use_nulls = 0;
|
||||||
|
|
||||||
SELECT * from view_no_nulls;
|
SELECT 'join_use_nulls = 0';
|
||||||
SELECT * from view_no_nulls_set;
|
|
||||||
SELECT * from view_nulls_set;
|
SELECT '-';
|
||||||
SELECT * from view_nulls;
|
SELECT * FROM view_no_nulls;
|
||||||
|
SELECT '-';
|
||||||
|
SELECT * FROM view_no_nulls_set;
|
||||||
|
SELECT '-';
|
||||||
|
SELECT * FROM view_nulls_set;
|
||||||
|
SELECT '-';
|
||||||
|
SELECT * FROM view_nulls;
|
||||||
|
|
||||||
DROP TABLE IF EXISTS view_no_nulls;
|
DROP TABLE IF EXISTS view_no_nulls;
|
||||||
DROP TABLE IF EXISTS view_no_nulls_set;
|
DROP TABLE IF EXISTS view_no_nulls_set;
|
||||||
|
Loading…
Reference in New Issue
Block a user