dbms: development [#CONV-2944].

This commit is contained in:
Alexey Milovidov 2011-09-24 20:32:41 +00:00
parent 5c7ea6af1f
commit 5c8b01da3a
14 changed files with 216 additions and 192 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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_) {}

View File

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

View File

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

View File

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

View File

@ -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,14 +14,14 @@ 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();
}
void Expression::addSemantic(ASTPtr & ast)
{
@ -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);
}
}

View File

@ -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))
{