mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #29650 from kitaisreal/add-default-database-visitor-support-dict-get
AddDefaultDatabaseVisitor support dictGet
This commit is contained in:
commit
27fdeef4b6
@ -40,6 +40,22 @@ struct QualifiedTableName
|
||||
return hash_state.get64();
|
||||
}
|
||||
|
||||
std::vector<std::string> getParts() const
|
||||
{
|
||||
if (database.empty())
|
||||
return {table};
|
||||
else
|
||||
return {database, table};
|
||||
}
|
||||
|
||||
std::string getFullName() const
|
||||
{
|
||||
if (database.empty())
|
||||
return table;
|
||||
else
|
||||
return database + '.' + table;
|
||||
}
|
||||
|
||||
/// NOTE: It's different from compound identifier parsing and does not support escaping and dots in name.
|
||||
/// Usually it's better to use ParserIdentifier instead,
|
||||
/// but we parse DDL dictionary name (and similar things) this way for historical reasons.
|
||||
|
@ -10,10 +10,14 @@
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/misc.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -26,11 +30,12 @@ class AddDefaultDatabaseVisitor
|
||||
{
|
||||
public:
|
||||
explicit AddDefaultDatabaseVisitor(
|
||||
const String & database_name_, bool only_replace_current_database_function_ = false, WriteBuffer * ostr_ = nullptr)
|
||||
: database_name(database_name_)
|
||||
ContextPtr context_,
|
||||
const String & database_name_,
|
||||
bool only_replace_current_database_function_ = false)
|
||||
: context(context_)
|
||||
, database_name(database_name_)
|
||||
, only_replace_current_database_function(only_replace_current_database_function_)
|
||||
, visit_depth(0)
|
||||
, ostr(ostr_)
|
||||
{}
|
||||
|
||||
void visitDDL(ASTPtr & ast) const
|
||||
@ -64,11 +69,19 @@ public:
|
||||
visit(select, unused);
|
||||
}
|
||||
|
||||
void visit(ASTColumns & columns) const
|
||||
{
|
||||
for (auto & child : columns.children)
|
||||
visit(child);
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
ContextPtr context;
|
||||
|
||||
const String database_name;
|
||||
|
||||
bool only_replace_current_database_function = false;
|
||||
mutable size_t visit_depth;
|
||||
WriteBuffer * ostr;
|
||||
|
||||
void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const
|
||||
{
|
||||
@ -117,15 +130,8 @@ private:
|
||||
|
||||
void visit(ASTFunction & function, ASTPtr &) const
|
||||
{
|
||||
bool is_operator_in = false;
|
||||
for (const auto * name : {"in", "notIn", "globalIn", "globalNotIn"})
|
||||
{
|
||||
if (function.name == name)
|
||||
{
|
||||
is_operator_in = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
bool is_operator_in = functionIsInOrGlobalInOperator(function.name);
|
||||
bool is_dict_get = functionIsDictGet(function.name);
|
||||
|
||||
for (auto & child : function.children)
|
||||
{
|
||||
@ -133,7 +139,30 @@ private:
|
||||
{
|
||||
for (size_t i = 0; i < child->children.size(); ++i)
|
||||
{
|
||||
if (is_operator_in && i == 1)
|
||||
if (is_dict_get && i == 0)
|
||||
{
|
||||
if (auto * identifier = child->children[i]->as<ASTIdentifier>())
|
||||
{
|
||||
/// Identifier already qualified
|
||||
if (identifier->compound())
|
||||
continue;
|
||||
|
||||
auto qualified_dictionary_name = context->getExternalDictionariesLoader().qualifyDictionaryNameWithDatabase(identifier->name(), context);
|
||||
child->children[i] = std::make_shared<ASTIdentifier>(qualified_dictionary_name.getParts());
|
||||
}
|
||||
else if (auto * literal = child->children[i]->as<ASTLiteral>())
|
||||
{
|
||||
auto & literal_value = literal->value;
|
||||
|
||||
if (literal_value.getType() != Field::Types::String)
|
||||
continue;
|
||||
|
||||
auto dictionary_name = literal_value.get<String>();
|
||||
auto qualified_dictionary_name = context->getExternalDictionariesLoader().qualifyDictionaryNameWithDatabase(dictionary_name, context);
|
||||
literal_value = qualified_dictionary_name.getFullName();
|
||||
}
|
||||
}
|
||||
else if (is_operator_in && i == 1)
|
||||
{
|
||||
/// XXX: for some unknown reason this place assumes that argument can't be an alias,
|
||||
/// like in the similar code in `MarkTableIdentifierVisitor`.
|
||||
@ -151,11 +180,15 @@ private:
|
||||
visit(child->children[i]);
|
||||
}
|
||||
else
|
||||
{
|
||||
visit(child->children[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
visit(child);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -170,7 +203,6 @@ private:
|
||||
{
|
||||
if (T * t = typeid_cast<T *>(ast.get()))
|
||||
{
|
||||
DumpASTNode dump(*ast, ostr, visit_depth, "addDefaultDatabaseName");
|
||||
visit(*t, ast);
|
||||
return true;
|
||||
}
|
||||
|
@ -87,6 +87,28 @@ DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std
|
||||
return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config);
|
||||
}
|
||||
|
||||
QualifiedTableName ExternalDictionariesLoader::qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr query_context) const
|
||||
{
|
||||
auto qualified_name = QualifiedTableName::tryParseFromString(dictionary_name);
|
||||
if (!qualified_name)
|
||||
{
|
||||
QualifiedTableName qualified_dictionary_name;
|
||||
qualified_dictionary_name.table = dictionary_name;
|
||||
return qualified_dictionary_name;
|
||||
}
|
||||
|
||||
if (qualified_name->database.empty() && has(dictionary_name))
|
||||
{
|
||||
/// This is xml dictionary
|
||||
return *qualified_name;
|
||||
}
|
||||
|
||||
if (qualified_name->database.empty())
|
||||
qualified_name->database = query_context->getCurrentDatabase();
|
||||
|
||||
return *qualified_name;
|
||||
}
|
||||
|
||||
std::string ExternalDictionariesLoader::resolveDictionaryName(const std::string & dictionary_name, const std::string & current_database_name) const
|
||||
{
|
||||
if (has(dictionary_name))
|
||||
|
@ -27,6 +27,8 @@ public:
|
||||
|
||||
void reloadDictionary(const std::string & dictionary_name, ContextPtr context) const;
|
||||
|
||||
QualifiedTableName qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr context) const;
|
||||
|
||||
DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const;
|
||||
|
||||
static DictionaryStructure getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config = "dictionary");
|
||||
|
@ -80,7 +80,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
|
||||
/// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc.
|
||||
AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName());
|
||||
AddDefaultDatabaseVisitor visitor(getContext(), table_id.getDatabaseName());
|
||||
ASTPtr command_list_ptr = alter.command_list->ptr();
|
||||
visitor.visit(command_list_ptr);
|
||||
|
||||
|
@ -920,10 +920,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
{
|
||||
// Expand CTE before filling default database
|
||||
ApplyWithSubqueryVisitor().visit(*create.select);
|
||||
AddDefaultDatabaseVisitor visitor(current_database);
|
||||
AddDefaultDatabaseVisitor visitor(getContext(), current_database);
|
||||
visitor.visit(*create.select);
|
||||
}
|
||||
|
||||
if (create.columns_list)
|
||||
{
|
||||
AddDefaultDatabaseVisitor visitor(getContext(), current_database);
|
||||
visitor.visit(*create.columns_list);
|
||||
}
|
||||
|
||||
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
|
||||
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create);
|
||||
|
||||
|
@ -152,7 +152,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
|
||||
}
|
||||
}
|
||||
|
||||
AddDefaultDatabaseVisitor visitor(current_database, !use_local_default_database);
|
||||
AddDefaultDatabaseVisitor visitor(context, current_database, !use_local_default_database);
|
||||
visitor.visitDDL(query_ptr);
|
||||
|
||||
/// Check access rights, assume that all servers have the same users config
|
||||
|
@ -70,7 +70,7 @@ static StorageID extractDependentTable(ASTPtr & query, ContextPtr context, const
|
||||
if (db_and_table->database.empty())
|
||||
{
|
||||
db_and_table->database = select_database_name;
|
||||
AddDefaultDatabaseVisitor visitor(select_database_name);
|
||||
AddDefaultDatabaseVisitor visitor(context, select_database_name);
|
||||
visitor.visit(select_query);
|
||||
}
|
||||
else
|
||||
|
@ -48,7 +48,7 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, ContextPt
|
||||
{
|
||||
if (add_default_db)
|
||||
{
|
||||
AddDefaultDatabaseVisitor visitor(context->getCurrentDatabase(), false, nullptr);
|
||||
AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase());
|
||||
visitor.visit(query);
|
||||
}
|
||||
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||
#include <Access/AccessFlags.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
|
@ -0,0 +1 @@
|
||||
CREATE TABLE `02097_db`.test_table_default (`data_1` UInt64 DEFAULT dictGetUInt64(\'02097_db.test_dictionary\', \'data_column_1\', toUInt64(0)), `data_2` UInt8 DEFAULT dictGet(`02097_db`.test_dictionary, \'data_column_2\', toUInt64(0))) ENGINE = TinyLog
|
@ -0,0 +1,44 @@
|
||||
-- Tags: no-parallel
|
||||
|
||||
DROP DATABASE IF EXISTS 02097_db;
|
||||
CREATE DATABASE 02097_db;
|
||||
|
||||
USE 02097_db;
|
||||
|
||||
DROP TABLE IF EXISTS test_table;
|
||||
CREATE TABLE test_table
|
||||
(
|
||||
key_column UInt64,
|
||||
data_column_1 UInt64,
|
||||
data_column_2 UInt8
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY key_column;
|
||||
|
||||
DROP DICTIONARY IF EXISTS test_dictionary;
|
||||
CREATE DICTIONARY test_dictionary
|
||||
(
|
||||
key_column UInt64 DEFAULT 0,
|
||||
data_column_1 UInt64 DEFAULT 1,
|
||||
data_column_2 UInt8 DEFAULT 1
|
||||
)
|
||||
PRIMARY KEY key_column
|
||||
LAYOUT(DIRECT())
|
||||
SOURCE(CLICKHOUSE(TABLE 'test_table'));
|
||||
|
||||
DROP TABLE IF EXISTS test_table_default;
|
||||
CREATE TABLE test_table_default
|
||||
(
|
||||
data_1 DEFAULT dictGetUInt64('test_dictionary', 'data_column_1', toUInt64(0)),
|
||||
data_2 DEFAULT dictGet(test_dictionary, 'data_column_2', toUInt64(0))
|
||||
)
|
||||
ENGINE=TinyLog;
|
||||
|
||||
SELECT create_table_query FROM system.tables WHERE name = 'test_table_default' AND database = '02097_db';
|
||||
|
||||
DROP DICTIONARY test_dictionary;
|
||||
DROP TABLE test_table;
|
||||
DROP TABLE test_table_default;
|
||||
|
||||
DROP DATABASE 02097_db;
|
||||
|
Loading…
Reference in New Issue
Block a user