Use settings from global context in StorageView

This commit is contained in:
vdimir 2021-06-22 17:32:02 +03:00
parent 85bcf8a1dc
commit 80a001ab0c
No known key found for this signature in database
GPG Key ID: F57B3E10A21DBB31
9 changed files with 55 additions and 115 deletions

View File

@ -29,64 +29,12 @@ namespace ErrorCodes
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(
const StorageID & table_id_,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
const String & comment,
const Settings & settings)
const String & comment)
: IStorage(table_id_)
{
StorageInMemoryMetadata storage_metadata;
@ -95,8 +43,6 @@ StorageView::StorageView(
if (!query.select)
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
saveSettingsToAst(query.select, settings);
SelectQueryDescription description;
description.inner_query = query.select->ptr();
@ -140,7 +86,12 @@ void StorageView::read(
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);
/// It's expected that the columns read from storage are not constant.
@ -228,7 +179,7 @@ void registerStorageView(StorageFactory & factory)
if (args.query.storage)
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);
});
}

View File

@ -53,8 +53,7 @@ protected:
const StorageID & table_id_,
const ASTCreateQuery & query,
const ColumnsDescription & columns_,
const String & comment,
const Settings & settings);
const String & comment);
};
}

View File

@ -42,7 +42,7 @@ StoragePtr TableFunctionView::executeImpl(
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
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();
return res;
}

View File

@ -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)

View File

@ -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 `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 `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

View File

@ -8,7 +8,6 @@ FROM
SELECT *
FROM default.test
HAVING id = 1
SETTINGS join_use_nulls = 0
) AS test_view
WHERE id = 1
SELECT
@ -21,7 +20,6 @@ FROM
SELECT *
FROM default.test
HAVING id = 2
SETTINGS join_use_nulls = 0
) AS test_view
WHERE id = 2
SELECT id
@ -30,7 +28,6 @@ FROM
SELECT *
FROM default.test
HAVING id = 1
SETTINGS join_use_nulls = 0
) AS test_view
WHERE id = 1
SELECT id
@ -39,6 +36,5 @@ FROM
SELECT *
FROM default.test
HAVING id = 1
SETTINGS join_use_nulls = 0
) AS s
WHERE id = 1

View File

@ -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 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.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.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.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.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
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

View File

@ -1,48 +1,34 @@
SELECT
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
join_use_nulls = 1
-
1 11 0
2 12 22
3 0 23
-
1 11 0
2 12 22
3 0 23
-
1 11 \N
2 12 22
3 \N 23
1 11 \N
2 12 22
3 \N 23
-
1 11 0
2 12 22
3 0 23
join_use_nulls = 0
-
1 11 0
2 12 22
3 0 23
-
1 11 0
2 12 22
3 0 23
-
1 11 \N
2 12 22
3 \N 23
1 11 \N
-
1 11 0
2 12 22
3 \N 23
3 0 23

View File

@ -8,17 +8,13 @@ SET join_use_nulls = 0;
CREATE OR REPLACE VIEW view_no_nulls AS
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
USING a ORDER BY a
SETTINGS max_block_size = 666;
-- check that max_block_size not rewriten
EXPLAIN SYNTAX SELECT * FROM view_no_nulls;
USING a ORDER BY a;
CREATE OR REPLACE VIEW view_nulls_set AS
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
USING a ORDER BY a
SETTINGS join_use_nulls = 1, max_block_size = 666;
SETTINGS join_use_nulls = 1;
SET join_use_nulls = 1;
@ -35,17 +31,29 @@ SETTINGS join_use_nulls = 0;
SET join_use_nulls = 1;
SELECT * from view_no_nulls;
SELECT * from view_no_nulls_set;
SELECT * from view_nulls_set;
SELECT * from view_nulls;
SELECT 'join_use_nulls = 1';
SELECT '-';
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;
SELECT * from view_no_nulls;
SELECT * from view_no_nulls_set;
SELECT * from view_nulls_set;
SELECT * from view_nulls;
SELECT 'join_use_nulls = 0';
SELECT '-';
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_set;