rename evaluateDatabaseName function and header, add comments

This commit is contained in:
Andrey Mironov 2014-09-23 20:02:04 +04:00
parent 0a4e3f1280
commit f77ef5f84d
5 changed files with 58 additions and 50 deletions

View File

@ -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;
}
}
}

View 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);
}
}

View File

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

View File

@ -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";

View File

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