mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
dbms: development [#CONV-2944].
This commit is contained in:
parent
3843ff4078
commit
61e080f7b3
@ -6,6 +6,7 @@
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
#include <Poco/Unicode.h>
|
||||
|
||||
#include <DB/IO/WriteBufferFromVector.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
@ -28,22 +29,17 @@ namespace DB
|
||||
* - reinterpretAsType - преобразования чисел и дат в строки, содержащие тот же набор байт в машинном представлении, и наоборот.
|
||||
*/
|
||||
|
||||
/** Преобразование чисел в строки и наоборот: через форматирование и парсинг.
|
||||
* Преобразование дат и дат с временем в строки и наоборот: то же, что и выше, но форматировать и парсить надо по другому.
|
||||
*/
|
||||
|
||||
/** Преобразование чисел друг в друга, дат/дат-с-временем в числа и наоборот: делается обычным присваиванием.
|
||||
* (дата внутри хранится как количество дней с какого-то, дата-с-временем - как unix timestamp)
|
||||
* Преобразование даты в дату-с-временем и наоборот: добавление нулевого времени или вырезание только даты.
|
||||
*/
|
||||
template <typename ToDataType, typename Name>
|
||||
class FunctionConvert : public IFunction
|
||||
template <typename FromDataType, typename ToDataType, typename Name>
|
||||
struct ConvertImpl
|
||||
{
|
||||
private:
|
||||
typedef typename FromDataType::FieldType FromFieldType;
|
||||
typedef typename ToDataType::FieldType ToFieldType;
|
||||
|
||||
template <typename FromFieldType>
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnVector<FromFieldType> * col_from = dynamic_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
@ -57,24 +53,32 @@ private:
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
vec_to[i] = vec_from[i];
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (const ColumnConst<FromFieldType> * col_from = dynamic_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
block.getByPosition(result).column = new ColumnConst<ToFieldType>(col_from->size(), col_from->getData());
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
void executeDateToDateTime(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
|
||||
/** Преобразование даты в дату-с-временем: добавление нулевого времени.
|
||||
*/
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
|
||||
{
|
||||
typedef DataTypeDate::FieldType FromFieldType;
|
||||
typedef DataTypeDateTime::FieldType ToFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
typedef DataTypeDate::FieldType FromFieldType;
|
||||
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
|
||||
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = dynamic_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<ToFieldType> * col_to = new ColumnVector<ToFieldType>;
|
||||
@ -94,13 +98,22 @@ private:
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
void executeDateTimeToDate(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
|
||||
/** Преобразование даты-с-временем в дату: отбрасывание времени.
|
||||
*/
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
|
||||
{
|
||||
typedef DataTypeDateTime::FieldType FromFieldType;
|
||||
typedef DataTypeDate::FieldType ToFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
typedef DataTypeDateTime::FieldType FromFieldType;
|
||||
Yandex::DateLUTSingleton & date_lut = Yandex::DateLUTSingleton::instance();
|
||||
|
||||
if (const ColumnVector<FromFieldType> * col_from = dynamic_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
@ -122,10 +135,83 @@ private:
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
|
||||
/** Преобразование чисел в строки: через форматирование.
|
||||
*/
|
||||
template <typename FromDataType, typename Name>
|
||||
struct ConvertImpl<FromDataType, DataTypeString, Name>
|
||||
{
|
||||
typedef typename FromDataType::FieldType FromFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnVector<FromFieldType> * col_from = dynamic_cast<const ColumnVector<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnString * col_to = new ColumnString;
|
||||
block.getByPosition(result).column = col_to;
|
||||
|
||||
const typename ColumnVector<FromFieldType>::Container_t & vec_from = col_from->getData();
|
||||
ColumnUInt8::Container_t & data_to = dynamic_cast<ColumnUInt8 &>(col_to->getData()).getData();
|
||||
ColumnString::Offsets_t & offsets_to = col_to->getOffsets();
|
||||
size_t size = vec_from.size();
|
||||
data_to.resize(size * 2);
|
||||
offsets_to.resize(size);
|
||||
|
||||
WriteBufferFromVector<UInt8> write_buffer(data_to);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
writeText<FromFieldType>(vec_from[i], write_buffer);
|
||||
writeChar(0, write_buffer);
|
||||
offsets_to[i] = write_buffer.count();
|
||||
}
|
||||
data_to.resize(write_buffer.count());
|
||||
}
|
||||
else if (const ColumnConst<FromFieldType> * col_from = dynamic_cast<const ColumnConst<FromFieldType> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
std::vector<char> buf;
|
||||
WriteBufferFromVector<char> write_buffer(buf);
|
||||
writeText<FromFieldType>(col_from->getData(), write_buffer);
|
||||
block.getByPosition(result).column = new ColumnConstString(col_from->size(), std::string(&buf[0], write_buffer.count()));
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Преобразование строки в числа: через парсинг.
|
||||
*/
|
||||
template <typename ToDataType, typename Name>
|
||||
struct ConvertImpl<DataTypeString, ToDataType, Name>
|
||||
{
|
||||
typedef typename ToDataType::FieldType ToFieldType;
|
||||
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Name>
|
||||
struct ConvertImpl<DataTypeString, DataTypeString, Name>
|
||||
{
|
||||
static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
template <typename ToDataType, typename Name>
|
||||
class FunctionConvert : public IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
@ -141,43 +227,32 @@ public:
|
||||
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[0]->isNumeric()
|
||||
&& !dynamic_cast<const DataTypeDate *>(&*arguments[0])
|
||||
&& !dynamic_cast<const DataTypeDateTime *>(&*arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new ToDataType;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (dynamic_cast<const DataTypeDate *>(&*block.getByPosition(arguments[0]).type)
|
||||
&& std::tr1::is_same<ToDataType, DataTypeDateTime>::value)
|
||||
{
|
||||
/// Преобразование даты в дату с временем
|
||||
executeDateToDateTime(block, arguments, result);
|
||||
}
|
||||
else if (dynamic_cast<const DataTypeDateTime *>(&*block.getByPosition(arguments[0]).type)
|
||||
&& std::tr1::is_same<ToDataType, DataTypeDate>::value)
|
||||
{
|
||||
/// Преобразование даты с временем в дату
|
||||
executeDateTimeToDate(block, arguments, result);
|
||||
}
|
||||
else if (!( executeType<UInt8>(block, arguments, result)
|
||||
|| executeType<UInt16>(block, arguments, result)
|
||||
|| executeType<UInt32>(block, arguments, result)
|
||||
|| executeType<UInt64>(block, arguments, result)
|
||||
|| executeType<Int8>(block, arguments, result)
|
||||
|| executeType<Int16>(block, arguments, result)
|
||||
|| executeType<Int32>(block, arguments, result)
|
||||
|| executeType<Int64>(block, arguments, result)
|
||||
|| executeType<Float32>(block, arguments, result)
|
||||
|| executeType<Float64>(block, arguments, result)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
IDataType * from_type = &*block.getByPosition(arguments[0]).type;
|
||||
|
||||
if (dynamic_cast<const DataTypeUInt8 * >(from_type)) ConvertImpl<DataTypeUInt8, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeUInt16 * >(from_type)) ConvertImpl<DataTypeUInt16, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeUInt32 * >(from_type)) ConvertImpl<DataTypeUInt32, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeUInt64 * >(from_type)) ConvertImpl<DataTypeUInt64, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeInt8 * >(from_type)) ConvertImpl<DataTypeInt8, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeInt16 * >(from_type)) ConvertImpl<DataTypeInt16, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeInt32 * >(from_type)) ConvertImpl<DataTypeInt32, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeInt64 * >(from_type)) ConvertImpl<DataTypeInt64, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeFloat32 * >(from_type)) ConvertImpl<DataTypeFloat32, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeFloat64 * >(from_type)) ConvertImpl<DataTypeFloat64, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeVarUInt * >(from_type)) ConvertImpl<DataTypeVarUInt, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeVarInt * >(from_type)) ConvertImpl<DataTypeVarInt, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeDate * >(from_type)) ConvertImpl<DataTypeDate, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeDateTime * >(from_type)) ConvertImpl<DataTypeDateTime, ToDataType, Name>::execute(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeString * >(from_type)) ConvertImpl<DataTypeString, ToDataType, Name>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
@ -196,6 +271,7 @@ struct NameToVarUInt { static const char * get() { return "toVarUInt"; } };
|
||||
struct NameToVatInt { static const char * get() { return "toVarInt"; } };
|
||||
struct NameToDate { static const char * get() { return "toDate"; } };
|
||||
struct NameToDateTime { static const char * get() { return "toDateTime"; } };
|
||||
struct NameToString { static const char * get() { return "toString"; } };
|
||||
|
||||
typedef FunctionConvert<DataTypeUInt8, NameToUInt8> FunctionToUInt8;
|
||||
typedef FunctionConvert<DataTypeUInt16, NameToUInt16> FunctionToUInt16;
|
||||
@ -211,6 +287,7 @@ typedef FunctionConvert<DataTypeVarUInt, NameToVarUInt> FunctionToVarUInt;
|
||||
typedef FunctionConvert<DataTypeVarInt, NameToVatInt> FunctionToVarInt;
|
||||
typedef FunctionConvert<DataTypeDate, NameToDate> FunctionToDate;
|
||||
typedef FunctionConvert<DataTypeDateTime, NameToDateTime> FunctionToDateTime;
|
||||
typedef FunctionConvert<DataTypeString, NameToString> FunctionToString;
|
||||
|
||||
|
||||
}
|
||||
|
44
dbms/include/DB/IO/WriteBufferFromVector.h
Normal file
44
dbms/include/DB/IO/WriteBufferFromVector.h
Normal file
@ -0,0 +1,44 @@
|
||||
#pragma once
|
||||
|
||||
#include <iostream>
|
||||
|
||||
#include <DB/IO/WriteBuffer.h>
|
||||
|
||||
|
||||
#define WRITE_BUFFER_FROM_VECTOR_INITIAL_SIZE_IF_EMPTY 32
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Инициализируется вектором. Пишет данные в него. Когда вектор закончится - увеличивает его размер в два раза.
|
||||
* CharType - char или unsigned char.
|
||||
*/
|
||||
template <typename CharType>
|
||||
class WriteBufferFromVector : public WriteBuffer
|
||||
{
|
||||
private:
|
||||
typedef std::vector<CharType> VectorType;
|
||||
VectorType & vector;
|
||||
|
||||
void nextImpl()
|
||||
{
|
||||
size_t old_size = vector.size();
|
||||
vector.resize(old_size * 2);
|
||||
internal_buffer = Buffer(reinterpret_cast<Position>(&vector[old_size]), reinterpret_cast<Position>(&*vector.end()));
|
||||
working_buffer = internal_buffer;
|
||||
}
|
||||
|
||||
public:
|
||||
WriteBufferFromVector(VectorType & vector_)
|
||||
: WriteBuffer(reinterpret_cast<Position>(&vector_[0]), vector_.size()), vector(vector_)
|
||||
{
|
||||
if (vector.empty())
|
||||
{
|
||||
vector.resize(WRITE_BUFFER_FROM_VECTOR_INITIAL_SIZE_IF_EMPTY);
|
||||
set(reinterpret_cast<Position>(&vector[0]), vector.size());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -161,6 +161,7 @@ int main(int argc, char ** argv)
|
||||
(*context.functions)["toVarInt"] = new DB::FunctionToVarInt;
|
||||
(*context.functions)["toDate"] = new DB::FunctionToDate;
|
||||
(*context.functions)["toDateTime"] = new DB::FunctionToDateTime;
|
||||
(*context.functions)["toString"] = new DB::FunctionToString;
|
||||
|
||||
context.aggregate_function_factory = new DB::AggregateFunctionFactory;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user