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
e0ab9853d0
commit
f3c427ae63
@ -20,6 +20,8 @@ template <typename T>
|
||||
class ColumnConst : public IColumn
|
||||
{
|
||||
public:
|
||||
typedef T Type;
|
||||
|
||||
ColumnConst(size_t s_, const T & data_) : s(s_), data(data_) {}
|
||||
|
||||
ColumnPtr cloneEmpty() const { return new ColumnConst(0, data); }
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -22,4 +23,18 @@ typedef ColumnVector<Int64> ColumnInt64;
|
||||
typedef ColumnVector<Float32> ColumnFloat32;
|
||||
typedef ColumnVector<Float64> ColumnFloat64;
|
||||
|
||||
|
||||
typedef ColumnConst<UInt8> ColumnConstUInt8;
|
||||
typedef ColumnConst<UInt16> ColumnConstUInt16;
|
||||
typedef ColumnConst<UInt32> ColumnConstUInt32;
|
||||
typedef ColumnConst<UInt64> ColumnConstUInt64;
|
||||
|
||||
typedef ColumnConst<Int8> ColumnConstInt8;
|
||||
typedef ColumnConst<Int16> ColumnConstInt16;
|
||||
typedef ColumnConst<Int32> ColumnConstInt32;
|
||||
typedef ColumnConst<Int64> ColumnConstInt64;
|
||||
|
||||
typedef ColumnConst<Float32> ColumnConstFloat32;
|
||||
typedef ColumnConst<Float64> ColumnConstFloat64;
|
||||
|
||||
}
|
||||
|
@ -51,6 +51,8 @@ public:
|
||||
ColumnWithNameAndType & getByName(const std::string & name);
|
||||
const ColumnWithNameAndType & getByName(const std::string & name) const;
|
||||
|
||||
size_t getPositionByName(const std::string & name) const;
|
||||
|
||||
/** Возвращает количество строк в блоке.
|
||||
* Заодно проверяет, что все столбцы кроме констант (которые содержат единственное значение),
|
||||
* содержат одинаковое число значений.
|
||||
|
@ -36,7 +36,7 @@ struct PlusImpl
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a + b[i];
|
||||
}
|
||||
@ -68,7 +68,7 @@ struct MultiplyImpl
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a * b[i];
|
||||
}
|
||||
@ -100,7 +100,7 @@ struct MinusImpl
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a - b[i];
|
||||
}
|
||||
@ -132,7 +132,7 @@ struct DivideFloatingImpl
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = static_cast<ResultType>(a) / b[i];
|
||||
}
|
||||
@ -164,7 +164,7 @@ struct DivideIntegralImpl
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a / b[i];
|
||||
}
|
||||
@ -196,7 +196,7 @@ struct ModuloImpl
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
size_t size = b.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a % b[i];
|
||||
}
|
||||
@ -211,6 +211,148 @@ struct ModuloImpl
|
||||
template <template <typename, typename> class Impl, typename Name>
|
||||
class FunctionBinaryArithmetic : public IFunction
|
||||
{
|
||||
private:
|
||||
template <typename T0, typename T1>
|
||||
bool checkRightType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
{
|
||||
if (dynamic_cast<const typename DataTypeFromFieldType<T1>::Type *>(&*arguments[1]))
|
||||
{
|
||||
types_res.push_back(new typename DataTypeFromFieldType<typename Impl<T0, T1>::ResultType>::Type);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool checkLeftType(const DataTypes & arguments, DataTypes & types_res) const
|
||||
{
|
||||
if (dynamic_cast<const typename DataTypeFromFieldType<T0>::Type *>(&*arguments[0]))
|
||||
{
|
||||
if ( checkRightType<T0, UInt8>(arguments, types_res)
|
||||
|| checkRightType<T0, UInt16>(arguments, types_res)
|
||||
|| checkRightType<T0, UInt32>(arguments, types_res)
|
||||
|| checkRightType<T0, UInt64>(arguments, types_res)
|
||||
|| checkRightType<T0, Int8>(arguments, types_res)
|
||||
|| checkRightType<T0, Int16>(arguments, types_res)
|
||||
|| checkRightType<T0, Int32>(arguments, types_res)
|
||||
|| checkRightType<T0, Int64>(arguments, types_res)
|
||||
|| checkRightType<T0, Float32>(arguments, types_res)
|
||||
|| checkRightType<T0, Float64>(arguments, types_res))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & 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;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
Impl<T0, T1>::vector_vector(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<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;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->getData().size());
|
||||
Impl<T0, T1>::vector_constant(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0, typename T1>
|
||||
bool executeConstRightType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & 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;
|
||||
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = col_res->getData();
|
||||
vec_res.resize(col_left->size());
|
||||
Impl<T0, T1>::constant_vector(col_left->getData(), col_right->getData(), vec_res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T1> * col_right = dynamic_cast<ColumnConst<T1> *>(&*block.getByPosition(arguments[1]).column))
|
||||
{
|
||||
typedef typename Impl<T0, T1>::ResultType ResultType;
|
||||
|
||||
ResultType res = 0;
|
||||
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;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T0>
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
{
|
||||
if (ColumnVector<T0> * col_left = dynamic_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
if ( executeRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (ColumnConst<T0> * col_left = dynamic_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
if ( executeConstRightType<T0, UInt8>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt16>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt32>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, UInt64>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int8>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int16>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int32>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Int64>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float32>(block, arguments, result, col_left)
|
||||
|| executeConstRightType<T0, Float64>(block, arguments, result, col_left))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal column of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
public:
|
||||
/// Получить все имена функции.
|
||||
Names getNames() const
|
||||
@ -230,41 +372,19 @@ public:
|
||||
|
||||
DataTypes types_res;
|
||||
|
||||
#define CHECK_RIGHT_TYPE(TYPE0, TYPE1) \
|
||||
if (dynamic_cast<const DataType ## TYPE1 *>(&*arguments[1])) \
|
||||
types_res.push_back(new typename DataTypeFromFieldType<typename Impl<TYPE0, TYPE1>::ResultType>::Type);
|
||||
|
||||
#define CHECK_LEFT_TYPE(TYPE0) \
|
||||
if (dynamic_cast<const DataType ## TYPE0 *>(&*arguments[0])) \
|
||||
{ \
|
||||
CHECK_RIGHT_TYPE(TYPE0, UInt8) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, UInt16) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, UInt32) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, UInt64) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int8) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int16) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int32) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int64) \
|
||||
else \
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(), \
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); \
|
||||
}
|
||||
|
||||
CHECK_LEFT_TYPE(UInt8)
|
||||
else CHECK_LEFT_TYPE(UInt16)
|
||||
else CHECK_LEFT_TYPE(UInt32)
|
||||
else CHECK_LEFT_TYPE(UInt64)
|
||||
else CHECK_LEFT_TYPE(Int8)
|
||||
else CHECK_LEFT_TYPE(Int16)
|
||||
else CHECK_LEFT_TYPE(Int32)
|
||||
else CHECK_LEFT_TYPE(Int64)
|
||||
else
|
||||
if (!( checkLeftType<UInt8>(arguments, types_res)
|
||||
|| checkLeftType<UInt16>(arguments, types_res)
|
||||
|| checkLeftType<UInt32>(arguments, types_res)
|
||||
|| checkLeftType<UInt64>(arguments, types_res)
|
||||
|| checkLeftType<Int8>(arguments, types_res)
|
||||
|| checkLeftType<Int16>(arguments, types_res)
|
||||
|| checkLeftType<Int32>(arguments, types_res)
|
||||
|| checkLeftType<Int64>(arguments, types_res)
|
||||
|| checkLeftType<Float32>(arguments, types_res)
|
||||
|| checkLeftType<Float64>(arguments, types_res)))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
#undef CHECK_LEFT_TYPE
|
||||
#undef CHECK_RIGHT_TYPE
|
||||
|
||||
return types_res;
|
||||
}
|
||||
|
||||
@ -275,49 +395,18 @@ public:
|
||||
throw Exception("Wrong number of result columns in function " + getName() + ", should be 1.",
|
||||
ErrorCodes::ILLEGAL_NUMBER_OF_RESULT_COLUMNS);
|
||||
|
||||
#define CHECK_RIGHT_TYPE(TYPE0, TYPE1) \
|
||||
if (Column ## TYPE1 * col_right = dynamic_cast<Column ## TYPE1 *>(&*block.getByPosition(arguments[1]).column)) \
|
||||
{ \
|
||||
typedef typename Impl<TYPE0, TYPE1>::ResultType ResultType; \
|
||||
\
|
||||
typename ColumnVector<ResultType>::Container_t & vec_res = \
|
||||
dynamic_cast<ColumnVector<ResultType> &>(*block.getByPosition(result[0]).column).getData(); \
|
||||
\
|
||||
vec_res.resize(col_left->getData().size()); \
|
||||
\
|
||||
Impl<TYPE0, TYPE1>::vector_vector(col_left->getData(), col_right->getData(), vec_res); \
|
||||
}
|
||||
|
||||
#define CHECK_LEFT_TYPE(TYPE0) \
|
||||
if (Column ## TYPE0 * col_left = dynamic_cast<Column ## TYPE0 *>(&*block.getByPosition(arguments[0]).column)) \
|
||||
{ \
|
||||
CHECK_RIGHT_TYPE(TYPE0, UInt8) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, UInt16) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, UInt32) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, UInt64) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int8) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int16) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int32) \
|
||||
else CHECK_RIGHT_TYPE(TYPE0, Int64) \
|
||||
else \
|
||||
throw Exception("Illegal column of second argument of function " + getName(), \
|
||||
ErrorCodes::ILLEGAL_COLUMN); \
|
||||
}
|
||||
|
||||
CHECK_LEFT_TYPE(UInt8)
|
||||
else CHECK_LEFT_TYPE(UInt16)
|
||||
else CHECK_LEFT_TYPE(UInt32)
|
||||
else CHECK_LEFT_TYPE(UInt64)
|
||||
else CHECK_LEFT_TYPE(Int8)
|
||||
else CHECK_LEFT_TYPE(Int16)
|
||||
else CHECK_LEFT_TYPE(Int32)
|
||||
else CHECK_LEFT_TYPE(Int64)
|
||||
else
|
||||
if (!( executeLeftType<UInt8>(block, arguments, result)
|
||||
|| executeLeftType<UInt16>(block, arguments, result)
|
||||
|| executeLeftType<UInt32>(block, arguments, result)
|
||||
|| executeLeftType<UInt64>(block, arguments, result)
|
||||
|| executeLeftType<Int8>(block, arguments, result)
|
||||
|| executeLeftType<Int16>(block, arguments, result)
|
||||
|| executeLeftType<Int32>(block, arguments, result)
|
||||
|| executeLeftType<Int64>(block, arguments, result)
|
||||
|| executeLeftType<Float32>(block, arguments, result)
|
||||
|| executeLeftType<Float64>(block, arguments, result)))
|
||||
throw Exception("Illegal column of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
#undef CHECK_LEFT_TYPE
|
||||
#undef CHECK_RIGHT_TYPE
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -100,6 +100,16 @@ const ColumnWithNameAndType & Block::getByName(const std::string & name) const
|
||||
}
|
||||
|
||||
|
||||
size_t Block::getPositionByName(const std::string & name) const
|
||||
{
|
||||
IndexByName_t::const_iterator it = index_by_name.find(name);
|
||||
if (index_by_name.end() == it)
|
||||
throw Exception("Not found column " + name + " in block.", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
|
||||
return std::distance(const_cast<Container_t &>(data).begin(), it->second);
|
||||
}
|
||||
|
||||
|
||||
size_t Block::rows() const
|
||||
{
|
||||
size_t res = 0;
|
||||
|
@ -54,7 +54,6 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
DB::ColumnWithNameAndType descr_res;
|
||||
descr_res.type = *it;
|
||||
descr_res.column = descr_res.type->createColumn();
|
||||
descr_res.name = "z";
|
||||
|
||||
block.insert(descr_res);
|
||||
|
@ -112,7 +112,7 @@ void Expression::setNotCalculated(ASTPtr ast)
|
||||
ast->calculated = false;
|
||||
ASTs children = ast->getChildren();
|
||||
for (ASTs::iterator it = children.begin(); it != children.end(); ++it)
|
||||
setNotCalculated(ast);
|
||||
setNotCalculated(*it);
|
||||
}
|
||||
|
||||
|
||||
@ -140,12 +140,50 @@ void Expression::executeImpl(ASTPtr ast, Block & block)
|
||||
|
||||
if (ASTFunction * node = dynamic_cast<ASTFunction *>(&*ast))
|
||||
{
|
||||
std::cerr << node->getTreeID() << std::endl;
|
||||
|
||||
/// Вставляем в блок столбцы - результаты вычисления функции
|
||||
ColumnNumbers argument_numbers;
|
||||
ColumnNumbers result_numbers;
|
||||
|
||||
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() + "_" + Poco::NumberFormatter::format(res_num);
|
||||
|
||||
result_numbers.push_back(block.columns());
|
||||
block.insert(column);
|
||||
++res_num;
|
||||
}
|
||||
|
||||
ASTs arguments = node->arguments->getChildren();
|
||||
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
|
||||
argument_numbers.push_back(block.getPositionByName((*it)->getTreeID()));
|
||||
}
|
||||
|
||||
node->function->execute(block, argument_numbers, result_numbers);
|
||||
}
|
||||
else if (ASTLiteral * node = dynamic_cast<ASTLiteral *>(&*ast))
|
||||
{
|
||||
std::cerr << node->getTreeID() << std::endl;
|
||||
|
||||
/// Вставляем в блок столбец - константу
|
||||
|
||||
ColumnWithNameAndType column;
|
||||
column.column = node->type->createConstColumn(block.rows(), node->value);
|
||||
column.type = node->type;
|
||||
column.name = node->getTreeID();
|
||||
|
||||
block.insert(column);
|
||||
}
|
||||
|
||||
ast->calculated = true;
|
||||
}
|
||||
|
||||
|
||||
|
@ -2,6 +2,8 @@
|
||||
|
||||
#include <mysqlxx/mysqlxx.h>
|
||||
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
@ -10,6 +12,9 @@
|
||||
#include <DB/Parsers/ParserSelectQuery.h>
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
|
||||
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
|
||||
#include <DB/Interpreters/Expression.h>
|
||||
|
||||
|
||||
@ -44,6 +49,27 @@ void dump(DB::IAST & ast, int level = 0)
|
||||
}
|
||||
|
||||
|
||||
class OneBlockInputStream : public DB::IBlockInputStream
|
||||
{
|
||||
private:
|
||||
const DB::Block & block;
|
||||
bool has_been_read;
|
||||
public:
|
||||
OneBlockInputStream(const DB::Block & block_) : block(block_), has_been_read(false) {}
|
||||
|
||||
DB::Block read()
|
||||
{
|
||||
if (!has_been_read)
|
||||
{
|
||||
has_been_read = true;
|
||||
return block;
|
||||
}
|
||||
else
|
||||
return DB::Block();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
try
|
||||
@ -78,6 +104,34 @@ int main(int argc, char ** argv)
|
||||
DB::Expression expression(ast, context);
|
||||
|
||||
dump(*ast);
|
||||
|
||||
size_t n = 10;
|
||||
|
||||
DB::Block block;
|
||||
DB::ColumnWithNameAndType column_x;
|
||||
column_x.name = "x";
|
||||
column_x.type = new DB::DataTypeInt16;
|
||||
DB::ColumnInt16 * x = new DB::ColumnInt16;
|
||||
column_x.column = x;
|
||||
std::vector<Int16> & vec = x->getData();
|
||||
|
||||
vec.resize(n);
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
vec[i] = i;
|
||||
|
||||
block.insert(column_x);
|
||||
|
||||
expression.execute(block);
|
||||
|
||||
DB::DataTypes * data_types = new DB::DataTypes;
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
data_types->push_back(block.getByPosition(i).type);
|
||||
|
||||
OneBlockInputStream is(block);
|
||||
DB::WriteBufferFromOStream out_buf(std::cout);
|
||||
DB::TabSeparatedRowOutputStream os(out_buf, data_types);
|
||||
|
||||
DB::copyData(is, os);
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user