mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
dbms: removed VarUInt and VarInt data types [#CONV-2944].
This commit is contained in:
parent
8c787fde81
commit
494b212f3a
@ -3,7 +3,7 @@
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
#include <DB/AggregateFunctions/INullaryAggregateFunction.h>
|
||||
|
||||
@ -30,7 +30,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return new DataTypeVarUInt;
|
||||
return new DataTypeUInt64;
|
||||
}
|
||||
|
||||
void addZero() { ++count; }
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
@ -16,14 +15,14 @@ template <typename T> struct AggregateFunctionSumTraits;
|
||||
|
||||
template <> struct AggregateFunctionSumTraits<UInt64>
|
||||
{
|
||||
static DataTypePtr getReturnType() { return new DataTypeVarUInt; }
|
||||
static DataTypePtr getReturnType() { return new DataTypeUInt64; }
|
||||
static void write(UInt64 x, WriteBuffer & buf) { writeVarUInt(x, buf); }
|
||||
static void read(UInt64 & x, ReadBuffer & buf) { readVarUInt(x, buf); }
|
||||
};
|
||||
|
||||
template <> struct AggregateFunctionSumTraits<Int64>
|
||||
{
|
||||
static DataTypePtr getReturnType() { return new DataTypeVarInt; }
|
||||
static DataTypePtr getReturnType() { return new DataTypeInt64; }
|
||||
static void write(Int64 x, WriteBuffer & buf) { writeVarInt(x, buf); }
|
||||
static void read(Int64 & x, ReadBuffer & buf) { readVarInt(x, buf); }
|
||||
};
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
|
||||
@ -66,7 +65,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return new DataTypeVarUInt;
|
||||
return new DataTypeUInt64;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
|
@ -1,26 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/DataTypes/IDataTypeNumberVariable.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Типы столбцов для чисел переменной ширины. */
|
||||
|
||||
class DataTypeVarUInt : public IDataTypeNumberVariable<UInt64, ColumnUInt64>
|
||||
{
|
||||
public:
|
||||
std::string getName() const { return "VarUInt"; }
|
||||
DataTypePtr clone() const { return new DataTypeVarUInt; }
|
||||
};
|
||||
|
||||
class DataTypeVarInt : public IDataTypeNumberVariable<Int64, ColumnInt64>
|
||||
{
|
||||
public:
|
||||
std::string getName() const { return "VarInt"; }
|
||||
DataTypePtr clone() const { return new DataTypeVarInt; }
|
||||
};
|
||||
|
||||
}
|
@ -1,75 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
|
||||
#include <DB/IO/VarInt.h>
|
||||
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
|
||||
#include <DB/DataTypes/IDataTypeNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Реализует часть интерфейса IDataType, общую для знаковых и беззнаковых чисел переменной длины
|
||||
* - ввод и вывод в текстовом и бинарном виде.
|
||||
* Остаётся лишь чисто виртуальный метод getName().
|
||||
*
|
||||
* Параметры: FieldType - тип единичного значения, ColumnType - тип столбца со значениями.
|
||||
* (см. Core/Field.h, Columns/IColumn.h)
|
||||
*/
|
||||
template <typename FieldType, typename ColumnType>
|
||||
class IDataTypeNumberVariable : public IDataTypeNumber<FieldType>
|
||||
{
|
||||
public:
|
||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
writeVarT(static_cast<typename ColumnType::value_type>(boost::get<FieldType>(field)), ostr);
|
||||
}
|
||||
|
||||
void deserializeBinary(Field & field, ReadBuffer & istr) const
|
||||
{
|
||||
readVarT(static_cast<typename ColumnType::value_type &>(boost::get<FieldType &>(field)), istr);
|
||||
}
|
||||
|
||||
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const
|
||||
{
|
||||
const typename ColumnType::Container_t & x = dynamic_cast<const ColumnType &>(column).getData();
|
||||
size_t size = x.size();
|
||||
|
||||
size_t end = limit && offset + limit < size
|
||||
? offset + limit
|
||||
: size;
|
||||
|
||||
for (size_t i = offset; i < end; ++i)
|
||||
writeVarT(x[i], ostr);
|
||||
}
|
||||
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit) const
|
||||
{
|
||||
typename ColumnType::Container_t & x = dynamic_cast<ColumnType &>(column).getData();
|
||||
x.resize(limit);
|
||||
size_t i = 0;
|
||||
while (i < limit && !istr.eof())
|
||||
{
|
||||
readVarT(x[i], istr);
|
||||
++i;
|
||||
}
|
||||
x.resize(i);
|
||||
}
|
||||
|
||||
ColumnPtr createColumn() const
|
||||
{
|
||||
return new ColumnType;
|
||||
}
|
||||
|
||||
ColumnPtr createConstColumn(size_t size, const Field & field) const
|
||||
{
|
||||
return new ColumnConst<FieldType>(size, boost::get<FieldType>(field));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -3,7 +3,6 @@
|
||||
#include <Poco/NumberFormatter.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Functions/NumberTraits.h>
|
||||
|
||||
@ -262,9 +261,7 @@ private:
|
||||
|| 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))
|
||||
|| checkRightType<T0, DataTypeFloat64>(arguments, type_res))
|
||||
return true;
|
||||
else
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
@ -408,9 +405,7 @@ public:
|
||||
|| 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)))
|
||||
|| checkLeftType<DataTypeFloat64>(arguments, type_res)))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -511,9 +506,7 @@ public:
|
||||
|| checkType<DataTypeInt32>(arguments, result)
|
||||
|| checkType<DataTypeInt64>(arguments, result)
|
||||
|| checkType<DataTypeFloat32>(arguments, result)
|
||||
|| checkType<DataTypeFloat64>(arguments, result)
|
||||
|| checkType<DataTypeVarUInt>(arguments, result)
|
||||
|| checkType<DataTypeVarInt>(arguments, result)))
|
||||
|| checkType<DataTypeFloat64>(arguments, result)))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <DB/IO/WriteBufferFromVector.h>
|
||||
#include <DB/IO/ReadBufferFromString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
@ -297,8 +296,6 @@ public:
|
||||
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);
|
||||
@ -319,8 +316,6 @@ struct NameToInt32 { static const char * get() { return "toInt32"; } };
|
||||
struct NameToInt64 { static const char * get() { return "toInt64"; } };
|
||||
struct NameToFloat32 { static const char * get() { return "toFloat32"; } };
|
||||
struct NameToFloat64 { static const char * get() { return "toFloat64"; } };
|
||||
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"; } };
|
||||
@ -335,8 +330,6 @@ typedef FunctionConvert<DataTypeInt32, NameToInt32> FunctionToInt32;
|
||||
typedef FunctionConvert<DataTypeInt64, NameToInt64> FunctionToInt64;
|
||||
typedef FunctionConvert<DataTypeFloat32, NameToFloat32> FunctionToFloat32;
|
||||
typedef FunctionConvert<DataTypeFloat64, NameToFloat64> FunctionToFloat64;
|
||||
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;
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Poco/ByteOrder.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
@ -196,8 +195,6 @@ public:
|
||||
else if (dynamic_cast<const DataTypeInt16 * >(from_type)) executeType<Int16 >(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeInt32 * >(from_type)) executeType<Int32 >(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeInt64 * >(from_type)) executeType<Int64 >(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeVarUInt * >(from_type)) executeType<UInt64>(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeVarInt * >(from_type)) executeType<Int64 >(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeDate * >(from_type)) executeType<UInt16>(block, arguments, result);
|
||||
else if (dynamic_cast<const DataTypeDateTime * >(from_type)) executeType<UInt32>(block, arguments, result);
|
||||
else
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Poco/NumberFormatter.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Poco/Unicode.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
|
@ -48,12 +48,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
String argument_type_name = argument_types[0]->getName();
|
||||
|
||||
if (argument_type_name == "UInt8" || argument_type_name == "UInt16"
|
||||
|| argument_type_name == "UInt32" || argument_type_name == "UInt64"
|
||||
|| argument_type_name == "VarUInt")
|
||||
|| argument_type_name == "UInt32" || argument_type_name == "UInt64")
|
||||
return new AggregateFunctionSum<UInt64>;
|
||||
else if (argument_type_name == "Int8" || argument_type_name == "Int16"
|
||||
|| argument_type_name == "Int32" || argument_type_name == "Int64"
|
||||
|| argument_type_name == "VarInt")
|
||||
|| argument_type_name == "Int32" || argument_type_name == "Int64")
|
||||
return new AggregateFunctionSum<Int64>;
|
||||
else if (argument_type_name == "Float32" || argument_type_name == "Float64")
|
||||
return new AggregateFunctionSum<Float64>;
|
||||
@ -68,12 +66,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
String argument_type_name = argument_types[0]->getName();
|
||||
|
||||
if (argument_type_name == "UInt8" || argument_type_name == "UInt16"
|
||||
|| argument_type_name == "UInt32" || argument_type_name == "UInt64"
|
||||
|| argument_type_name == "VarUInt")
|
||||
|| argument_type_name == "UInt32" || argument_type_name == "UInt64")
|
||||
return new AggregateFunctionAvg<UInt64>;
|
||||
else if (argument_type_name == "Int8" || argument_type_name == "Int16"
|
||||
|| argument_type_name == "Int32" || argument_type_name == "Int64"
|
||||
|| argument_type_name == "VarInt")
|
||||
|| argument_type_name == "Int32" || argument_type_name == "Int64")
|
||||
return new AggregateFunctionAvg<Int64>;
|
||||
else if (argument_type_name == "Float32" || argument_type_name == "Float64")
|
||||
return new AggregateFunctionAvg<Float64>;
|
||||
@ -89,12 +85,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
|
||||
if (argument_type_name == "UInt8" || argument_type_name == "UInt16"
|
||||
|| argument_type_name == "UInt32" || argument_type_name == "UInt64"
|
||||
|| argument_type_name == "VarUInt"
|
||||
|| argument_type_name == "Date" || argument_type_name == "DateTime")
|
||||
return new AggregateFunctionUniq<UInt64>;
|
||||
else if (argument_type_name == "Int8" || argument_type_name == "Int16"
|
||||
|| argument_type_name == "Int32" || argument_type_name == "Int64"
|
||||
|| argument_type_name == "VarInt")
|
||||
|| argument_type_name == "Int32" || argument_type_name == "Int64")
|
||||
return new AggregateFunctionUniq<Int64>;
|
||||
else if (argument_type_name == "Float32" || argument_type_name == "Float64")
|
||||
return new AggregateFunctionUniq<Float64>;
|
||||
@ -118,8 +112,6 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
else if (argument_type_name == "Int16") return new AggregateFunctionQuantile<Int16>;
|
||||
else if (argument_type_name == "Int32") return new AggregateFunctionQuantile<Int32>;
|
||||
else if (argument_type_name == "Int64") return new AggregateFunctionQuantile<Int64>;
|
||||
else if (argument_type_name == "VarUInt") return new AggregateFunctionQuantile<UInt64>;
|
||||
else if (argument_type_name == "VarInt") return new AggregateFunctionQuantile<Int64>;
|
||||
else if (argument_type_name == "Float32") return new AggregateFunctionQuantile<Float32>;
|
||||
else if (argument_type_name == "Float64") return new AggregateFunctionQuantile<Float64>;
|
||||
else if (argument_type_name == "Date") return new AggregateFunctionQuantile<DataTypeDate::FieldType, false>;
|
||||
|
@ -9,7 +9,6 @@
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
@ -106,7 +105,7 @@ int main(int argc, char ** argv)
|
||||
boost::assign::push_back(*result_types)
|
||||
(new DB::DataTypeInt16)
|
||||
// (new DB::DataTypeString)
|
||||
(new DB::DataTypeVarUInt)
|
||||
(new DB::DataTypeUInt64)
|
||||
;
|
||||
|
||||
DB::Block sample;
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <mysqlxx/String.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
@ -39,8 +38,6 @@ DataTypeFactory::DataTypeFactory()
|
||||
("Int64", new DataTypeInt64)
|
||||
("Float32", new DataTypeFloat32)
|
||||
("Float64", new DataTypeFloat64)
|
||||
("VarUInt", new DataTypeVarUInt)
|
||||
("VarInt", new DataTypeVarInt)
|
||||
("Date", new DataTypeDate)
|
||||
("DateTime", new DataTypeDateTime)
|
||||
("String", new DataTypeString)
|
||||
|
@ -1,51 +0,0 @@
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
|
||||
#include <Poco/Stopwatch.h>
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/IO/WriteBufferFromOStream.h>
|
||||
#include <DB/IO/ReadBufferFromIStream.h>
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/DataTypes/DataTypesNumberVariable.h>
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
const char * file_name = "/dev/shm/test";
|
||||
Poco::SharedPtr<DB::ColumnUInt64> column1 = new DB::ColumnUInt64();
|
||||
Poco::SharedPtr<DB::ColumnUInt64> column2 = new DB::ColumnUInt64();
|
||||
DB::ColumnUInt64::Container_t & vec1 = column1->getData();
|
||||
DB::DataTypeVarUInt data_type;
|
||||
|
||||
Poco::Stopwatch stopwatch;
|
||||
size_t n = atoi(argv[1]);
|
||||
|
||||
vec1.resize(n);
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
vec1[i] = i;
|
||||
|
||||
{
|
||||
std::ofstream ostr(file_name);
|
||||
DB::WriteBufferFromOStream out_buf(ostr);
|
||||
|
||||
stopwatch.restart();
|
||||
data_type.serializeBinary(*column1, out_buf);
|
||||
stopwatch.stop();
|
||||
|
||||
std::cout << "Writing, elapsed: " << static_cast<double>(stopwatch.elapsed()) / 1000000 << std::endl;
|
||||
}
|
||||
|
||||
{
|
||||
std::ifstream istr(file_name);
|
||||
DB::ReadBufferFromIStream in_buf(istr);
|
||||
|
||||
stopwatch.restart();
|
||||
data_type.deserializeBinary(*column2, in_buf, n);
|
||||
stopwatch.stop();
|
||||
|
||||
std::cout << "Reading, elapsed: " << static_cast<double>(stopwatch.elapsed()) / 1000000 << std::endl;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
@ -72,8 +72,6 @@ namespace FunctionsLibrary
|
||||
("toInt64", new FunctionToInt64)
|
||||
("toFloat32", new FunctionToFloat32)
|
||||
("toFloat64", new FunctionToFloat64)
|
||||
("toVarUInt", new FunctionToVarUInt)
|
||||
("toVarInt", new FunctionToVarInt)
|
||||
("toDate", new FunctionToDate)
|
||||
("toDateTime", new FunctionToDateTime)
|
||||
("toString", new FunctionToString)
|
||||
|
Loading…
Reference in New Issue
Block a user