Merge pull request #44547 from ClickHouse/fix_44496

Fix too aggressive evaluation of args in default column expr
This commit is contained in:
Alexander Tokmakov 2023-01-16 15:08:58 +03:00 committed by GitHub
commit ee888f7f38
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 70 additions and 10 deletions

View File

@ -2,6 +2,7 @@
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
#include <Interpreters/misc.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/getClusterName.h>
@ -175,7 +176,7 @@ namespace
/// Finds dependencies of a function.
void visitFunction(const ASTFunction & function)
{
if (function.name == "joinGet" || function.name == "dictHas" || function.name == "dictIsIn" || function.name.starts_with("dictGet"))
if (functionIsJoinGet(function.name) || functionIsDictGet(function.name))
{
/// dictGet('dict_name', attr_names, id_expr)
/// dictHas('dict_name', id_expr)

View File

@ -1,6 +1,7 @@
#include <Databases/DDLLoadingDependencyVisitor.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Interpreters/Context.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -52,23 +53,41 @@ bool DDLMatcherBase::needChildVisit(const ASTPtr & node, const ASTPtr & child)
return true;
}
ssize_t DDLMatcherBase::getPositionOfTableNameArgument(const ASTFunction & function)
ssize_t DDLMatcherBase::getPositionOfTableNameArgumentToEvaluate(const ASTFunction & function)
{
if (function.name == "joinGet" ||
function.name == "dictHas" ||
function.name == "dictIsIn" ||
function.name.starts_with("dictGet"))
if (functionIsJoinGet(function.name) || functionIsDictGet(function.name))
return 0;
if (Poco::toLower(function.name) == "in")
return -1;
}
ssize_t DDLMatcherBase::getPositionOfTableNameArgumentToVisit(const ASTFunction & function)
{
ssize_t maybe_res = getPositionOfTableNameArgumentToEvaluate(function);
if (0 <= maybe_res)
return maybe_res;
if (functionIsInOrGlobalInOperator(function.name))
{
if (function.children.empty())
return -1;
const auto * args = function.children[0]->as<ASTExpressionList>();
if (!args || args->children.size() != 2)
return -1;
if (args->children[1]->as<ASTFunction>())
return -1;
return 1;
}
return -1;
}
void DDLLoadingDependencyVisitor::visit(const ASTFunction & function, Data & data)
{
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
ssize_t table_name_arg_idx = getPositionOfTableNameArgumentToVisit(function);
if (table_name_arg_idx < 0)
return;
extractTableNameFromArgument(function, data, table_name_arg_idx);

View File

@ -23,7 +23,8 @@ class DDLMatcherBase
{
public:
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
static ssize_t getPositionOfTableNameArgument(const ASTFunction & function);
static ssize_t getPositionOfTableNameArgumentToVisit(const ASTFunction & function);
static ssize_t getPositionOfTableNameArgumentToEvaluate(const ASTFunction & function);
};
/// Visits ASTCreateQuery and extracts the names of all tables which should be loaded before a specified table.

View File

@ -23,7 +23,7 @@ void NormalizeAndEvaluateConstants::visit(const ASTFunction & function, Data & d
{
/// Replace expressions like "dictGet(currentDatabase() || '.dict', 'value', toUInt32(1))"
/// with "dictGet('db_name.dict', 'value', toUInt32(1))"
ssize_t table_name_arg_idx = getPositionOfTableNameArgument(function);
ssize_t table_name_arg_idx = getPositionOfTableNameArgumentToEvaluate(function);
if (table_name_arg_idx < 0)
return;

View File

@ -0,0 +1,2 @@
CREATE TABLE default.dep\n(\n `id` Int32,\n `country` LowCardinality(String),\n `purchase_location` UInt16 MATERIALIZED if(id IN joinGet(\'default.id_join\', \'location\', \'CLICK\'), 123, 456)\n)\nENGINE = ReplicatedMergeTree(\'/test/02433/default/dep\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.dep2\n(\n `id` Int32,\n `country` LowCardinality(String),\n `purchase_location` UInt16 MATERIALIZED if(id IN joinGet(\'default.id_join\', \'location\', \'CLICK\'), 123, 456)\n)\nENGINE = ReplicatedMergeTree(\'/test/02433/default/dep\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,37 @@
DROP TABLE IF EXISTS dep;
DROP TABLE IF EXISTS dep2;
DROP TABLE IF EXISTS id_join;
CREATE TABLE id_join (`country` String, `location` Array(Int32)) ENGINE = Join(ANY, LEFT, country);
INSERT INTO id_join values ('CLICK', [1234]);
CREATE TABLE dep
(
`id` Int32,
`country` LowCardinality(String),
`purchase_location` UInt16 MATERIALIZED if(id IN joinGet(concat(currentDatabase(), '.id_join'), 'location', 'CLICK'), 123, 456)
)
ENGINE = ReplicatedMergeTree('/test/02433/{database}/dep', '1') ORDER BY tuple();
SHOW CREATE TABLE dep;
TRUNCATE TABLE id_join;
CREATE TABLE dep2
(
`id` Int32,
`country` LowCardinality(String),
`purchase_location` UInt16 MATERIALIZED if(id IN joinGet(concat(currentDatabase(), '.id_join'), 'location', 'CLICK'), 123, 456)
)
ENGINE = ReplicatedMergeTree('/test/02433/{database}/dep', '2') ORDER BY tuple();
SHOW CREATE TABLE dep2;
-- Ensure that a table name cannot be passed to IN as string literal
create table test (n int, m default n in 'default.table_name') engine=Memory; -- { serverError TYPE_MISMATCH }
create table test (n int, m default in(n, 'default.table_name')) engine=Memory; -- { serverError TYPE_MISMATCH }
DROP TABLE dep;
DROP TABLE dep2;
DROP TABLE id_join;