mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: development [#CONV-2944].
This commit is contained in:
parent
b3bca9c160
commit
54808431ec
@ -1,5 +1,4 @@
|
||||
#ifndef DBMS_CORE_COLUMNS_NUMBER_H
|
||||
#define DBMS_CORE_COLUMNS_NUMBER_H
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
@ -24,5 +23,3 @@ typedef ColumnVector<Float32> ColumnFloat32;
|
||||
typedef ColumnVector<Float64> ColumnFloat64;
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,5 +1,4 @@
|
||||
#ifndef DBMS_CORE_BLOCK_H
|
||||
#define DBMS_CORE_BLOCK_H
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <map>
|
||||
@ -64,5 +63,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,15 +0,0 @@
|
||||
#ifndef DBMS_CORE_COLUMN_NAMES_H
|
||||
#define DBMS_CORE_COLUMN_NAMES_H
|
||||
|
||||
#include <vector>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
typedef std::vector<std::string> ColumnNames;
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -49,6 +49,10 @@ namespace ErrorCodes
|
||||
TOO_LARGE_SIZE_COMPRESSED,
|
||||
CHECKSUM_DOESNT_MATCH,
|
||||
CANNOT_PARSE_DATETIME,
|
||||
NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
ILLEGAL_COLUMN,
|
||||
ILLEGAL_NUMBER_OF_RESULT_COLUMNS,
|
||||
};
|
||||
}
|
||||
|
||||
|
12
dbms/include/DB/Core/Names.h
Normal file
12
dbms/include/DB/Core/Names.h
Normal file
@ -0,0 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
typedef std::vector<std::string> Names;
|
||||
|
||||
}
|
@ -10,12 +10,20 @@ namespace DB
|
||||
|
||||
/** Типы столбцов для чисел фиксированной ширины. */
|
||||
|
||||
template <typename T>
|
||||
struct DataTypeFromFieldType;
|
||||
|
||||
#define DEFINE_DATA_TYPE_NUMBER_FIXED(TYPE) \
|
||||
class DataType ## TYPE : public IDataTypeNumberFixed<TYPE, Column ## TYPE> \
|
||||
{ \
|
||||
public: \
|
||||
std::string getName() const { return #TYPE; } \
|
||||
SharedPtr<IDataType> clone() const { return new DataType ## TYPE; } \
|
||||
}; \
|
||||
\
|
||||
template <> struct DataTypeFromFieldType<TYPE> \
|
||||
{ \
|
||||
typedef DataType ## TYPE Type; \
|
||||
};
|
||||
|
||||
DEFINE_DATA_TYPE_NUMBER_FIXED(UInt8);
|
||||
|
168
dbms/include/DB/Functions/FunctionsArithmetic.h
Normal file
168
dbms/include/DB/Functions/FunctionsArithmetic.h
Normal file
@ -0,0 +1,168 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/NumberFormatter.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Functions/NumberTraits.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Арифметические функции: +, -, *, /, %, div.
|
||||
*/
|
||||
|
||||
template<typename A, typename B>
|
||||
struct PlusImpl
|
||||
{
|
||||
typedef typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type ResultType;
|
||||
|
||||
static void vector_vector(const std::vector<A> & a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a[i] + b[i];
|
||||
}
|
||||
|
||||
static void vector_constant(const std::vector<A> & a, B b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a[i] + b;
|
||||
}
|
||||
|
||||
static void constant_vector(A a, const std::vector<B> & b, std::vector<ResultType> & c)
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a + b[i];
|
||||
}
|
||||
|
||||
static void constant_constant(A a, B b, ResultType & c)
|
||||
{
|
||||
c = a + b;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <template <typename, typename> class Impl, typename Name>
|
||||
class FunctionBinaryArithmetic : public IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить все имена функции.
|
||||
Names getNames() const
|
||||
{
|
||||
Names names;
|
||||
names.push_back(Name::get());
|
||||
return names;
|
||||
}
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypes getReturnTypes(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;
|
||||
|
||||
#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
|
||||
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;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result)
|
||||
{
|
||||
if (result.size() != 1)
|
||||
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
|
||||
throw Exception("Illegal column of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
#undef CHECK_LEFT_TYPE
|
||||
#undef CHECK_RIGHT_TYPE
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NamePlus { static const char * get() { return "plus"; } };
|
||||
|
||||
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
||||
|
||||
|
||||
}
|
43
dbms/include/DB/Functions/IFunction.h
Normal file
43
dbms/include/DB/Functions/IFunction.h
Normal file
@ -0,0 +1,43 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Names.h>
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/Core/ColumnNumbers.h>
|
||||
#include <DB/DataTypes/IDataType.h>
|
||||
#include <DB/DataTypes/DataTypes.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Интерфейс для обычных функций.
|
||||
* Обычные функции - это функции, которые не меняют количество строк в таблице,
|
||||
* и результат работы которых для каждой строчки не зависит от других строк.
|
||||
*
|
||||
* Функция может принимать произвольное количество аргументов и возвращать произвольное количество значений.
|
||||
* Типы и количество значений результата зависят от типов и количества аргументов.
|
||||
*
|
||||
* Функция диспетчеризуется для целого блока. Это позволяет производить всевозможные проверки редко,
|
||||
* и делать основную работу в виде эффективного цикла.
|
||||
*
|
||||
* Функция применяется к одному или нескольким столбцам блока, и записывает свой результат,
|
||||
* добавляя новые столбцы к блоку. Функция не модифицирует свои агрументы.
|
||||
*/
|
||||
class IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить все имена функции.
|
||||
virtual Names getNames() const = 0;
|
||||
|
||||
/// Получить основное имя функции.
|
||||
std::string getName() const { return getNames()[0]; }
|
||||
|
||||
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
virtual DataTypes getReturnTypes(const DataTypes & arguments) const = 0;
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
virtual void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & result) = 0;
|
||||
};
|
||||
|
||||
|
||||
}
|
143
dbms/include/DB/Functions/NumberTraits.h
Normal file
143
dbms/include/DB/Functions/NumberTraits.h
Normal file
@ -0,0 +1,143 @@
|
||||
#pragma once
|
||||
|
||||
#include <boost/mpl/bool.hpp>
|
||||
#include <boost/mpl/int.hpp>
|
||||
#include <boost/mpl/if.hpp>
|
||||
#include <boost/mpl/and.hpp>
|
||||
#include <boost/mpl/or.hpp>
|
||||
#include <boost/mpl/not.hpp>
|
||||
#include <boost/mpl/greater.hpp>
|
||||
#include <boost/mpl/max.hpp>
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Позволяет получить тип результата применения функций +, -, *, /, %, div (целочисленное деление).
|
||||
* Правила отличаются от используемых в C++.
|
||||
*/
|
||||
|
||||
namespace NumberTraits
|
||||
{
|
||||
typedef boost::mpl::false_ Unsigned;
|
||||
typedef boost::mpl::true_ Signed;
|
||||
|
||||
typedef boost::mpl::false_ Integer;
|
||||
typedef boost::mpl::true_ Floating;
|
||||
|
||||
typedef boost::mpl::int_<8> Bits8;
|
||||
typedef boost::mpl::int_<16> Bits16;
|
||||
typedef boost::mpl::int_<32> Bits32;
|
||||
typedef boost::mpl::int_<64> Bits64;
|
||||
|
||||
template <typename T> struct Next;
|
||||
|
||||
template <> struct Next<Bits8> { typedef Bits16 Type; };
|
||||
template <> struct Next<Bits16> { typedef Bits32 Type; };
|
||||
template <> struct Next<Bits32> { typedef Bits64 Type; };
|
||||
template <> struct Next<Bits64> { typedef Bits64 Type; };
|
||||
|
||||
template <typename T> struct Traits;
|
||||
|
||||
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; };
|
||||
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; };
|
||||
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; };
|
||||
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; };
|
||||
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; };
|
||||
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; };
|
||||
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; };
|
||||
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; };
|
||||
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; };
|
||||
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; };
|
||||
|
||||
template <typename Sign, typename Floatness, typename Bits> struct Construct;
|
||||
|
||||
template <> struct Construct<Unsigned, Integer, Bits8> { typedef UInt8 Type; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits16> { typedef UInt16 Type; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits32> { typedef UInt32 Type; };
|
||||
template <> struct Construct<Unsigned, Integer, Bits64> { typedef UInt64 Type; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits8> { typedef Float32 Type; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits16> { typedef Float32 Type; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits32> { typedef Float32 Type; };
|
||||
template <> struct Construct<Unsigned, Floating, Bits64> { typedef Float64 Type; };
|
||||
template <> struct Construct<Signed, Integer, Bits8> { typedef Int8 Type; };
|
||||
template <> struct Construct<Signed, Integer, Bits16> { typedef Int16 Type; };
|
||||
template <> struct Construct<Signed, Integer, Bits32> { typedef Int32 Type; };
|
||||
template <> struct Construct<Signed, Integer, Bits64> { typedef Int64 Type; };
|
||||
template <> struct Construct<Signed, Floating, Bits8> { typedef Float32 Type; };
|
||||
template <> struct Construct<Signed, Floating, Bits16> { typedef Float32 Type; };
|
||||
template <> struct Construct<Signed, Floating, Bits32> { typedef Float32 Type; };
|
||||
template <> struct Construct<Signed, Floating, Bits64> { typedef Float64 Type; };
|
||||
|
||||
/** Результат сложения или умножения вычисляется по следующим правилам:
|
||||
* - если один из аргументов с плавающей запятой, то результат - с плавающей запятой, иначе - целый;
|
||||
* - если одно из аргументов со знаком, то результат - со знаком, иначе - без знака;
|
||||
* - результат содержит больше бит (не только значащих), чем максимум в аргументах
|
||||
* (например, UInt8 + Int32 = Int64).
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfAdditionMultiplication
|
||||
{
|
||||
typedef typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct ResultOfSubstraction
|
||||
{
|
||||
typedef typename Construct<
|
||||
typename boost::mpl::if_<
|
||||
typename boost::mpl::and_<
|
||||
typename boost::mpl::not_<typename Traits<A>::Sign>,
|
||||
typename boost::mpl::not_<typename Traits<A>::Floatness>,
|
||||
typename boost::mpl::not_<typename Traits<B>::Floatness>,
|
||||
typename boost::mpl::greater<typename Traits<A>::Bits, typename Traits<B>::Bits> >,
|
||||
Unsigned,
|
||||
Signed>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename boost::mpl::if_<
|
||||
typename boost::mpl::and_<
|
||||
typename boost::mpl::not_<typename Traits<A>::Sign>,
|
||||
typename boost::mpl::not_<typename Traits<B>::Sign>,
|
||||
typename boost::mpl::not_<typename Traits<A>::Floatness>,
|
||||
typename boost::mpl::not_<typename Traits<B>::Floatness>,
|
||||
typename boost::mpl::greater<typename Traits<A>::Bits, typename Traits<B>::Bits> >,
|
||||
typename Traits<A>::Bits,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::type>::Type Type;
|
||||
};
|
||||
|
||||
/** При делении всегда получается число с плавающей запятой.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfFloatingPointDivision
|
||||
{
|
||||
typedef typename Construct<
|
||||
Signed,
|
||||
Floating,
|
||||
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
|
||||
};
|
||||
|
||||
/** При целочисленном делении получается число, битность которого равна делимому.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfIntegerDivision
|
||||
{
|
||||
typedef typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Traits<A>::Bits>::Type Type;
|
||||
};
|
||||
|
||||
/** При взятии остатка получается число, битность которого равна делителю.
|
||||
*/
|
||||
template <typename A, typename B> struct ResultOfModulo
|
||||
{
|
||||
typedef typename Construct<
|
||||
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
|
||||
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
|
||||
typename Traits<B>::Bits>::Type Type;
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -1,7 +1,7 @@
|
||||
#ifndef DBMS_COMMON_READBUFFER_H
|
||||
#define DBMS_COMMON_READBUFFER_H
|
||||
#pragma once
|
||||
|
||||
#include <cstring>
|
||||
#include <algorithm>
|
||||
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
@ -106,5 +106,3 @@ private:
|
||||
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -1,5 +1,4 @@
|
||||
#ifndef DBMS_COMMON_WRITEBUFFER_H
|
||||
#define DBMS_COMMON_WRITEBUFFER_H
|
||||
#pragma once
|
||||
|
||||
#include <algorithm>
|
||||
#include <cstring>
|
||||
@ -78,5 +77,3 @@ private:
|
||||
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Core/Defines.h>
|
||||
#include <DB/Core/ColumnNames.h>
|
||||
#include <DB/Core/Names.h>
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/DataStreams/IBlockInputStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
@ -39,7 +39,7 @@ public:
|
||||
* Возвращает объект, с помощью которого можно последовательно читать данные.
|
||||
*/
|
||||
virtual SharedPtr<IBlockInputStream> read(
|
||||
const ColumnNames & column_names,
|
||||
const Names & column_names,
|
||||
const ptree & query,
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE)
|
||||
{
|
||||
|
@ -24,11 +24,11 @@ class StorageLog;
|
||||
class LogBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
public:
|
||||
LogBlockInputStream(size_t block_size_, const ColumnNames & column_names_, StorageLog & storage_);
|
||||
LogBlockInputStream(size_t block_size_, const Names & column_names_, StorageLog & storage_);
|
||||
Block read();
|
||||
private:
|
||||
size_t block_size;
|
||||
const ColumnNames & column_names;
|
||||
const Names & column_names;
|
||||
StorageLog & storage;
|
||||
|
||||
struct Stream
|
||||
@ -89,7 +89,7 @@ public:
|
||||
std::string getName() const { return "Log"; }
|
||||
|
||||
SharedPtr<IBlockInputStream> read(
|
||||
const ColumnNames & column_names,
|
||||
const Names & column_names,
|
||||
const ptree & query,
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE);
|
||||
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
std::string getName() const { return "SystemNumbers"; }
|
||||
|
||||
SharedPtr<IBlockInputStream> read(
|
||||
const ColumnNames & column_names,
|
||||
const Names & column_names,
|
||||
const ptree & query,
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE);
|
||||
};
|
||||
|
89
dbms/src/Functions/tests/functions_arithmetic.cpp
Normal file
89
dbms/src/Functions/tests/functions_arithmetic.cpp
Normal file
@ -0,0 +1,89 @@
|
||||
#include <iostream>
|
||||
#include <iomanip>
|
||||
|
||||
#include <Poco/Stopwatch.h>
|
||||
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
try
|
||||
{
|
||||
size_t n = atoi(argv[1]);
|
||||
|
||||
DB::ColumnWithNameAndType descr1;
|
||||
DB::ColumnUInt8 * col1 = new DB::ColumnUInt8;
|
||||
descr1.type = new DB::DataTypeUInt8;
|
||||
descr1.column = col1;
|
||||
descr1.name = "x";
|
||||
col1->getData().resize(n);
|
||||
|
||||
DB::ColumnWithNameAndType descr2;
|
||||
DB::ColumnInt16 * col2 = new DB::ColumnInt16;
|
||||
descr2.type = new DB::DataTypeInt16;
|
||||
descr2.column = col2;
|
||||
descr2.name = "x";
|
||||
|
||||
DB::Block block;
|
||||
block.insert(descr1);
|
||||
block.insert(descr2);
|
||||
col2->getData().resize(n);
|
||||
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
{
|
||||
col1->getData()[i] = 1;
|
||||
col2->getData()[i] = 1;
|
||||
}
|
||||
|
||||
DB::FunctionPlus f;
|
||||
DB::DataTypes arg_types;
|
||||
arg_types.push_back(descr1.type);
|
||||
arg_types.push_back(descr2.type);
|
||||
|
||||
DB::ColumnNumbers arg_nums;
|
||||
arg_nums.push_back(0);
|
||||
arg_nums.push_back(1);
|
||||
|
||||
DB::ColumnNumbers res_nums;
|
||||
res_nums.push_back(2);
|
||||
|
||||
DB::DataTypes res_types = f.getReturnTypes(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.column = descr_res.type->createColumn();
|
||||
descr_res.name = "z";
|
||||
|
||||
block.insert(descr_res);
|
||||
}
|
||||
|
||||
{
|
||||
Poco::Stopwatch stopwatch;
|
||||
stopwatch.start();
|
||||
|
||||
f.execute(block, arg_nums, res_nums);
|
||||
|
||||
stopwatch.stop();
|
||||
std::cout << std::fixed << std::setprecision(2)
|
||||
<< "Elapsed " << stopwatch.elapsed() / 1000000.0 << " sec."
|
||||
<< ", " << n * 1000000 / stopwatch.elapsed() << " rows/sec."
|
||||
<< std::endl;
|
||||
}
|
||||
|
||||
size_t x = 0;
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
x += boost::get<DB::Int64>((*block.getByPosition(2).column)[i]);
|
||||
|
||||
std::cout << x << std::endl;
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
std::cerr << e.message() << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
32
dbms/src/Functions/tests/number_traits.cpp
Normal file
32
dbms/src/Functions/tests/number_traits.cpp
Normal file
@ -0,0 +1,32 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <DB/Functions/NumberTraits.h>
|
||||
|
||||
|
||||
void printType(DB::UInt8 x) { std::cout << "UInt8" << std::endl; }
|
||||
void printType(DB::UInt16 x) { std::cout << "UInt16" << std::endl; }
|
||||
void printType(DB::UInt32 x) { std::cout << "UInt32" << std::endl; }
|
||||
void printType(DB::UInt64 x) { std::cout << "UInt64" << std::endl; }
|
||||
void printType(DB::Int8 x) { std::cout << "Int8" << std::endl; }
|
||||
void printType(DB::Int16 x) { std::cout << "Int16" << std::endl; }
|
||||
void printType(DB::Int32 x) { std::cout << "Int32" << std::endl; }
|
||||
void printType(DB::Int64 x) { std::cout << "Int64" << std::endl; }
|
||||
void printType(DB::Float32 x) { std::cout << "Float32" << std::endl; }
|
||||
void printType(DB::Float64 x) { std::cout << "Float64" << std::endl; }
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
printType(DB::NumberTraits::ResultOfAdditionMultiplication<DB::UInt8, DB::UInt8>::Type());
|
||||
printType(DB::NumberTraits::ResultOfAdditionMultiplication<DB::UInt8, DB::Int32>::Type());
|
||||
printType(DB::NumberTraits::ResultOfAdditionMultiplication<DB::UInt8, DB::Float32>::Type());
|
||||
printType(DB::NumberTraits::ResultOfSubstraction<DB::UInt8, DB::UInt8>::Type());
|
||||
printType(DB::NumberTraits::ResultOfSubstraction<DB::UInt16, DB::UInt8>::Type());
|
||||
printType(DB::NumberTraits::ResultOfSubstraction<DB::UInt16, DB::Int8>::Type());
|
||||
printType(DB::NumberTraits::ResultOfFloatingPointDivision<DB::UInt16, DB::Int16>::Type());
|
||||
printType(DB::NumberTraits::ResultOfFloatingPointDivision<DB::UInt32, DB::Int16>::Type());
|
||||
printType(DB::NumberTraits::ResultOfIntegerDivision<DB::UInt8, DB::Int16>::Type());
|
||||
printType(DB::NumberTraits::ResultOfModulo<DB::UInt32, DB::Int8>::Type());
|
||||
|
||||
return 0;
|
||||
}
|
@ -12,7 +12,7 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
LogBlockInputStream::LogBlockInputStream(size_t block_size_, const ColumnNames & column_names_, StorageLog & storage_)
|
||||
LogBlockInputStream::LogBlockInputStream(size_t block_size_, const Names & column_names_, StorageLog & storage_)
|
||||
: block_size(block_size_), column_names(column_names_), storage(storage_)
|
||||
{
|
||||
}
|
||||
@ -22,7 +22,7 @@ Block LogBlockInputStream::read()
|
||||
{
|
||||
Block res;
|
||||
|
||||
for (ColumnNames::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
{
|
||||
if (storage.columns->end() == storage.columns->find(*it))
|
||||
throw Exception("There is no column with name " + *it + " in table.",
|
||||
@ -31,7 +31,7 @@ Block LogBlockInputStream::read()
|
||||
streams.insert(std::make_pair(*it, new Stream(storage.files[*it].path())));
|
||||
}
|
||||
|
||||
for (ColumnNames::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
for (Names::const_iterator it = column_names.begin(); it != column_names.end(); ++it)
|
||||
{
|
||||
ColumnWithNameAndType column;
|
||||
column.name = *it;
|
||||
@ -93,7 +93,7 @@ StorageLog::StorageLog(const std::string & path_, const std::string & name_, Sha
|
||||
|
||||
|
||||
SharedPtr<IBlockInputStream> StorageLog::read(
|
||||
const ColumnNames & column_names,
|
||||
const Names & column_names,
|
||||
const ptree & query,
|
||||
size_t max_block_size)
|
||||
{
|
||||
|
@ -39,7 +39,7 @@ Block NumbersBlockInputStream::read()
|
||||
|
||||
|
||||
SharedPtr<IBlockInputStream> StorageSystemNumbers::read(
|
||||
const ColumnNames & column_names, const ptree & query, size_t max_block_size)
|
||||
const Names & column_names, const ptree & query, size_t max_block_size)
|
||||
{
|
||||
if (column_names.size() != 1)
|
||||
throw Exception("Incorrect number of columns.", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
|
||||
|
@ -96,7 +96,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
SharedPtr<DB::NamesAndTypes> names_and_types_map = new DB::NamesAndTypes;
|
||||
SharedPtr<DB::DataTypes> data_types = new DB::DataTypes;
|
||||
DB::ColumnNames column_names;
|
||||
DB::Names column_names;
|
||||
|
||||
for (NamesAndTypesList::const_iterator it = names_and_types_list.begin(); it != names_and_types_list.end(); ++it)
|
||||
{
|
||||
@ -135,7 +135,7 @@ int main(int argc, char ** argv)
|
||||
if (argc == 2 && 0 == strcmp(argv[1], "read"))
|
||||
{
|
||||
/*
|
||||
DB::ColumnNames column_names;
|
||||
DB::Names column_names;
|
||||
boost::assign::push_back(column_names)
|
||||
("UniqID");
|
||||
|
||||
|
@ -61,7 +61,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
/// читаем из неё
|
||||
{
|
||||
DB::ColumnNames column_names;
|
||||
DB::Names column_names;
|
||||
column_names.push_back("a");
|
||||
column_names.push_back("b");
|
||||
|
||||
|
@ -18,7 +18,7 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
DB::StorageSystemNumbers table;
|
||||
|
||||
DB::ColumnNames column_names;
|
||||
DB::Names column_names;
|
||||
column_names.push_back("number");
|
||||
|
||||
Poco::SharedPtr<DB::DataTypes> column_types = new DB::DataTypes;
|
||||
|
Loading…
Reference in New Issue
Block a user