mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge
This commit is contained in:
commit
3925f6dcf9
@ -36,12 +36,13 @@ namespace DB
|
||||
* Быстрая некриптографическая хэш функция для строк:
|
||||
* cityHash64: String -> UInt64
|
||||
*
|
||||
* Некриптографический хеш от кортежа значений любых типов (использует cityHash64 для строк и intHash64 для чисел):
|
||||
* cityHash64: any* -> UInt64
|
||||
*
|
||||
* Быстрая некриптографическая хэш функция от любого целого числа:
|
||||
* intHash32: number -> UInt32
|
||||
* intHash64: number -> UInt64
|
||||
*
|
||||
* Некриптографический хеш от кортежа значений любых типов (использует cityHash64 для строк и intHash64 для чисел):
|
||||
* anyHash64: any* -> UInt64
|
||||
*/
|
||||
|
||||
struct HalfMD5Impl
|
||||
@ -71,14 +72,6 @@ struct SipHash64Impl
|
||||
}
|
||||
};
|
||||
|
||||
struct CityHash64Impl
|
||||
{
|
||||
static UInt64 apply(const char * begin, size_t size)
|
||||
{
|
||||
return CityHash64(begin, size);
|
||||
}
|
||||
};
|
||||
|
||||
struct IntHash32Impl
|
||||
{
|
||||
typedef UInt32 ReturnType;
|
||||
@ -258,10 +251,10 @@ UInt64 toInteger<Float64>(Float64 x)
|
||||
}
|
||||
|
||||
|
||||
class FunctionAnyHash64 : public IFunction
|
||||
class FunctionCityHash64 : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename FromType>
|
||||
template <typename FromType, bool first>
|
||||
void executeIntType(const IColumn * column, ColumnUInt64::Container_t & vec_to)
|
||||
{
|
||||
if (const ColumnVector<FromType> * col_from = typeid_cast<const ColumnVector<FromType> *>(column))
|
||||
@ -269,14 +262,27 @@ private:
|
||||
const typename ColumnVector<FromType>::Container_t & vec_from = col_from->getData();
|
||||
size_t size = vec_from.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], IntHash64Impl::apply(toInteger(vec_from[i]))));
|
||||
{
|
||||
UInt64 h = IntHash64Impl::apply(toInteger(vec_from[i]));
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
else
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], h));
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst<FromType> * col_from = typeid_cast<const ColumnConst<FromType> *>(column))
|
||||
{
|
||||
UInt64 hash = IntHash64Impl::apply(toInteger(col_from->getData()));
|
||||
size_t size = vec_to.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], hash));
|
||||
if (first)
|
||||
{
|
||||
vec_to.assign(size, hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], hash));
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + column->getName()
|
||||
@ -284,6 +290,7 @@ private:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <bool first>
|
||||
void executeString(const IColumn * column, ColumnUInt64::Container_t & vec_to)
|
||||
{
|
||||
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(column))
|
||||
@ -293,26 +300,45 @@ private:
|
||||
size_t size = offsets.size();
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], CityHash64Impl::apply(
|
||||
{
|
||||
UInt64 h = CityHash64(
|
||||
reinterpret_cast<const char *>(&data[i == 0 ? 0 : offsets[i - 1]]),
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]))));
|
||||
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
else
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], h));
|
||||
}
|
||||
}
|
||||
else if (const ColumnFixedString * col_from = typeid_cast<const ColumnFixedString *>(column))
|
||||
{
|
||||
const typename ColumnString::Chars_t & data = col_from->getChars();
|
||||
size_t n = col_from->getN();
|
||||
size_t size = data.size() / n;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], CityHash64Impl::apply(
|
||||
reinterpret_cast<const char *>(&data[i * n]), n)));
|
||||
{
|
||||
UInt64 h = CityHash64(reinterpret_cast<const char *>(&data[i * n]), n);
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
else
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], h));
|
||||
}
|
||||
}
|
||||
else if (const ColumnConstString * col_from = typeid_cast<const ColumnConstString *>(column))
|
||||
{
|
||||
UInt64 hash = CityHash64Impl::apply(col_from->getData().data(), col_from->getData().size());
|
||||
UInt64 hash = CityHash64(col_from->getData().data(), col_from->getData().size());
|
||||
size_t size = vec_to.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], hash));
|
||||
if (first)
|
||||
{
|
||||
vec_to.assign(size, hash);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], hash));
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + column->getName()
|
||||
@ -320,6 +346,7 @@ private:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <bool first>
|
||||
void executeArray(const IDataType * type, const IColumn * column, ColumnUInt64::Container_t & vec_to)
|
||||
{
|
||||
const IDataType * nested_type = &*typeid_cast<const DataTypeArray *>(type)->getNestedType();
|
||||
@ -330,8 +357,8 @@ private:
|
||||
const ColumnArray::Offsets_t & offsets = col_from->getOffsets();
|
||||
size_t nested_size = nested_column->size();
|
||||
|
||||
ColumnUInt64::Container_t vec_temp(nested_size, 0);
|
||||
executeAny(nested_type, nested_column, vec_temp);
|
||||
ColumnUInt64::Container_t vec_temp(nested_size);
|
||||
executeAny<true>(nested_type, nested_column, vec_temp);
|
||||
|
||||
size_t size = offsets.size();
|
||||
|
||||
@ -339,7 +366,13 @@ private:
|
||||
{
|
||||
size_t begin = i == 0 ? 0 : offsets[i - 1];
|
||||
size_t end = offsets[i];
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], IntHash64Impl::apply(end - begin)));
|
||||
|
||||
UInt64 h = IntHash64Impl::apply(end - begin);
|
||||
if (first)
|
||||
vec_to[i] = h;
|
||||
else
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], h));
|
||||
|
||||
for (size_t j = begin; j < end; ++j)
|
||||
vec_to[i] = Hash128to64(uint128(vec_to[i], vec_temp[j]));
|
||||
}
|
||||
@ -348,7 +381,7 @@ private:
|
||||
{
|
||||
/// NOTE: тут, конечно, можно обойтись без материалиации столбца.
|
||||
ColumnPtr full_column = col_from->convertToFullColumn();
|
||||
executeArray(type, &*full_column, vec_to);
|
||||
executeArray<first>(type, &*full_column, vec_to);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + column->getName()
|
||||
@ -356,23 +389,24 @@ private:
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <bool first>
|
||||
void executeAny(const IDataType * from_type, const IColumn * icolumn, ColumnUInt64::Container_t & vec_to)
|
||||
{
|
||||
if (typeid_cast<const DataTypeUInt8 * >(from_type)) executeIntType<UInt8 >(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeUInt16 * >(from_type)) executeIntType<UInt16>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeUInt32 * >(from_type)) executeIntType<UInt32>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeUInt64 * >(from_type)) executeIntType<UInt64>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt8 * >(from_type)) executeIntType<Int8 >(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt16 * >(from_type)) executeIntType<Int16 >(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt32 * >(from_type)) executeIntType<Int32 >(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt64 * >(from_type)) executeIntType<Int64 >(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeDate * >(from_type)) executeIntType<UInt16>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeDateTime * >(from_type)) executeIntType<UInt32>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeFloat32 * >(from_type)) executeIntType<Float32>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeFloat64 * >(from_type)) executeIntType<Float64>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeString * >(from_type)) executeString(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeFixedString * >(from_type)) executeString(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeArray * >(from_type)) executeArray(from_type, icolumn, vec_to);
|
||||
if (typeid_cast<const DataTypeUInt8 * >(from_type)) executeIntType<UInt8, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeUInt16 * >(from_type)) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeUInt32 * >(from_type)) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeUInt64 * >(from_type)) executeIntType<UInt64, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt8 * >(from_type)) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt16 * >(from_type)) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt32 * >(from_type)) executeIntType<Int32, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeInt64 * >(from_type)) executeIntType<Int64, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeDate * >(from_type)) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeDateTime * >(from_type)) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeFloat32 * >(from_type)) executeIntType<Float32, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeFloat64 * >(from_type)) executeIntType<Float64, first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeString * >(from_type)) executeString < first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeFixedString *>(from_type)) executeString < first>(icolumn, vec_to);
|
||||
else if (typeid_cast<const DataTypeArray * >(from_type)) executeArray < first>(from_type, icolumn, vec_to);
|
||||
else
|
||||
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -382,7 +416,7 @@ public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return "anyHash64";
|
||||
return "cityHash64";
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
@ -399,15 +433,23 @@ public:
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
ColumnUInt64::Container_t & vec_to = col_to->getData();
|
||||
vec_to.assign(rows, 0ul);
|
||||
|
||||
for (size_t index : arguments)
|
||||
if (arguments.empty())
|
||||
{
|
||||
const ColumnWithNameAndType & column = block.getByPosition(index);
|
||||
/// Случайное число из /dev/urandom используется как хеш пустого кортежа.
|
||||
vec_to.assign(rows, 0xe28dbde7fe22e41c);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const ColumnWithNameAndType & column = block.getByPosition(arguments[i]);
|
||||
const IDataType * from_type = &*column.type;
|
||||
const IColumn * icolumn = &*column.column;
|
||||
|
||||
executeAny(from_type, icolumn, vec_to);
|
||||
if (i == 0)
|
||||
executeAny<true>(from_type, icolumn, vec_to);
|
||||
else
|
||||
executeAny<false>(from_type, icolumn, vec_to);
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -421,7 +463,6 @@ struct NameIntHash64 { static const char * get() { return "intHash64"; } };
|
||||
|
||||
typedef FunctionStringHash64<HalfMD5Impl, NameHalfMD5> FunctionHalfMD5;
|
||||
typedef FunctionStringHash64<SipHash64Impl, NameSipHash64> FunctionSipHash64;
|
||||
typedef FunctionStringHash64<CityHash64Impl, NameCityHash64> FunctionCityHash64;
|
||||
typedef FunctionIntHash<IntHash32Impl, NameIntHash32> FunctionIntHash32;
|
||||
typedef FunctionIntHash<IntHash64Impl, NameIntHash64> FunctionIntHash64;
|
||||
|
||||
|
@ -199,8 +199,8 @@ private:
|
||||
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
|
||||
void makeSet(ASTFunction * node, const Block & sample_block);
|
||||
|
||||
/// Запустить подзапрос в секции GLOBAL IN, создать временную таблицу типа Memory и запомнить эту пару в переменной external_tables.
|
||||
void addExternalStorage(ASTFunction * node);
|
||||
/// Запустить подзапрос в секции GLOBAL IN/JOIN, создать временную таблицу типа Memory и запомнить эту пару в переменной external_tables.
|
||||
void addExternalStorage(ASTPtr & subquery_or_table_name);
|
||||
|
||||
void getArrayJoinedColumns();
|
||||
void getArrayJoinedColumnsImpl(ASTPtr ast);
|
||||
@ -211,13 +211,13 @@ private:
|
||||
struct ScopeStack;
|
||||
void getActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack);
|
||||
|
||||
void getRootActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions);
|
||||
void getRootActions(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions);
|
||||
|
||||
void getActionsBeforeAggregationImpl(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries);
|
||||
void getActionsBeforeAggregation(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries);
|
||||
|
||||
/// Добавить агрегатные функции в aggregate_descriptions.
|
||||
/// Установить has_aggregation = true, если есть хоть одна агрегатная функция.
|
||||
void getAggregatesImpl(ASTPtr ast, ExpressionActionsPtr & actions);
|
||||
void getAggregates(ASTPtr ast, ExpressionActionsPtr & actions);
|
||||
|
||||
/** Получить множество нужных столбцов для чтения из таблицы.
|
||||
* При этом, столбцы, указанные в ignored_names, считаются ненужными. И параметр ignored_names может модифицироваться.
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IAST.h>
|
||||
#include <DB/Parsers/ASTWithAlias.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
@ -11,7 +11,7 @@ namespace DB
|
||||
|
||||
/** Применение функции или оператора
|
||||
*/
|
||||
class ASTFunction : public IAST
|
||||
class ASTFunction : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
enum FunctionKind
|
||||
@ -23,20 +23,18 @@ public:
|
||||
LAMBDA_EXPRESSION,
|
||||
ARRAY_JOIN,
|
||||
};
|
||||
|
||||
|
||||
/// имя функции
|
||||
String name;
|
||||
/// аргументы
|
||||
ASTPtr arguments;
|
||||
/// параметры - для параметрических агрегатных функций. Пример: quantile(0.9)(x) - то, что в первых скобках - параметры.
|
||||
ASTPtr parameters;
|
||||
/// алиас, если есть
|
||||
String alias;
|
||||
|
||||
|
||||
FunctionKind kind;
|
||||
|
||||
ASTFunction() : kind(UNKNOWN) {}
|
||||
ASTFunction(StringRange range_) : IAST(range_), kind(UNKNOWN) {}
|
||||
ASTFunction(StringRange range_) : ASTWithAlias(range_), kind(UNKNOWN) {}
|
||||
|
||||
String getColumnName() const
|
||||
{
|
||||
@ -68,8 +66,6 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
String getAlias() const { return alias.empty() ? getColumnName() : alias; }
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return "Function_" + name; }
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
#include <DB/Parsers/IAST.h>
|
||||
#include <DB/Parsers/ASTWithAlias.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -9,7 +9,7 @@ namespace DB
|
||||
|
||||
/** Идентификатор (столбца или алиас, или именованый элемент кортежа)
|
||||
*/
|
||||
class ASTIdentifier : public IAST
|
||||
class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
enum Kind
|
||||
@ -19,22 +19,18 @@ public:
|
||||
Table,
|
||||
Format,
|
||||
};
|
||||
|
||||
|
||||
/// имя
|
||||
String name;
|
||||
/// алиас, если есть
|
||||
String alias;
|
||||
|
||||
/// чего идентифицирует этот идентификатор
|
||||
Kind kind;
|
||||
|
||||
ASTIdentifier() {}
|
||||
ASTIdentifier(StringRange range_, const String & name_, Kind kind_ = Column) : IAST(range_), name(name_), kind(kind_) {}
|
||||
ASTIdentifier(StringRange range_, const String & name_, Kind kind_ = Column) : ASTWithAlias(range_), name(name_), kind(kind_) {}
|
||||
|
||||
String getColumnName() const { return name; }
|
||||
|
||||
String getAlias() const { return alias.empty() ? getColumnName() : alias; }
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return "Identifier_" + name; }
|
||||
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
Strictness strictness = Any;
|
||||
Kind kind = Inner;
|
||||
|
||||
ASTPtr subquery; /// Подзапрос - "правая" таблица для соединения.
|
||||
ASTPtr table; /// "Правая" таблица для соединения - подзапрос или имя таблицы.
|
||||
ASTPtr using_expr_list; /// По каким столбцам выполнять соединение.
|
||||
|
||||
ASTJoin() {}
|
||||
@ -66,7 +66,7 @@ public:
|
||||
ASTJoin * res = new ASTJoin(*this);
|
||||
res->children.clear();
|
||||
|
||||
if (subquery) { res->subquery = subquery->clone(); res->children.push_back(res->subquery); }
|
||||
if (table) { res->table = table->clone(); res->children.push_back(res->table); }
|
||||
if (using_expr_list) { res->using_expr_list = using_expr_list->clone(); res->children.push_back(res->using_expr_list); }
|
||||
|
||||
return res;
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <DB/Core/Field.h>
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
#include <DB/Parsers/IAST.h>
|
||||
#include <DB/Parsers/ASTWithAlias.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,22 +10,18 @@ namespace DB
|
||||
|
||||
/** Литерал (атомарный) - число, строка, NULL
|
||||
*/
|
||||
class ASTLiteral : public IAST
|
||||
class ASTLiteral : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
Field value;
|
||||
/// тип
|
||||
DataTypePtr type;
|
||||
/// алиас, если есть
|
||||
String alias;
|
||||
|
||||
ASTLiteral() {}
|
||||
ASTLiteral(StringRange range_, const Field & value_) : IAST(range_), value(value_) {}
|
||||
ASTLiteral(StringRange range_, const Field & value_) : ASTWithAlias(range_), value(value_) {}
|
||||
|
||||
String getColumnName() const { return apply_visitor(FieldVisitorToString(), value); }
|
||||
|
||||
String getAlias() const { return alias.empty() ? getColumnName() : alias; }
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return "Literal_" + apply_visitor(FieldVisitorDump(), value); }
|
||||
|
||||
|
@ -73,7 +73,7 @@ public:
|
||||
bool done = 0;
|
||||
for (size_t j = 0; j < asts.size(); ++j)
|
||||
{
|
||||
if (asts[j]->getAlias() == column_names[i])
|
||||
if (asts[j]->getAliasOrColumnName() == column_names[i])
|
||||
{
|
||||
if (!unremovable_asts.count(asts[j]))
|
||||
result->children.push_back(asts[j]->clone());
|
||||
@ -81,7 +81,7 @@ public:
|
||||
}
|
||||
}
|
||||
if (!done)
|
||||
throw Exception("Error while rewriting expressioin list for select query."
|
||||
throw Exception("Error while rewriting expression list for select query."
|
||||
" Could not find alias: " + column_names[i],
|
||||
DB::ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
|
24
dbms/include/DB/Parsers/ASTWithAlias.h
Normal file
24
dbms/include/DB/Parsers/ASTWithAlias.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IAST.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Базовый класс для AST, которые могут содержать алиас (идентификаторы, литералы, функции).
|
||||
*/
|
||||
class ASTWithAlias : public IAST
|
||||
{
|
||||
public:
|
||||
/// Алиас, если есть, или пустая строка.
|
||||
String alias;
|
||||
|
||||
using IAST::IAST;
|
||||
|
||||
String getAliasOrColumnName() const override { return alias.empty() ? getColumnName() : alias; }
|
||||
String tryGetAlias() const override { return alias; }
|
||||
void setAlias(const String & to) override { alias = to; }
|
||||
};
|
||||
|
||||
}
|
@ -32,7 +32,7 @@ public:
|
||||
typedef std::vector<SharedPtr<IAST> > ASTs;
|
||||
ASTs children;
|
||||
StringRange range;
|
||||
|
||||
|
||||
/** Строка с полным запросом.
|
||||
* Этот указатель не дает ее удалить, пока range в нее ссылается.
|
||||
*/
|
||||
@ -45,9 +45,18 @@ public:
|
||||
/** Получить каноническое имя столбца, если элемент является столбцом */
|
||||
virtual String getColumnName() const { throw Exception("Trying to get name of not a column: " + getID(), ErrorCodes::NOT_A_COLUMN); }
|
||||
|
||||
/** Получить алиас, если он есть, или каноническое имя столбца; если элемент является столбцом */
|
||||
virtual String getAlias() const { return getColumnName(); }
|
||||
|
||||
/** Получить алиас, если он есть, или каноническое имя столбца, если его нет. */
|
||||
virtual String getAliasOrColumnName() const { return getColumnName(); }
|
||||
|
||||
/** Получить алиас, если он есть, или пустую строку, если его нет, или если элемент не поддерживает алиасы. */
|
||||
virtual String tryGetAlias() const { return String(); }
|
||||
|
||||
/** Установить алиас. */
|
||||
virtual void setAlias(const String & to)
|
||||
{
|
||||
throw Exception("Can't set alias of " + getColumnName(), ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
||||
}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
virtual String getID() const = 0;
|
||||
|
||||
@ -55,7 +64,7 @@ public:
|
||||
virtual SharedPtr<IAST> clone() const = 0;
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент и всё поддерево.
|
||||
* Обычно он содержит идентификатор элемента и getTreeID от всех детей.
|
||||
* Обычно он содержит идентификатор элемента и getTreeID от всех детей.
|
||||
*/
|
||||
String getTreeID() const
|
||||
{
|
||||
@ -93,7 +102,7 @@ public:
|
||||
{
|
||||
return checkDepthImpl(max_depth, 0);
|
||||
}
|
||||
|
||||
|
||||
/** То же самое для общего количества элементов дерева.
|
||||
*/
|
||||
size_t checkSize(size_t max_size) const
|
||||
@ -104,7 +113,7 @@ public:
|
||||
|
||||
if (res > max_size)
|
||||
throw Exception("AST is too big. Maximum: " + toString(max_size), ErrorCodes::TOO_BIG_AST);
|
||||
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -149,7 +149,6 @@ FunctionPtr FunctionFactory::get(
|
||||
else if (name == "cityHash64") return new FunctionCityHash64;
|
||||
else if (name == "intHash32") return new FunctionIntHash32;
|
||||
else if (name == "intHash64") return new FunctionIntHash64;
|
||||
else if (name == "anyHash64") return new FunctionAnyHash64;
|
||||
|
||||
else if (name == "IPv4NumToString") return new FunctionIPv4NumToString;
|
||||
else if (name == "IPv4StringToNum") return new FunctionIPv4StringToNum;
|
||||
|
@ -37,44 +37,11 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
static std::string * tryGetAlias(ASTPtr & ast)
|
||||
{
|
||||
if (ASTFunction * node = typeid_cast<ASTFunction *>(&*ast))
|
||||
return &node->alias;
|
||||
else if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(&*ast))
|
||||
return &node->alias;
|
||||
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
|
||||
return &node->alias;
|
||||
else
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
static void setAlias(ASTPtr & ast, const std::string & alias)
|
||||
{
|
||||
if (ASTFunction * node = typeid_cast<ASTFunction *>(&*ast))
|
||||
{
|
||||
node->alias = alias;
|
||||
}
|
||||
else if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(&*ast))
|
||||
{
|
||||
node->alias = alias;
|
||||
}
|
||||
else if (ASTLiteral * node = typeid_cast<ASTLiteral *>(&*ast))
|
||||
{
|
||||
node->alias = alias;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Can't set alias of " + ast->getColumnName(), ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::init()
|
||||
{
|
||||
select_query = typeid_cast<ASTSelectQuery *>(&*ast);
|
||||
|
||||
createAliasesDict(ast); /// Если есть агрегатные функции, присвоит has_aggregation=true.
|
||||
createAliasesDict(ast); /// Если есть агрегатные функции, присвоит has_aggregation = true.
|
||||
normalizeTree();
|
||||
|
||||
findExternalTables(ast);
|
||||
@ -91,17 +58,17 @@ void ExpressionAnalyzer::init()
|
||||
|
||||
if (select_query && select_query->array_join_expression_list)
|
||||
{
|
||||
getRootActionsImpl(select_query->array_join_expression_list, true, false, temp_actions);
|
||||
getRootActions(select_query->array_join_expression_list, true, false, temp_actions);
|
||||
addMultipleArrayJoinAction(temp_actions);
|
||||
}
|
||||
|
||||
if (select_query && select_query->join)
|
||||
{
|
||||
getRootActionsImpl(typeid_cast<ASTJoin &>(*select_query->join).using_expr_list, true, false, temp_actions);
|
||||
getRootActions(typeid_cast<ASTJoin &>(*select_query->join).using_expr_list, true, false, temp_actions);
|
||||
addJoinAction(temp_actions, true);
|
||||
}
|
||||
|
||||
getAggregatesImpl(ast, temp_actions);
|
||||
getAggregates(ast, temp_actions);
|
||||
|
||||
if (has_aggregation)
|
||||
{
|
||||
@ -114,7 +81,7 @@ void ExpressionAnalyzer::init()
|
||||
const ASTs & group_asts = select_query->group_expression_list->children;
|
||||
for (size_t i = 0; i < group_asts.size(); ++i)
|
||||
{
|
||||
getRootActionsImpl(group_asts[i], true, false, temp_actions);
|
||||
getRootActions(group_asts[i], true, false, temp_actions);
|
||||
|
||||
NameAndTypePair key;
|
||||
key.first = group_asts[i]->getColumnName();
|
||||
@ -175,13 +142,13 @@ void ExpressionAnalyzer::createAliasesDict(ASTPtr & ast, int ignore_levels)
|
||||
if (ignore_levels > 0)
|
||||
return;
|
||||
|
||||
std::string * alias = tryGetAlias(ast);
|
||||
if (alias && !alias->empty())
|
||||
String alias = ast->tryGetAlias();
|
||||
if (!alias.empty())
|
||||
{
|
||||
if (aliases.count(*alias) && ast->getTreeID() != aliases[*alias]->getTreeID())
|
||||
throw Exception("Different expressions with the same alias " + *alias, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
|
||||
if (aliases.count(alias) && ast->getTreeID() != aliases[alias]->getTreeID())
|
||||
throw Exception("Different expressions with the same alias " + alias, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS);
|
||||
|
||||
aliases[*alias] = ast;
|
||||
aliases[alias] = ast;
|
||||
}
|
||||
}
|
||||
|
||||
@ -226,9 +193,9 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
|
||||
ASTPtr initial_ast = ast;
|
||||
current_asts.insert(initial_ast);
|
||||
|
||||
std::string * my_alias = tryGetAlias(ast);
|
||||
if (my_alias && !my_alias->empty())
|
||||
current_alias = *my_alias;
|
||||
String my_alias = ast->tryGetAlias();
|
||||
if (!my_alias.empty())
|
||||
current_alias = my_alias;
|
||||
|
||||
/// rewrite правила, которые действуют при обходе сверху-вниз.
|
||||
bool replaced = false;
|
||||
@ -247,7 +214,8 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
|
||||
current_asts.insert(ast);
|
||||
replaced = true;
|
||||
}
|
||||
/// может быть указано in t, где t - таблица, что равносильно select * from t.
|
||||
|
||||
/// может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t).
|
||||
if (node->name == "in" || node->name == "notIn" || node->name == "globalIn" || node->name == "globalNotIn")
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*node->arguments->children[1]))
|
||||
right->kind = ASTIdentifier::Table;
|
||||
@ -263,11 +231,11 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
|
||||
/// Заменим его на соответствующий узел дерева.
|
||||
if (current_asts.count(jt->second))
|
||||
throw Exception("Cyclic aliases", ErrorCodes::CYCLIC_ALIASES);
|
||||
if (my_alias && !my_alias->empty() && *my_alias != jt->second->getAlias())
|
||||
if (!my_alias.empty() && my_alias != jt->second->getAliasOrColumnName())
|
||||
{
|
||||
/// В конструкции вроде "a AS b", где a - алиас, нужно перевесить алиас b на результат подстановки алиаса a.
|
||||
ast = jt->second->clone();
|
||||
setAlias(ast, *my_alias);
|
||||
ast->setAlias(my_alias);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -347,8 +315,20 @@ void ExpressionAnalyzer::normalizeTreeImpl(ASTPtr & ast, MapOfASTs & finished_as
|
||||
node->kind = ASTFunction::FUNCTION;
|
||||
}
|
||||
|
||||
/// Для GLOBAL IN.
|
||||
if (do_global && (node->name == "globalIn" || node->name == "globalNotIn"))
|
||||
addExternalStorage(node);
|
||||
addExternalStorage(node->arguments->children.at(1));
|
||||
}
|
||||
|
||||
if (ASTJoin * node = typeid_cast<ASTJoin *>(&*ast))
|
||||
{
|
||||
/// может быть указано JOIN t, где t - таблица, что равносильно JOIN (SELECT * FROM t).
|
||||
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(&*node->table))
|
||||
right->kind = ASTIdentifier::Table;
|
||||
|
||||
/// Для GLOBAL JOIN.
|
||||
if (do_global && node->locality == ASTJoin::Global)
|
||||
addExternalStorage(node->table);
|
||||
}
|
||||
|
||||
current_asts.erase(initial_ast);
|
||||
@ -393,111 +373,99 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl
|
||||
|
||||
void ExpressionAnalyzer::findExternalTables(ASTPtr & ast)
|
||||
{
|
||||
/// Рекурсивные вызовы. Намеренно опускаемся в подзапросы.
|
||||
/// Обход снизу. Намеренно опускаемся в подзапросы.
|
||||
for (ASTs::iterator it = ast->children.begin(); it != ast->children.end(); ++it)
|
||||
findExternalTables(*it);
|
||||
|
||||
/// Если идентификатор типа таблица
|
||||
StoragePtr external_storage;
|
||||
|
||||
if (ASTIdentifier * node = typeid_cast<ASTIdentifier *>(&*ast))
|
||||
if (node->kind == ASTIdentifier::Kind::Table)
|
||||
if ((external_storage = context.tryGetExternalTable(node->name)))
|
||||
external_tables[node->name] = external_storage;
|
||||
|
||||
if (ASTFunction * node = typeid_cast<ASTFunction *>(&*ast))
|
||||
{
|
||||
if (node->name == "globalIn" || node->name == "globalNotIn" || node->name == "In" || node->name == "NotIn")
|
||||
{
|
||||
IAST & args = *node->arguments;
|
||||
ASTPtr & arg = args.children[1];
|
||||
/// Если имя таблицы для селекта
|
||||
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(&*arg))
|
||||
if ((external_storage = context.tryGetExternalTable(id->name)))
|
||||
external_tables[id->name] = external_storage;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::addExternalStorage(ASTFunction * node)
|
||||
void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name)
|
||||
{
|
||||
/// Сгенерируем имя для внешней таблицы.
|
||||
String external_table_name = "_data";
|
||||
while (context.tryGetExternalTable(external_table_name + toString(external_table_id)))
|
||||
while (context.tryGetExternalTable("_data" + toString(external_table_id)))
|
||||
++external_table_id;
|
||||
|
||||
IAST & args = *node->arguments; /// TODO Для JOIN.
|
||||
ASTPtr & arg = args.children[1];
|
||||
StoragePtr external_storage;
|
||||
|
||||
/// Если подзапрос или имя таблицы для селекта
|
||||
if (typeid_cast<const ASTSubquery *>(&*arg) || typeid_cast<const ASTIdentifier *>(&*arg))
|
||||
/// Подзапрос или имя таблицы. Имя таблицы аналогично подзапросу SELECT * FROM t.
|
||||
const ASTSubquery * subquery = typeid_cast<const ASTSubquery *>(&*subquery_or_table_name);
|
||||
const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(&*subquery_or_table_name);
|
||||
|
||||
if (!subquery && !table)
|
||||
throw Exception("IN/JOIN supports only SELECT subqueries.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/** Для подзапроса в секции IN/JOIN не действуют ограничения на максимальный размер результата.
|
||||
* Так как результат этого поздапроса - ещё не результат всего запроса.
|
||||
* Вместо этого работают ограничения
|
||||
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
|
||||
* max_rows_in_join, max_bytes_in_join, join_overflow_mode.
|
||||
*/
|
||||
Context subquery_context = context;
|
||||
Settings subquery_settings = context.getSettings();
|
||||
subquery_settings.limits.max_result_rows = 0;
|
||||
subquery_settings.limits.max_result_bytes = 0;
|
||||
/// Вычисление extremes не имеет смысла и не нужно (если его делать, то в результате всего запроса могут взяться extremes подзапроса).
|
||||
subquery_settings.extremes = 0;
|
||||
subquery_context.setSettings(subquery_settings);
|
||||
|
||||
ASTPtr query;
|
||||
if (table)
|
||||
{
|
||||
/** Для подзапроса в секции IN не действуют ограничения на максимальный размер результата.
|
||||
* Так как результат этого поздапроса - ещё не результат всего запроса.
|
||||
* Вместо этого работают ограничения max_rows_in_set, max_bytes_in_set, set_overflow_mode.
|
||||
*/
|
||||
Context subquery_context = context;
|
||||
Settings subquery_settings = context.getSettings();
|
||||
subquery_settings.limits.max_result_rows = 0;
|
||||
subquery_settings.limits.max_result_bytes = 0;
|
||||
/// Вычисление extremes не имеет смысла и не нужно (если его делать, то в результате всего запроса могут взяться extremes подзапроса).
|
||||
subquery_settings.extremes = 0;
|
||||
subquery_context.setSettings(subquery_settings);
|
||||
ParserSelectQuery parser;
|
||||
|
||||
ASTPtr subquery;
|
||||
if (const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(&*arg))
|
||||
StoragePtr existing_storage;
|
||||
|
||||
/// Если это уже внешняя таблица, ничего заполять не нужно. Просто запоминаем ее наличие.
|
||||
if ((existing_storage = context.tryGetExternalTable(table->name)))
|
||||
{
|
||||
ParserSelectQuery parser;
|
||||
|
||||
StoragePtr existing_storage;
|
||||
|
||||
/// Если это уже внешняя таблица, ничего заполять не нужно. Просто запоминаем ее наличие.
|
||||
if ((existing_storage = context.tryGetExternalTable(table->name)))
|
||||
{
|
||||
external_tables[table->name] = existing_storage;
|
||||
return;
|
||||
}
|
||||
|
||||
String query = "SELECT * FROM " + table->name;
|
||||
const char * begin = query.data();
|
||||
const char * end = begin + query.size();
|
||||
const char * pos = begin;
|
||||
Expected expected = "";
|
||||
|
||||
bool parse_res = parser.parse(pos, end, subquery, expected);
|
||||
if (!parse_res)
|
||||
throw Exception("Error in parsing SELECT query while creating set for table " + table->name + ".",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
external_tables[table->name] = existing_storage;
|
||||
return;
|
||||
}
|
||||
else
|
||||
subquery = arg->children[0];
|
||||
|
||||
InterpreterSelectQuery interpreter(subquery, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1);
|
||||
String query_str = "SELECT * FROM " + table->name;
|
||||
const char * begin = query_str.data();
|
||||
const char * end = begin + query_str.size();
|
||||
const char * pos = begin;
|
||||
Expected expected = "";
|
||||
|
||||
Block sample = interpreter.getSampleBlock();
|
||||
NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList());
|
||||
|
||||
String external_table_name = "_data" + toString(external_table_id++);
|
||||
external_storage = StorageMemory::create(external_table_name, columns);
|
||||
|
||||
ASTIdentifier * ast_ident = new ASTIdentifier;
|
||||
ast_ident->kind = ASTIdentifier::Table;
|
||||
ast_ident->name = external_storage->getTableName();
|
||||
arg = ast_ident;
|
||||
external_tables[external_table_name] = external_storage;
|
||||
external_data[external_table_name] = interpreter.execute();
|
||||
|
||||
/// Добавляем множество, при обработке которого будет заполнена внешняя таблица.
|
||||
ASTSet * ast_set = new ASTSet("external_" + arg->getColumnName());
|
||||
ast_set->set = new Set(settings.limits);
|
||||
ast_set->set->setSource(external_data[external_table_name]);
|
||||
ast_set->set->setExternalOutput(external_tables[external_table_name]);
|
||||
ast_set->set->setOnlyExternal(true);
|
||||
sets_with_subqueries[ast_set->getColumnName()] = ast_set->set;
|
||||
bool parse_res = parser.parse(pos, end, query, expected);
|
||||
if (!parse_res)
|
||||
throw Exception("Error in parsing SELECT query while creating set or join for table " + table->name + ".",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
else
|
||||
throw Exception("GLOBAL [NOT] IN supports only SELECT data.", ErrorCodes::BAD_ARGUMENTS);
|
||||
query = subquery->children.at(0);
|
||||
|
||||
InterpreterSelectQuery interpreter(query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1);
|
||||
|
||||
Block sample = interpreter.getSampleBlock();
|
||||
NamesAndTypesListPtr columns = new NamesAndTypesList(sample.getColumnsList());
|
||||
|
||||
String external_table_name = "_data" + toString(external_table_id++);
|
||||
external_storage = StorageMemory::create(external_table_name, columns);
|
||||
|
||||
ASTIdentifier * ast_ident = new ASTIdentifier;
|
||||
ast_ident->kind = ASTIdentifier::Table;
|
||||
ast_ident->name = external_storage->getTableName();
|
||||
subquery_or_table_name = ast_ident;
|
||||
external_tables[external_table_name] = external_storage;
|
||||
external_data[external_table_name] = interpreter.execute();
|
||||
|
||||
/// Добавляем множество, при обработке которого будет заполнена внешняя таблица. // TODO JOIN
|
||||
ASTSet * ast_set = new ASTSet("external_" + subquery_or_table_name->getColumnName());
|
||||
ast_set->set = new Set(settings.limits);
|
||||
ast_set->set->setSource(external_data[external_table_name]);
|
||||
ast_set->set->setExternalOutput(external_tables[external_table_name]);
|
||||
ast_set->set->setOnlyExternal(true);
|
||||
sets_with_subqueries[ast_set->getColumnName()] = ast_set->set;
|
||||
}
|
||||
|
||||
|
||||
@ -777,7 +745,7 @@ struct ExpressionAnalyzer::ScopeStack
|
||||
};
|
||||
|
||||
|
||||
void ExpressionAnalyzer::getRootActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions)
|
||||
void ExpressionAnalyzer::getRootActions(ASTPtr ast, bool no_subqueries, bool only_consts, ExpressionActionsPtr & actions)
|
||||
{
|
||||
ScopeStack scopes(actions, settings);
|
||||
getActionsImpl(ast, no_subqueries, only_consts, scopes);
|
||||
@ -795,7 +763,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
|
||||
ASTPtr ast = array_join_asts [i];
|
||||
|
||||
String nested_table_name = ast->getColumnName();
|
||||
String nested_table_alias = ast->getAlias();
|
||||
String nested_table_alias = ast->getAliasOrColumnName();
|
||||
if (nested_table_alias == nested_table_name && !typeid_cast<ASTIdentifier *>(&*ast))
|
||||
throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED);
|
||||
|
||||
@ -819,7 +787,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns()
|
||||
{
|
||||
ASTPtr expr = select_query->array_join_expression_list->children[0];
|
||||
String source_name = expr->getColumnName();
|
||||
String result_name = expr->getAlias();
|
||||
String result_name = expr->getAliasOrColumnName();
|
||||
|
||||
/// Это массив.
|
||||
if (!typeid_cast<ASTIdentifier *>(&*expr) || findColumn(source_name, columns) != columns.end())
|
||||
@ -1123,7 +1091,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::getAggregatesImpl(ASTPtr ast, ExpressionActionsPtr & actions)
|
||||
void ExpressionAnalyzer::getAggregates(ASTPtr ast, ExpressionActionsPtr & actions)
|
||||
{
|
||||
ASTFunction * node = typeid_cast<ASTFunction *>(&*ast);
|
||||
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
||||
@ -1142,7 +1110,7 @@ void ExpressionAnalyzer::getAggregatesImpl(ASTPtr ast, ExpressionActionsPtr & ac
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
getRootActionsImpl(arguments[i], true, false, actions);
|
||||
getRootActions(arguments[i], true, false, actions);
|
||||
const std::string & name = arguments[i]->getColumnName();
|
||||
types[i] = actions->getSampleBlock().getByName(name).type;
|
||||
aggregate.argument_names[i] = name;
|
||||
@ -1178,7 +1146,7 @@ void ExpressionAnalyzer::getAggregatesImpl(ASTPtr ast, ExpressionActionsPtr & ac
|
||||
{
|
||||
ASTPtr child = ast->children[i];
|
||||
if (!typeid_cast<ASTSubquery *>(&*child) && !typeid_cast<ASTSelectQuery *>(&*child))
|
||||
getAggregatesImpl(child, actions);
|
||||
getAggregates(child, actions);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1227,7 +1195,7 @@ bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool on
|
||||
initChain(chain, columns);
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
getRootActionsImpl(select_query->array_join_expression_list, only_types, false, step.actions);
|
||||
getRootActions(select_query->array_join_expression_list, only_types, false, step.actions);
|
||||
|
||||
addMultipleArrayJoinAction(step.actions);
|
||||
|
||||
@ -1250,7 +1218,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
ASTJoin & ast_join = typeid_cast<ASTJoin &>(*select_query->join);
|
||||
getRootActionsImpl(ast_join.using_expr_list, only_types, false, step.actions);
|
||||
getRootActions(ast_join.using_expr_list, only_types, false, step.actions);
|
||||
|
||||
{
|
||||
Names join_key_names_left(join_key_names_left_set.begin(), join_key_names_left_set.end());
|
||||
@ -1277,8 +1245,9 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
for (const auto & name_type : columns_added_by_join)
|
||||
required_joined_columns.push_back(name_type.first);
|
||||
|
||||
/// TODO: поддержка идентификаторов вместо подзапросов.
|
||||
InterpreterSelectQuery interpreter(
|
||||
typeid_cast<ASTJoin &>(*select_query->join).subquery->children[0], subquery_context,
|
||||
typeid_cast<ASTJoin &>(*select_query->join).table->children[0], subquery_context,
|
||||
required_joined_columns,
|
||||
QueryProcessingStage::Complete, subquery_depth + 1);
|
||||
|
||||
@ -1304,7 +1273,7 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
step.required_output.push_back(select_query->where_expression->getColumnName());
|
||||
getRootActionsImpl(select_query->where_expression, only_types, false, step.actions);
|
||||
getRootActions(select_query->where_expression, only_types, false, step.actions);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -1323,7 +1292,7 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
{
|
||||
step.required_output.push_back(asts[i]->getColumnName());
|
||||
getRootActionsImpl(asts[i], only_types, false, step.actions);
|
||||
getRootActions(asts[i], only_types, false, step.actions);
|
||||
}
|
||||
|
||||
return true;
|
||||
@ -1344,13 +1313,13 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai
|
||||
}
|
||||
}
|
||||
|
||||
getActionsBeforeAggregationImpl(select_query->select_expression_list, step.actions, only_types);
|
||||
getActionsBeforeAggregation(select_query->select_expression_list, step.actions, only_types);
|
||||
|
||||
if (select_query->having_expression)
|
||||
getActionsBeforeAggregationImpl(select_query->having_expression, step.actions, only_types);
|
||||
getActionsBeforeAggregation(select_query->having_expression, step.actions, only_types);
|
||||
|
||||
if (select_query->order_expression_list)
|
||||
getActionsBeforeAggregationImpl(select_query->order_expression_list, step.actions, only_types);
|
||||
getActionsBeforeAggregation(select_query->order_expression_list, step.actions, only_types);
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
||||
@ -1364,7 +1333,7 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
step.required_output.push_back(select_query->having_expression->getColumnName());
|
||||
getRootActionsImpl(select_query->having_expression, only_types, false, step.actions);
|
||||
getRootActions(select_query->having_expression, only_types, false, step.actions);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -1376,7 +1345,7 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_
|
||||
initChain(chain, aggregated_columns);
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
getRootActionsImpl(select_query->select_expression_list, only_types, false, step.actions);
|
||||
getRootActions(select_query->select_expression_list, only_types, false, step.actions);
|
||||
|
||||
ASTs asts = select_query->select_expression_list->children;
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
@ -1395,7 +1364,7 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
|
||||
initChain(chain, aggregated_columns);
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
getRootActionsImpl(select_query->order_expression_list, only_types, false, step.actions);
|
||||
getRootActions(select_query->order_expression_list, only_types, false, step.actions);
|
||||
|
||||
ASTs asts = select_query->order_expression_list->children;
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
@ -1422,7 +1391,7 @@ void ExpressionAnalyzer::appendProjectResult(DB::ExpressionActionsChain & chain,
|
||||
ASTs asts = select_query->select_expression_list->children;
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
{
|
||||
result_columns.push_back(NameWithAlias(asts[i]->getColumnName(), asts[i]->getAlias()));
|
||||
result_columns.push_back(NameWithAlias(asts[i]->getColumnName(), asts[i]->getAliasOrColumnName()));
|
||||
step.required_output.push_back(result_columns.back().second);
|
||||
}
|
||||
|
||||
@ -1454,8 +1423,8 @@ Block ExpressionAnalyzer::getSelectSampleBlock()
|
||||
ASTs asts = select_query->select_expression_list->children;
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
{
|
||||
result_columns.push_back(NameWithAlias(asts[i]->getColumnName(), asts[i]->getAlias()));
|
||||
getRootActionsImpl(asts[i], true, false, temp_actions);
|
||||
result_columns.push_back(NameWithAlias(asts[i]->getColumnName(), asts[i]->getAliasOrColumnName()));
|
||||
getRootActions(asts[i], true, false, temp_actions);
|
||||
}
|
||||
|
||||
temp_actions->add(ExpressionAction::project(result_columns));
|
||||
@ -1463,25 +1432,16 @@ Block ExpressionAnalyzer::getSelectSampleBlock()
|
||||
return temp_actions->getSampleBlock();
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::getActionsBeforeAggregationImpl(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries)
|
||||
void ExpressionAnalyzer::getActionsBeforeAggregation(ASTPtr ast, ExpressionActionsPtr & actions, bool no_subqueries)
|
||||
{
|
||||
ASTFunction * node = typeid_cast<ASTFunction *>(&*ast);
|
||||
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
||||
{
|
||||
ASTs & arguments = node->arguments->children;
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
getRootActionsImpl(arguments[i], no_subqueries, false, actions);
|
||||
}
|
||||
}
|
||||
if (node && node->kind == ASTFunction::AGGREGATE_FUNCTION)
|
||||
for (auto & argument : node->arguments->children)
|
||||
getRootActions(argument, no_subqueries, false, actions);
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < ast->children.size(); ++i)
|
||||
{
|
||||
getActionsBeforeAggregationImpl(ast->children[i], actions, no_subqueries);
|
||||
}
|
||||
}
|
||||
for (auto & child : ast->children)
|
||||
getActionsBeforeAggregation(child, actions, no_subqueries);
|
||||
}
|
||||
|
||||
|
||||
@ -1503,12 +1463,12 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result)
|
||||
std::string name = asts[i]->getColumnName();
|
||||
std::string alias;
|
||||
if (project_result)
|
||||
alias = asts[i]->getAlias();
|
||||
alias = asts[i]->getAliasOrColumnName();
|
||||
else
|
||||
alias = name;
|
||||
result_columns.push_back(NameWithAlias(name, alias));
|
||||
result_names.push_back(alias);
|
||||
getRootActionsImpl(asts[i], false, false, actions);
|
||||
getRootActions(asts[i], false, false, actions);
|
||||
}
|
||||
|
||||
if (project_result)
|
||||
@ -1532,7 +1492,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
|
||||
{
|
||||
ExpressionActionsPtr actions = new ExpressionActions(NamesAndTypesList(), settings);
|
||||
|
||||
getRootActionsImpl(ast, true, true, actions);
|
||||
getRootActions(ast, true, true, actions);
|
||||
|
||||
return actions;
|
||||
}
|
||||
@ -1572,7 +1532,7 @@ void ExpressionAnalyzer::removeUnusedColumns()
|
||||
getRequiredColumnsImpl(expressions[i], required, empty, empty, empty);
|
||||
}
|
||||
|
||||
ignored.insert(expressions[i]->getAlias());
|
||||
ignored.insert(expressions[i]->getAliasOrColumnName());
|
||||
}
|
||||
}
|
||||
|
||||
@ -1648,7 +1608,7 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
|
||||
|
||||
auto & node = typeid_cast<ASTJoin &>(*select_query->join);
|
||||
auto & keys = typeid_cast<ASTExpressionList &>(*node.using_expr_list);
|
||||
auto & select = node.subquery->children[0];
|
||||
auto & table = node.table->children[0]; /// TODO: поддержка идентификаторов.
|
||||
|
||||
size_t num_join_keys = keys.children.size();
|
||||
|
||||
@ -1657,11 +1617,11 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
|
||||
if (!join_key_names_left_set.insert(keys.children[i]->getColumnName()).second)
|
||||
throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN);
|
||||
|
||||
if (!join_key_names_right_set.insert(keys.children[i]->getAlias()).second)
|
||||
if (!join_key_names_right_set.insert(keys.children[i]->getAliasOrColumnName()).second)
|
||||
throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
|
||||
Block nested_result_sample = ExpressionAnalyzer(select, context, subquery_depth + 1).getSelectSampleBlock();
|
||||
Block nested_result_sample = ExpressionAnalyzer(table, context, subquery_depth + 1).getSelectSampleBlock();
|
||||
|
||||
size_t nested_result_columns = nested_result_sample.columns();
|
||||
for (size_t i = 0; i < nested_result_columns; ++i)
|
||||
|
@ -27,6 +27,7 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expecte
|
||||
|
||||
ParserNotEmptyExpressionList exp_list;
|
||||
ParserSubquery subquery;
|
||||
ParserIdentifier identifier;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
@ -66,7 +67,8 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expecte
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!subquery.parse(pos, end, join->subquery, expected))
|
||||
if (!identifier.parse(pos, end, join->table, expected)
|
||||
&& !subquery.parse(pos, end, join->table, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
@ -81,7 +83,7 @@ bool ParserJoin::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expecte
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
join->children.push_back(join->subquery);
|
||||
join->children.push_back(join->table);
|
||||
join->children.push_back(join->using_expr_list);
|
||||
|
||||
return true;
|
||||
|
@ -754,7 +754,7 @@ void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent, bool hilite
|
||||
<< "JOIN"
|
||||
<< (hilite ? hilite_none : "");
|
||||
|
||||
formatAST(*ast.subquery, s, indent, hilite, one_line, need_parens);
|
||||
formatAST(*ast.table, s, indent, hilite, one_line, need_parens);
|
||||
|
||||
s << (hilite ? hilite_keyword : "") << " USING " << (hilite ? hilite_none : "");
|
||||
|
||||
|
3
dbms/tests/queries/0_stateless/00058_global_in.reference
Normal file
3
dbms/tests/queries/0_stateless/00058_global_in.reference
Normal file
@ -0,0 +1,3 @@
|
||||
6
|
||||
10
|
||||
10
|
95
dbms/tests/queries/0_stateless/00058_global_in.sh
Executable file
95
dbms/tests/queries/0_stateless/00058_global_in.sh
Executable file
@ -0,0 +1,95 @@
|
||||
#!/bin/bash -e
|
||||
|
||||
# Скрипт довольно хрупкий.
|
||||
|
||||
# Попробуем угадать, где расположен конфиг сервера.
|
||||
[ -f '/etc/clickhouse-server/config-preprocessed.xml' ] && CONFIG='/etc/clickhouse-server/config-preprocessed.xml' || CONFIG='../src/Server/config-preprocessed.xml'
|
||||
|
||||
if [ ! -f "$CONFIG" ]; then
|
||||
echo "Cannot find config file for clickhouse-server" >&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Создадим директории для данных второго сервера.
|
||||
PATH2=/tmp/clickhouse/
|
||||
mkdir -p ${PATH2}{data,metadata}/default/
|
||||
|
||||
# Создадим второй конфиг с портом 9001.
|
||||
CONFIG2="config-9001.xml"
|
||||
|
||||
LOG=${PATH2}log
|
||||
|
||||
cat "$CONFIG" | sed -r \
|
||||
's/<path>.+<\/path>/<path>'${PATH2//\//\\/}'<\/path>/;
|
||||
s/<http_port>[0-9]+/<http_port>8124/;
|
||||
s/<tcp_port>[0-9]+/<tcp_port>9001/;
|
||||
s/<interserver_http_port>[0-9]+/<interserver_http_port>9010/;
|
||||
s/<use_olap_http_server>true/<use_olap_http_server>false/' > $CONFIG2
|
||||
|
||||
cp ${CONFIG/config/users} .
|
||||
|
||||
# Запустим второй сервер.
|
||||
BINARY=$(readlink /proc/$(pidof clickhouse-server | tr ' ' '\n' | head -n1)/exe || echo "/usr/bin/clickhouse-server")
|
||||
|
||||
$BINARY --config-file=${CONFIG2} 2>$LOG &
|
||||
PID=$!
|
||||
|
||||
function finish {
|
||||
kill $PID
|
||||
}
|
||||
|
||||
trap finish EXIT
|
||||
|
||||
i=0
|
||||
while true; do
|
||||
grep -q 'Ready for connections' ${LOG} && break
|
||||
grep -q 'shutting down' ${LOG} && echo "Cannot start second clickhouse-server" && exit 1
|
||||
sleep 0.05
|
||||
|
||||
i=$(($i + 1))
|
||||
[[ $i == 100 ]] && echo "Cannot start second clickhouse-server" && exit 1
|
||||
done
|
||||
|
||||
rm "$CONFIG2"
|
||||
|
||||
|
||||
# Теперь можно выполнять запросы.
|
||||
|
||||
CLIENT1='clickhouse-client --port=9000'
|
||||
CLIENT2='clickhouse-client --port=9001'
|
||||
|
||||
|
||||
$CLIENT1 -n --query="
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.half1;
|
||||
DROP TABLE IF EXISTS test.half2;
|
||||
CREATE TABLE test.half1 ENGINE = Memory AS SELECT number FROM system.numbers LIMIT 5;
|
||||
CREATE TABLE test.half2 ENGINE = Memory AS SELECT number FROM system.numbers WHERE number % 2 = 0 LIMIT 5;
|
||||
"
|
||||
|
||||
$CLIENT2 -n --query="
|
||||
CREATE DATABASE IF NOT EXISTS test;
|
||||
DROP TABLE IF EXISTS test.half1;
|
||||
DROP TABLE IF EXISTS test.half2;
|
||||
CREATE TABLE test.half1 ENGINE = Memory AS SELECT number FROM system.numbers LIMIT 5, 5;
|
||||
CREATE TABLE test.half2 ENGINE = Memory AS SELECT number FROM system.numbers WHERE number % 2 = 1 LIMIT 5;
|
||||
"
|
||||
|
||||
$CLIENT1 -n --query="
|
||||
SELECT count() FROM remote('localhost:{9000,9001}', test, half1)
|
||||
WHERE number IN (SELECT * FROM test.half2);
|
||||
|
||||
SELECT count() FROM remote('localhost:{9000,9001}', test, half1)
|
||||
WHERE number IN (SELECT * FROM remote('localhost:{9000,9001}', test, half2));
|
||||
|
||||
SELECT count() FROM remote('localhost:{9000,9001}', test, half1)
|
||||
WHERE number GLOBAL IN (SELECT * FROM remote('localhost:{9000,9001}', test, half2));
|
||||
"
|
||||
|
||||
$CLIENT1 -n --query="
|
||||
DROP TABLE test.half1;
|
||||
DROP TABLE test.half2;"
|
||||
|
||||
$CLIENT2 -n --query="
|
||||
DROP TABLE test.half1;
|
||||
DROP TABLE test.half2;"
|
Loading…
Reference in New Issue
Block a user