mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
rename evaluateDatabaseName function and header, add comments
This commit is contained in:
parent
0a4e3f1280
commit
f77ef5f84d
@ -1,42 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
ASTPtr makeIdentifier(const ASTPtr & expr, const Context & context)
|
||||
{
|
||||
/// for identifier just return its name
|
||||
if (typeid_cast<const ASTIdentifier *>(expr.get()))
|
||||
return expr;
|
||||
|
||||
/// for string literal return its value
|
||||
if (const auto literal = typeid_cast<const ASTLiteral *>(expr.get()))
|
||||
return new ASTIdentifier{{}, safeGet<const String &>(literal->value)};
|
||||
|
||||
/// otherwise evaluate expression and ensure it has string type
|
||||
Block block{};
|
||||
ExpressionAnalyzer{expr, context, { { "", new DataTypeString } }}.getActions(false)->execute(block);
|
||||
|
||||
const auto & column_name_type = block.getByName(expr->getColumnName());
|
||||
|
||||
if (!typeid_cast<const DataTypeString *>(column_name_type.type.get()))
|
||||
throw Exception{""};
|
||||
|
||||
return new ASTIdentifier{{}, column_name_type.column->getDataAt(0).toString()};
|
||||
}
|
||||
|
||||
String evaluateDatabaseName(ASTPtr & expr, const Context & context)
|
||||
{
|
||||
expr = makeIdentifier(expr, context);
|
||||
return static_cast<ASTIdentifier *>(expr.get())->name;
|
||||
}
|
||||
}
|
||||
}
|
50
dbms/include/DB/Interpreters/reinterpretAsIdentifier.h
Normal file
50
dbms/include/DB/Interpreters/reinterpretAsIdentifier.h
Normal file
@ -0,0 +1,50 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace
|
||||
{
|
||||
ASTPtr reinterpretAsIdentifierImpl(const ASTPtr & expr, const Context & context)
|
||||
{
|
||||
/// for string literal return its value
|
||||
if (const auto literal = typeid_cast<const ASTLiteral *>(expr.get()))
|
||||
return new ASTIdentifier{{}, safeGet<const String &>(literal->value)};
|
||||
|
||||
/// otherwise evaluate the expression
|
||||
Block block{};
|
||||
/** pass a dummy column name because ExpressioAnalyzer
|
||||
* does not work with no columns so far. */
|
||||
ExpressionAnalyzer{
|
||||
expr, context,
|
||||
{ { "", new DataTypeString } }
|
||||
}.getActions(false)->execute(block);
|
||||
|
||||
const auto & column_name_type = block.getByName(expr->getColumnName());
|
||||
|
||||
/// ensure the result of evaluation has String type
|
||||
if (!typeid_cast<const DataTypeString *>(column_name_type.type.get()))
|
||||
throw Exception{"Expression must evaluate to a String"};
|
||||
|
||||
return new ASTIdentifier{{}, column_name_type.column->getDataAt(0).toString()};
|
||||
}
|
||||
}
|
||||
|
||||
/** \brief if `expr` is not already ASTIdentifier evaluates it
|
||||
* and replaces by a new ASTIdentifier with the result of evaluation as its name.
|
||||
* `expr` must evaluate to a String type */
|
||||
inline ASTIdentifier & reinterpretAsIdentifier(ASTPtr & expr, const Context & context)
|
||||
{
|
||||
/// for identifier just return its name
|
||||
if (!typeid_cast<const ASTIdentifier *>(expr.get()))
|
||||
expr = reinterpretAsIdentifierImpl(expr, context);
|
||||
|
||||
return static_cast<ASTIdentifier &>(*expr);
|
||||
}
|
||||
}
|
@ -8,7 +8,7 @@
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
#include <DB/TableFunctions/ITableFunction.h>
|
||||
#include <DB/Interpreters/evaluateDatabaseName.h>
|
||||
#include <DB/Interpreters/reinterpretAsIdentifier.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -41,7 +41,7 @@ public:
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
String source_database = evaluateDatabaseName(args[0], context);
|
||||
String source_database = reinterpretAsIdentifier(args[0], context).name;
|
||||
String table_name_regexp = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[1]).value);
|
||||
|
||||
/// В InterpreterSelectQuery будет создан ExpressionAnalzyer, который при обработке запроса наткнется на этот Identifier.
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <DB/Storages/StorageDistributed.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
||||
#include <DB/Interpreters/evaluateDatabaseName.h>
|
||||
#include <DB/Interpreters/reinterpretAsIdentifier.h>
|
||||
|
||||
|
||||
struct data;
|
||||
@ -44,7 +44,7 @@ public:
|
||||
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
String descripton = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[0]).value);
|
||||
String remote_database = evaluateDatabaseName(args[1], context);
|
||||
String remote_database = reinterpretAsIdentifier(args[1], context).name;
|
||||
String remote_table = args.size() % 2 ? typeid_cast<ASTIdentifier &>(*args[2]).name : "";
|
||||
String username = args.size() >= 4
|
||||
? safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[args.size() - 2]).value) : "default";
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/evaluateDatabaseName.h>
|
||||
#include <DB/Interpreters/reinterpretAsIdentifier.h>
|
||||
|
||||
#include <DB/Storages/StorageLog.h>
|
||||
#include <DB/Storages/StorageTinyLog.h>
|
||||
@ -108,7 +108,7 @@ StoragePtr StorageFactory::get(
|
||||
if (args.size() < 3 || args.size() > 4)
|
||||
break;
|
||||
|
||||
String source_database = evaluateDatabaseName(args[0], local_context);
|
||||
String source_database = reinterpretAsIdentifier(args[0], local_context).name;
|
||||
String source_table_name_regexp = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[1]).value);
|
||||
size_t chunks_to_merge = safeGet<UInt64>(typeid_cast<ASTLiteral &>(*args[2]).value);
|
||||
|
||||
@ -156,7 +156,7 @@ StoragePtr StorageFactory::get(
|
||||
" - name of source database and regexp for table names.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
String source_database = evaluateDatabaseName(args[0], local_context);
|
||||
String source_database = reinterpretAsIdentifier(args[0], local_context).name;
|
||||
String table_name_regexp = safeGet<const String &>(typeid_cast<ASTLiteral &>(*args[1]).value);
|
||||
|
||||
return StorageMerge::create(table_name, columns, source_database, table_name_regexp, context);
|
||||
@ -182,7 +182,7 @@ StoragePtr StorageFactory::get(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
String cluster_name = typeid_cast<ASTIdentifier &>(*args[0]).name;
|
||||
String remote_database = evaluateDatabaseName(args[1], local_context);
|
||||
String remote_database = reinterpretAsIdentifier(args[1], local_context).name;
|
||||
String remote_table = typeid_cast<ASTIdentifier &>(*args[2]).name;
|
||||
|
||||
const auto & sharding_key = args.size() == 4 ? args[3] : nullptr;
|
||||
|
Loading…
Reference in New Issue
Block a user