mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
dbms: development [#CONV-2944].
This commit is contained in:
parent
5c7ea6af1f
commit
5c8b01da3a
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Interpreters/Aggregator.h>
|
||||
#include <DB/Interpreters/Expression.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
@ -19,19 +20,23 @@ class AggregatingBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
AggregatingBlockInputStream(BlockInputStreamPtr input_, const ColumnNumbers & keys_, AggregateDescriptions & aggregates_)
|
||||
: input(input_), keys(keys_), aggregates(aggregates_), aggregator(keys_, aggregates_), has_been_read(false)
|
||||
: input(input_), aggregator(keys_, aggregates_), has_been_read(false)
|
||||
{
|
||||
children.push_back(input);
|
||||
}
|
||||
|
||||
/** keys берутся из Expression::PART_GROUP
|
||||
* Агрегатные функции ищутся везде в выражении.
|
||||
* Столбцы, соответствующие keys и аргументам агрегатных функций, уже должны быть вычислены.
|
||||
*/
|
||||
// AggregatingBlockInputStream(BlockInputStreamPtr input_, SharedPtr<Expression> expression);
|
||||
|
||||
Block readImpl();
|
||||
|
||||
String getName() const { return "AggregatingBlockInputStream"; }
|
||||
|
||||
private:
|
||||
BlockInputStreamPtr input;
|
||||
const ColumnNumbers keys;
|
||||
AggregateDescriptions aggregates;
|
||||
Aggregator aggregator;
|
||||
bool has_been_read;
|
||||
};
|
||||
|
@ -235,34 +235,34 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename T0, typename T1>
|
||||
bool checkRightType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
bool checkRightType(const DataTypes & arguments, DataTypePtr & type_res) const
|
||||
{
|
||||
if (dynamic_cast<const T1 *>(&*arguments[1]))
|
||||
{
|
||||
types_res.push_back(new typename DataTypeFromFieldType<
|
||||
typename Impl<typename T0::FieldType, typename T1::FieldType>::ResultType>::Type);
|
||||
type_res = new typename DataTypeFromFieldType<
|
||||
typename Impl<typename T0::FieldType, typename T1::FieldType>::ResultType>::Type;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool checkLeftType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
bool checkLeftType(const DataTypes & arguments, DataTypePtr & type_res) const
|
||||
{
|
||||
if (dynamic_cast<const T0 *>(&*arguments[0]))
|
||||
{
|
||||
if ( checkRightType<T0, DataTypeUInt8>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeUInt16>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeUInt32>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeUInt64>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt8>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt16>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt32>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeInt64>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeFloat32>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeFloat64>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeVarUInt>(arguments, types_res)
|
||||
|| checkRightType<T0, DataTypeVarInt>(arguments, types_res))
|
||||
if ( checkRightType<T0, DataTypeUInt8>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt16>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt32>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeUInt64>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt8>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt16>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt32>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeInt64>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeFloat32>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeFloat64>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeVarUInt>(arguments, type_res)
|
||||
|| checkRightType<T0, DataTypeVarInt>(arguments, type_res))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
@ -273,14 +273,14 @@ private:
|
||||
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnVector<T0> * col_left)
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
@ -293,7 +293,7 @@ private:
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
@ -306,14 +306,14 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnConst<T0> * col_left)
|
||||
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->size());
|
||||
@ -329,7 +329,7 @@ private:
|
||||
Impl<T0, T1>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
|
||||
ColumnConst<ResultType> * col_res = new ColumnConst<ResultType>(col_left->size(), res);
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -338,7 +338,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -388,40 +388,36 @@ public:
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(const DataTypes & arguments) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
DataTypes types_res;
|
||||
DataTypePtr type_res;
|
||||
|
||||
if (!( checkLeftType<DataTypeUInt8>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeUInt16>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeUInt32>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeUInt64>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt8>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt16>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt32>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeInt64>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeFloat32>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeFloat64>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeVarUInt>(arguments, types_res)
|
||||
|| checkLeftType<DataTypeVarInt>(arguments, types_res)))
|
||||
if (!( checkLeftType<DataTypeUInt8>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt16>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt32>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeUInt64>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt8>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt16>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt32>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeInt64>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeFloat32>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeFloat64>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeVarUInt>(arguments, type_res)
|
||||
|| checkLeftType<DataTypeVarInt>(arguments, type_res)))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return types_res;
|
||||
return type_res;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
if (!( executeLeftType<UInt8>(block, arguments, result)
|
||||
|| executeLeftType<UInt16>(block, arguments, result)
|
||||
|| executeLeftType<UInt32>(block, arguments, result)
|
||||
@ -444,26 +440,26 @@ class FunctionUnaryArithmetic : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename T0>
|
||||
bool checkType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
bool checkType(const DataTypes & arguments, DataTypePtr result) const
|
||||
{
|
||||
if (dynamic_cast<const T0 *>(&*arguments[0]))
|
||||
{
|
||||
types_res.push_back(new typename DataTypeFromFieldType<
|
||||
typename Impl<typename T0::FieldType>::ResultType>::Type);
|
||||
result = new typename DataTypeFromFieldType<
|
||||
typename Impl<typename T0::FieldType>::ResultType>::Type;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
typedef typename Impl<T0>::ResultType ResultType;
|
||||
|
||||
ColumnVector<ResultType> * col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col->getData().size());
|
||||
@ -479,7 +475,7 @@ private:
|
||||
Impl<T0>::constant(col->getData(), res);
|
||||
|
||||
ColumnConst<ResultType> * col_res = new ColumnConst<ResultType>(col->size(), res);
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -495,40 +491,36 @@ public:
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(const DataTypes & arguments) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
DataTypes types_res;
|
||||
DataTypePtr result;
|
||||
|
||||
if (!( checkType<DataTypeUInt8>(arguments, types_res)
|
||||
|| checkType<DataTypeUInt16>(arguments, types_res)
|
||||
|| checkType<DataTypeUInt32>(arguments, types_res)
|
||||
|| checkType<DataTypeUInt64>(arguments, types_res)
|
||||
|| checkType<DataTypeInt8>(arguments, types_res)
|
||||
|| checkType<DataTypeInt16>(arguments, types_res)
|
||||
|| checkType<DataTypeInt32>(arguments, types_res)
|
||||
|| checkType<DataTypeInt64>(arguments, types_res)
|
||||
|| checkType<DataTypeFloat32>(arguments, types_res)
|
||||
|| checkType<DataTypeFloat64>(arguments, types_res)
|
||||
|| checkType<DataTypeVarUInt>(arguments, types_res)
|
||||
|| checkType<DataTypeVarInt>(arguments, types_res)))
|
||||
if (!( checkType<DataTypeUInt8>(arguments, result)
|
||||
|| checkType<DataTypeUInt16>(arguments, result)
|
||||
|| checkType<DataTypeUInt32>(arguments, result)
|
||||
|| checkType<DataTypeUInt64>(arguments, result)
|
||||
|| checkType<DataTypeInt8>(arguments, result)
|
||||
|| checkType<DataTypeInt16>(arguments, result)
|
||||
|| checkType<DataTypeInt32>(arguments, result)
|
||||
|| checkType<DataTypeInt64>(arguments, result)
|
||||
|| checkType<DataTypeFloat32>(arguments, result)
|
||||
|| checkType<DataTypeFloat64>(arguments, result)
|
||||
|| checkType<DataTypeVarUInt>(arguments, result)
|
||||
|| checkType<DataTypeVarInt>(arguments, result)))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return types_res;
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
|| executeType<UInt16>(block, arguments, result)
|
||||
|| executeType<UInt32>(block, arguments, result)
|
||||
|
@ -1130,12 +1130,12 @@ class FunctionComparison : public IFunction
|
||||
private:
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnVector<T0> * col_left)
|
||||
bool executeNumRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnUInt8::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
@ -1146,7 +1146,7 @@ private:
|
||||
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnUInt8::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
@ -1159,12 +1159,12 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeNumConstRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnConst<T0> * col_left)
|
||||
bool executeNumConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
ColumnUInt8 * col_res = new ColumnUInt8;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnUInt8::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->size());
|
||||
@ -1178,7 +1178,7 @@ private:
|
||||
NumImpl<T0, T1>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
|
||||
ColumnConstUInt8 * col_res = new ColumnConstUInt8(col_left->size(), res);
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -1187,7 +1187,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeNumLeftType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
bool executeNumLeftType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -1229,7 +1229,7 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeString(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
void executeString(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
IColumn * c0 = &*block.getByPosition(arguments[0]).column;
|
||||
IColumn * c1 = &*block.getByPosition(arguments[1]).column;
|
||||
@ -1244,13 +1244,13 @@ private:
|
||||
if (c0_const && c1_const)
|
||||
{
|
||||
ColumnConstUInt8 * c_res = new ColumnConstUInt8(c0_const->size(), 0);
|
||||
block.getByPosition(result[0]).column = c_res;
|
||||
block.getByPosition(result).column = c_res;
|
||||
StringImpl::constant_constant(c0_const->getData(), c1_const->getData(), c_res->getData());
|
||||
}
|
||||
else
|
||||
{
|
||||
ColumnUInt8 * c_res = new ColumnUInt8;
|
||||
block.getByPosition(result[0]).column = c_res;
|
||||
block.getByPosition(result).column = c_res;
|
||||
ColumnUInt8::Container_t & vec_res = c_res->getData();
|
||||
vec_res.resize(c0->size());
|
||||
|
||||
@ -1311,7 +1311,7 @@ public:
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(const DataTypes & arguments) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -1326,18 +1326,12 @@ public:
|
||||
throw Exception("Illegal types of arguments (" + arguments[0]->getName() + ", " + arguments[1]->getName() + ")"
|
||||
" of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
DataTypes types_res;
|
||||
types_res.push_back(new DataTypeUInt8);
|
||||
return types_res;
|
||||
return new DataTypeUInt8;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
if (block.getByPosition(arguments[0]).column->isNumeric())
|
||||
{
|
||||
if (!( executeNumLeftType<UInt8>(block, arguments, result)
|
||||
|
@ -127,12 +127,12 @@ class FunctionBinaryLogical : public IFunction
|
||||
private:
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnVector<T0> * col_left)
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
@ -143,7 +143,7 @@ private:
|
||||
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
@ -156,12 +156,12 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result, const ColumnConst<T0> * col_left)
|
||||
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||
{
|
||||
if (ColumnVector<T1> * col_right = dynamic_cast<ColumnVector<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->size());
|
||||
@ -175,7 +175,7 @@ private:
|
||||
Impl<T0, T1>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
|
||||
ColumnConst<UInt8> * col_res = new ColumnConst<UInt8>(col_left->size(), res);
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -184,7 +184,7 @@ private:
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -234,7 +234,7 @@ public:
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(const DataTypes & arguments) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -247,18 +247,12 @@ public:
|
||||
+ ") of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
DataTypes types_res;
|
||||
types_res.push_back(new DataTypeUInt8);
|
||||
return types_res;
|
||||
return new DataTypeUInt8;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
if (!( executeLeftType<UInt8>(block, arguments, result)
|
||||
|| executeLeftType<UInt16>(block, arguments, result)
|
||||
|| executeLeftType<UInt32>(block, arguments, result)
|
||||
@ -282,12 +276,12 @@ class FunctionUnaryLogical : public IFunction
|
||||
private:
|
||||
|
||||
template <typename T>
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T> * col = dynamic_cast<ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<UInt8> * col_res = new ColumnVector<UInt8>;
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
typename ColumnVector<UInt8>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col->getData().size());
|
||||
@ -301,7 +295,7 @@ private:
|
||||
Impl<T>::constant(col->getData(), res);
|
||||
|
||||
ColumnConst<UInt8> * col_res = new ColumnConst<UInt8>(col->size(), res);
|
||||
block.getByPosition(result[0]).column = col_res;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -317,7 +311,7 @@ public:
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(const DataTypes & arguments) const
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -330,18 +324,12 @@ public:
|
||||
+ ") of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
DataTypes types_res;
|
||||
types_res.push_back(new DataTypeUInt8);
|
||||
return types_res;
|
||||
return new DataTypeUInt8;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
if (!( executeType<UInt8>(block, arguments, result)
|
||||
|| executeType<UInt16>(block, arguments, result)
|
||||
|| executeType<UInt32>(block, arguments, result)
|
||||
|
@ -30,11 +30,11 @@ public:
|
||||
/// Получить основное имя функции.
|
||||
virtual String getName() const = 0;
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
virtual DataTypes getReturnTypes(const DataTypes & arguments) const = 0;
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
virtual DataTypePtr getReturnType(const DataTypes & arguments) const = 0;
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
virtual void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result) = 0;
|
||||
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result) = 0;
|
||||
};
|
||||
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/ColumnNumbers.h>
|
||||
#include <DB/Core/Names.h>
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
#include <DB/AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
@ -13,6 +14,7 @@ struct AggregateDescription
|
||||
{
|
||||
AggregateFunctionPtr function;
|
||||
ColumnNumbers arguments;
|
||||
Names argument_names; /// Используются, если arguments не заданы.
|
||||
};
|
||||
|
||||
typedef std::vector<AggregateDescription> AggregateDescriptions;
|
||||
@ -26,6 +28,7 @@ class Aggregator
|
||||
{
|
||||
public:
|
||||
Aggregator(const ColumnNumbers & keys_, AggregateDescriptions & aggregates_) : keys(keys_), aggregates(aggregates_) {};
|
||||
Aggregator(const Names & key_names_, AggregateDescriptions & aggregates_) : key_names(key_names_), aggregates(aggregates_) {};
|
||||
|
||||
AggregatedData execute(BlockInputStreamPtr stream);
|
||||
|
||||
@ -34,6 +37,7 @@ public:
|
||||
|
||||
private:
|
||||
ColumnNumbers keys;
|
||||
Names key_names;
|
||||
AggregateDescriptions aggregates;
|
||||
|
||||
Block sample;
|
||||
|
@ -6,6 +6,7 @@
|
||||
|
||||
#include <DB/Parsers/IAST.h>
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Interpreters/Aggregator.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -43,6 +44,10 @@ public:
|
||||
*/
|
||||
DataTypes getReturnTypes();
|
||||
|
||||
/** Получить список ключей агрегирования и описаний агрегатных функций, если в запросе есть GROUP BY.
|
||||
*/
|
||||
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates);
|
||||
|
||||
private:
|
||||
ASTPtr ast;
|
||||
const Context & context;
|
||||
@ -82,6 +87,8 @@ private:
|
||||
void collectFinalColumns(ASTPtr ast, Block & src, Block & dst, bool without_duplicates, unsigned part_id);
|
||||
|
||||
void getReturnTypesImpl(ASTPtr ast, DataTypes & res);
|
||||
|
||||
void getAggregateInfoImpl(ASTPtr ast, Names & key_names, AggregateDescriptions & aggregates);
|
||||
};
|
||||
|
||||
|
||||
|
@ -32,10 +32,11 @@ private:
|
||||
PART_OTHER = 1,
|
||||
PART_SELECT = 2,
|
||||
PART_WHERE = 4,
|
||||
PART_HAVING = 8,
|
||||
PART_ORDER = 16,
|
||||
PART_BELOW_AGGREGATE_FUNCTIONS = 32,
|
||||
PART_ABOVE_AGGREGATE_FUNCTIONS = 64,
|
||||
PART_GROUP = 8,
|
||||
PART_HAVING = 16,
|
||||
PART_ORDER = 32,
|
||||
PART_BELOW_AGGREGATE_FUNCTIONS = 64,
|
||||
PART_ABOVE_AGGREGATE_FUNCTIONS = 128,
|
||||
};
|
||||
|
||||
|
||||
|
@ -22,10 +22,10 @@ public:
|
||||
FunctionPtr function;
|
||||
/// или агрегатная функция
|
||||
AggregateFunctionPtr aggregate_function;
|
||||
/// типы возвращаемых значений
|
||||
DataTypes return_types;
|
||||
/// номера столбцов возвращаемых значений
|
||||
ColumnNumbers return_column_numbers;
|
||||
/// тип возвращаемого значения
|
||||
DataTypePtr return_type;
|
||||
/// номер столбца возвращаемого значения
|
||||
size_t return_column_number;
|
||||
|
||||
ASTFunction() {}
|
||||
ASTFunction(StringRange range_) : IAST(range_) {}
|
||||
|
@ -5,6 +5,16 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/*AggregatingBlockInputStream::AggregatingBlockInputStream(BlockInputStreamPtr input_, SharedPtr<Expression> expression)
|
||||
: input(input_), has_been_read(false)
|
||||
{
|
||||
children.push_back(input);
|
||||
|
||||
|
||||
}*/
|
||||
|
||||
|
||||
|
||||
Block AggregatingBlockInputStream::readImpl()
|
||||
{
|
||||
if (has_been_read)
|
||||
|
@ -45,25 +45,19 @@ int main(int argc, char ** argv)
|
||||
arg_nums.push_back(0);
|
||||
arg_nums.push_back(1);
|
||||
|
||||
DB::ColumnNumbers res_nums;
|
||||
res_nums.push_back(2);
|
||||
size_t res_num = 2;
|
||||
|
||||
DB::DataTypes res_types = f.getReturnTypes(arg_types);
|
||||
DB::DataTypePtr res_type = f.getReturnType(arg_types);
|
||||
|
||||
for (DB::DataTypes::const_iterator it = res_types.begin(); it != res_types.end(); ++it)
|
||||
{
|
||||
DB::ColumnWithNameAndType descr_res;
|
||||
descr_res.type = *it;
|
||||
descr_res.name = "z";
|
||||
|
||||
block.insert(descr_res);
|
||||
}
|
||||
DB::ColumnWithNameAndType descr_res;
|
||||
descr_res.type = res_type;
|
||||
descr_res.name = "z";
|
||||
|
||||
{
|
||||
Poco::Stopwatch stopwatch;
|
||||
stopwatch.start();
|
||||
|
||||
f.execute(block, arg_nums, res_nums);
|
||||
f.execute(block, arg_nums, res_num);
|
||||
|
||||
stopwatch.stop();
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
|
@ -37,6 +37,16 @@ AggregatedData Aggregator::execute(BlockInputStreamPtr stream)
|
||||
/// Читаем все данные
|
||||
while (Block block = stream->read())
|
||||
{
|
||||
/// Преобразуем имена столбцов в номера, если номера не заданы
|
||||
if (keys.empty() && !key_names.empty())
|
||||
for (Names::const_iterator it = key_names.begin(); it != key_names.end(); ++it)
|
||||
keys.push_back(block.getPositionByName(*it));
|
||||
|
||||
for (AggregateDescriptions::iterator it = aggregates.begin(); it != aggregates.end(); ++it)
|
||||
if (it->arguments.empty() && !it->argument_names.empty())
|
||||
for (Names::const_iterator jt = it->argument_names.begin(); jt != it->argument_names.end(); ++jt)
|
||||
it->arguments.push_back(block.getPositionByName(*jt));
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
for (size_t i = 0, size = keys_size; i < size; ++i)
|
||||
key_columns[i] = block.getByPosition(keys[i]).column;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DB/Parsers/ASTLiteral.h>
|
||||
#include <DB/Parsers/ASTAsterisk.h>
|
||||
#include <DB/Parsers/ASTExpressionList.h>
|
||||
#include <DB/Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <DB/Interpreters/Expression.h>
|
||||
|
||||
@ -13,12 +14,12 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Если функция возвращает много аргументов, то в имена всех соответствующих столбцов,
|
||||
* кроме первого, будем добавлять _1, _2 и т. п.
|
||||
*/
|
||||
static std::string functionReturnValueSuffix(size_t i)
|
||||
static std::string getName(ASTPtr & ast)
|
||||
{
|
||||
return i == 0 ? "" : ("_" + Poco::NumberFormatter::format(i));
|
||||
if (ASTIdentifier * ident = dynamic_cast<ASTIdentifier *>(&*ast))
|
||||
return ident->name;
|
||||
else
|
||||
return ast->getTreeID();
|
||||
}
|
||||
|
||||
|
||||
@ -78,7 +79,7 @@ void Expression::checkTypes(ASTPtr ast)
|
||||
for (ASTs::iterator it = arguments.begin(); it != arguments.end(); ++it)
|
||||
{
|
||||
if (ASTFunction * arg = dynamic_cast<ASTFunction *>(&**it))
|
||||
argument_types.insert(argument_types.end(), arg->return_types.begin(), arg->return_types.end());
|
||||
argument_types.push_back(arg->return_type);
|
||||
else if (ASTIdentifier * arg = dynamic_cast<ASTIdentifier *>(&**it))
|
||||
argument_types.push_back(arg->type);
|
||||
else if (ASTLiteral * arg = dynamic_cast<ASTLiteral *>(&**it))
|
||||
@ -89,10 +90,10 @@ void Expression::checkTypes(ASTPtr ast)
|
||||
if (node->aggregate_function)
|
||||
{
|
||||
node->aggregate_function->setArguments(argument_types);
|
||||
node->return_types.push_back(node->aggregate_function->getReturnType());
|
||||
node->return_type = node->aggregate_function->getReturnType();
|
||||
}
|
||||
else
|
||||
node->return_types = node->function->getReturnTypes(argument_types);
|
||||
node->return_type = node->function->getReturnType(argument_types);
|
||||
}
|
||||
}
|
||||
|
||||
@ -178,33 +179,19 @@ void Expression::executeImpl(ASTPtr ast, Block & block, unsigned part_id)
|
||||
{
|
||||
/// Вставляем в блок столбцы - результаты вычисления функции
|
||||
ColumnNumbers argument_numbers;
|
||||
ColumnNumbers & result_numbers = node->return_column_numbers;
|
||||
result_numbers.clear();
|
||||
|
||||
size_t res_num = 0;
|
||||
for (DataTypes::const_iterator it = node->return_types.begin(); it != node->return_types.end(); ++it)
|
||||
{
|
||||
ColumnWithNameAndType column;
|
||||
column.type = *it;
|
||||
column.name = node->getTreeID() + functionReturnValueSuffix(res_num);
|
||||
ColumnWithNameAndType column;
|
||||
column.type = node->return_type;
|
||||
column.name = getName(ast);
|
||||
|
||||
result_numbers.push_back(block.columns());
|
||||
block.insert(column);
|
||||
++res_num;
|
||||
}
|
||||
size_t result_number = block.columns();
|
||||
block.insert(column);
|
||||
|
||||
ASTs arguments = node->arguments->children;
|
||||
for (ASTs::iterator it = arguments.begin(); it != arguments.end(); ++it)
|
||||
{
|
||||
if (ASTIdentifier * ident = dynamic_cast<ASTIdentifier *>(&**it))
|
||||
argument_numbers.push_back(block.getPositionByName(ident->name));
|
||||
else if (ASTFunction * func = dynamic_cast<ASTFunction *>(&**it))
|
||||
argument_numbers.insert(argument_numbers.end(), func->return_column_numbers.begin(), func->return_column_numbers.end());
|
||||
else
|
||||
argument_numbers.push_back(block.getPositionByName((*it)->getTreeID()));
|
||||
}
|
||||
argument_numbers.push_back(block.getPositionByName(getName(*it)));
|
||||
|
||||
node->function->execute(block, argument_numbers, result_numbers);
|
||||
node->function->execute(block, argument_numbers, result_number);
|
||||
}
|
||||
}
|
||||
else if (ASTLiteral * node = dynamic_cast<ASTLiteral *>(&*ast))
|
||||
@ -212,7 +199,7 @@ void Expression::executeImpl(ASTPtr ast, Block & block, unsigned part_id)
|
||||
ColumnWithNameAndType column;
|
||||
column.column = node->type->createConstColumn(block.rows(), node->value);
|
||||
column.type = node->type;
|
||||
column.name = node->getTreeID();
|
||||
column.name = getName(ast);
|
||||
|
||||
block.insert(column);
|
||||
}
|
||||
@ -243,17 +230,10 @@ void Expression::collectFinalColumns(ASTPtr ast, Block & src, Block & dst, bool
|
||||
if (ASTIdentifier * ident = dynamic_cast<ASTIdentifier *>(&*ast))
|
||||
{
|
||||
if (ident->kind == ASTIdentifier::Column)
|
||||
without_duplicates ? dst.insertUnique(src.getByName(ident->name)) : dst.insert(src.getByName(ident->name));
|
||||
}
|
||||
else if (dynamic_cast<ASTLiteral *>(&*ast))
|
||||
without_duplicates ? dst.insertUnique(src.getByName(ast->getTreeID())) : dst.insert(src.getByName(ast->getTreeID()));
|
||||
else if (ASTFunction * func = dynamic_cast<ASTFunction *>(&*ast))
|
||||
{
|
||||
for (size_t i = 0, size = func->return_types.size(); i != size; ++i)
|
||||
without_duplicates
|
||||
? dst.insertUnique(src.getByName(ast->getTreeID() + functionReturnValueSuffix(i)))
|
||||
: dst.insert(src.getByName(ast->getTreeID() + functionReturnValueSuffix(i)));
|
||||
without_duplicates ? dst.insertUnique(src.getByName(getName(ast))) : dst.insert(src.getByName(getName(ast)));
|
||||
}
|
||||
else if (dynamic_cast<ASTLiteral *>(&*ast) || dynamic_cast<ASTFunction *>(&*ast))
|
||||
without_duplicates ? dst.insertUnique(src.getByName(getName(ast))) : dst.insert(src.getByName(getName(ast)));
|
||||
else
|
||||
for (ASTs::iterator it = ast->children.begin(); it != ast->children.end(); ++it)
|
||||
collectFinalColumns(*it, src, dst, without_duplicates, part_id);
|
||||
@ -279,11 +259,56 @@ void Expression::getReturnTypesImpl(ASTPtr ast, DataTypes & res)
|
||||
else if (ASTLiteral * lit = dynamic_cast<ASTLiteral *>(&*ast))
|
||||
res.push_back(lit->type);
|
||||
else if (ASTFunction * func = dynamic_cast<ASTFunction *>(&*ast))
|
||||
res.insert(res.end(), func->return_types.begin(), func->return_types.end());
|
||||
res.push_back(func->return_type);
|
||||
else
|
||||
for (ASTs::iterator it = ast->children.begin(); it != ast->children.end(); ++it)
|
||||
getReturnTypesImpl(*it, res);
|
||||
}
|
||||
|
||||
|
||||
void Expression::getAggregateInfoImpl(ASTPtr ast, Names & key_names, AggregateDescriptions & aggregates)
|
||||
{
|
||||
/// Обход в глубину
|
||||
if (ASTSelectQuery * select = dynamic_cast<ASTSelectQuery *>(&*ast))
|
||||
{
|
||||
if (select->group_expression_list)
|
||||
{
|
||||
for (ASTs::iterator it = select->group_expression_list->children.begin(); it != select->group_expression_list->children.end(); ++it)
|
||||
{
|
||||
if (ASTIdentifier * ident = dynamic_cast<ASTIdentifier *>(&**it))
|
||||
{
|
||||
if (ident->kind == ASTIdentifier::Column)
|
||||
key_names.push_back(getName(*it));
|
||||
}
|
||||
else if (dynamic_cast<ASTLiteral *>(&**it) || dynamic_cast<ASTFunction *>(&**it))
|
||||
key_names.push_back(getName(*it));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (ASTFunction * func = dynamic_cast<ASTFunction *>(&*ast))
|
||||
{
|
||||
if (func->aggregate_function)
|
||||
{
|
||||
AggregateDescription desc;
|
||||
desc.function = func->aggregate_function;
|
||||
|
||||
for (ASTs::iterator it = func->arguments->children.begin(); it != func->arguments->children.end(); ++it)
|
||||
desc.argument_names.push_back(getName(*it));
|
||||
|
||||
aggregates.push_back(desc);
|
||||
}
|
||||
}
|
||||
|
||||
for (ASTs::iterator it = ast->children.begin(); it != ast->children.end(); ++it)
|
||||
getAggregateInfoImpl(*it, key_names, aggregates);
|
||||
}
|
||||
|
||||
|
||||
void Expression::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates)
|
||||
{
|
||||
getAggregateInfoImpl(ast, key_names, aggregates);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -28,14 +28,8 @@ void dump(DB::IAST & ast, int level = 0)
|
||||
|
||||
if (DB::ASTFunction * node = dynamic_cast<DB::ASTFunction *>(&ast))
|
||||
{
|
||||
std::cout << prefix << node << " Function, name = " << node->function->getName() << ", return types: ";
|
||||
for (DB::DataTypes::const_iterator it = node->return_types.begin(); it != node->return_types.end(); ++it)
|
||||
{
|
||||
if (it != node->return_types.begin())
|
||||
std::cout << ", ";
|
||||
std::cout << (*it)->getName();
|
||||
}
|
||||
std::cout << std::endl;
|
||||
std::cout << prefix << node << " Function, name = " << node->function->getName()
|
||||
<< ", return type: " << node->return_type->getName() << std::endl;
|
||||
}
|
||||
else if (DB::ASTIdentifier * node = dynamic_cast<DB::ASTIdentifier *>(&ast))
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user