ClickHouse/src/Functions/byteSize.cpp

304 lines
12 KiB
C++
Raw Normal View History

2020-12-21 01:54:04 +00:00
#include <Functions/IFunctionImpl.h>
#include <Functions/FunctionFactory.h>
2020-12-23 19:39:02 +00:00
#include <Functions/FunctionHelpers.h>
2020-12-21 01:54:04 +00:00
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeEnum.h>
2020-12-23 19:39:02 +00:00
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
2020-12-21 01:54:04 +00:00
#include <DataTypes/DataTypeFixedString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
2020-12-23 19:39:02 +00:00
#include <Columns/ColumnVector.h>
#include <Core/Types.h>
#include <Common/UInt128.h>
2020-12-21 01:54:04 +00:00
#include <ext/range.h>
namespace DB
{
2020-12-23 19:39:02 +00:00
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
2020-12-24 17:15:11 +00:00
template <> struct NativeType<DataTypeDate> { using Type = DataTypeDate::FieldType; };
template <> struct NativeType<DataTypeDateTime> { using Type = DataTypeDateTime::FieldType; };
template <> struct NativeType<DataTypeDateTime64> { using Type = NativeType<DataTypeDateTime64::FieldType>::Type; };
template <> struct NativeType<DataTypeUUID> { using Type = DataTypeUUID::FieldType; };
template <> struct NativeType<DataTypeEnum8> { using Type = DataTypeEnum8::FieldType; };
template <> struct NativeType<DataTypeEnum16> { using Type = DataTypeEnum16::FieldType; };
2020-12-21 01:54:04 +00:00
namespace
{
template <typename T> struct ByteSizeForNative { static constexpr const UInt64 value = sizeof(typename NativeType<T>::Type); };
/** byteSize() - get the columns size in number of bytes.
*/
class FunctionByteSize : public IFunction
{
public:
static constexpr auto name = "byteSize";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionByteSize>();
}
/// Get the function name.
String getName() const override
{
return name;
}
bool isDeterministic() const override { return false; }
bool isDeterministicInScopeOfQuery() const override
{
return false;
}
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override
{
return 0;
}
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{
return std::make_shared<DataTypeUInt64>();
}
2020-12-24 17:15:11 +00:00
// TODO:
// case TypeIndex::Tuple: return "Tuple";
// case TypeIndex::Nullable: return "Nullable";
// case TypeIndex::Function: return "Function";
// case TypeIndex::AggregateFunction: return "AggregateFunction";
// case TypeIndex::LowCardinality: return "LowCardinality";
2020-12-21 01:54:04 +00:00
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
auto result_col = ColumnUInt64::create(input_rows_count, 0);
auto & vec_res = result_col->getData();
for (const auto & arg : arguments)
{
const ColumnPtr & column = arg.column;
const IDataType * data_type = arg.type.get();
byteSizeOne(data_type, column, vec_res);
}
return result_col;
}
2020-12-23 19:39:02 +00:00
static void byteSizeOne(const IDataType * data_type, const ColumnPtr & column, ColumnUInt64::Container & vec_res)
{
2020-12-21 01:54:04 +00:00
size_t vec_size = vec_res.size();
UInt64 byte_size = 0;
2020-12-23 19:39:02 +00:00
if (byteSizeByDataType(data_type, byte_size))
{
for (size_t i = 0; i < vec_size; ++i)
{
2020-12-21 01:54:04 +00:00
vec_res[i] += byte_size;
}
}
2020-12-23 19:39:02 +00:00
else
{
byteSizeByColumn(data_type, column, vec_res);
}
2020-12-21 01:54:04 +00:00
}
2020-12-24 17:15:11 +00:00
static bool byteSizeByDataType(const IDataType * data_type, UInt64 & byte_size)
{
TypeIndex type_id = data_type->getTypeId();
if (byteSizeByTypeId(type_id, byte_size))
return true;
switch (type_id)
{
case TypeIndex::FixedString:
byte_size = typeid_cast<const DataTypeFixedString *>(data_type)->getN();
break;
default: return false;
}
return true;
}
2020-12-23 19:39:02 +00:00
static bool byteSizeByTypeId(TypeIndex type_id, UInt64 & byte_size)
{
switch (type_id)
2020-12-21 01:54:04 +00:00
{
case TypeIndex::Nothing: byte_size = 0; break;
case TypeIndex::UInt8: byte_size = ByteSizeForNative<UInt8>::value; break;
case TypeIndex::UInt16: byte_size = ByteSizeForNative<UInt16>::value; break;
case TypeIndex::UInt32: byte_size = ByteSizeForNative<UInt32>::value; break;
case TypeIndex::UInt64: byte_size = ByteSizeForNative<UInt64>::value; break;
case TypeIndex::UInt128: byte_size = ByteSizeForNative<UInt128>::value; break;
case TypeIndex::UInt256: byte_size = ByteSizeForNative<UInt256>::value; break;
case TypeIndex::Int8: byte_size = ByteSizeForNative<Int8>::value; break;
case TypeIndex::Int16: byte_size = ByteSizeForNative<Int16>::value; break;
case TypeIndex::Int32: byte_size = ByteSizeForNative<Int32>::value; break;
case TypeIndex::Int64: byte_size = ByteSizeForNative<Int64>::value; break;
case TypeIndex::Int128: byte_size = ByteSizeForNative<Int128>::value; break;
case TypeIndex::Int256: byte_size = ByteSizeForNative<Int256>::value; break;
case TypeIndex::Float32: byte_size = ByteSizeForNative<Float32>::value; break;
case TypeIndex::Float64: byte_size = ByteSizeForNative<Float64>::value; break;
2020-12-24 17:15:11 +00:00
case TypeIndex::Date: byte_size = ByteSizeForNative<DataTypeDate>::value; break;
case TypeIndex::DateTime: byte_size = ByteSizeForNative<DataTypeDateTime>::value; break;
case TypeIndex::DateTime64: byte_size = ByteSizeForNative<DataTypeDateTime64>::value; break;
case TypeIndex::Enum8: byte_size = ByteSizeForNative<DataTypeEnum8>::value; break;
case TypeIndex::Enum16: byte_size = ByteSizeForNative<DataTypeEnum16>::value; break;
2020-12-21 01:54:04 +00:00
case TypeIndex::Decimal32: byte_size = ByteSizeForNative<Decimal32>::value; break;
case TypeIndex::Decimal64: byte_size = ByteSizeForNative<Decimal64>::value; break;
case TypeIndex::Decimal128: byte_size = ByteSizeForNative<Decimal128>::value; break;
case TypeIndex::Decimal256: byte_size = ByteSizeForNative<Decimal256>::value; break;
2020-12-24 17:15:11 +00:00
case TypeIndex::UUID: byte_size = ByteSizeForNative<DataTypeUUID>::value; break;
// case TypeIndex::Interval: internal use only.
// case TypeIndex::Set: internal use only.
2020-12-21 01:54:04 +00:00
default: return false;
}
return true;
}
2020-12-23 19:39:02 +00:00
#define INTEGRAL_TPL_PACK UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, \
Int8, Int16, Int32, Int64, Int128, Int256, \
Float32, Float64
// The following is not supported by ColumnVector:
// Decimal32, Decimal64, Decimal128, Decimal256
// DataTypeEnum8, DataTypeEnum16, DataTypeDate, DataTypeDateTime, DataTypeDateTime64
2020-12-24 17:15:11 +00:00
// DataTypeUUID
2020-12-23 19:39:02 +00:00
static bool byteSizeByColumn(const IDataType * data_type, const ColumnPtr & column, ColumnUInt64::Container & vec_res)
{
WhichDataType which(data_type);
2020-12-21 01:54:04 +00:00
size_t vec_size = vec_res.size();
2020-12-23 19:39:02 +00:00
if (which.isString()) // TypeIndex::String
2020-12-21 01:54:04 +00:00
{
2020-12-23 19:39:02 +00:00
const ColumnString * col_str = checkAndGetColumn<ColumnString>(column.get());
2020-12-21 01:54:04 +00:00
const auto & offsets = col_str->getOffsets();
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < vec_size; ++i)
{
vec_res[i] += offsets[i] - prev_offset + sizeof(offsets[0]);
prev_offset = offsets[i];
}
return true;
}
2020-12-23 19:39:02 +00:00
else if (which.isArray()) // TypeIndex::Array
2020-12-21 01:54:04 +00:00
{
2020-12-23 19:39:02 +00:00
if (byteSizeForConstArray(data_type, column, vec_res))
return true;
const ColumnArray * col_arr = checkAndGetColumn<ColumnArray>(column.get());
return byteSizeForIntegralArray<INTEGRAL_TPL_PACK>(col_arr, data_type, column, vec_res)
|| byteSizeForStringArray(col_arr, column, vec_res)
|| byteSizeForGenericArray(col_arr, column, vec_res);
}
return false;
}
#undef INTEGRAL_TPL_PACK
template <class ...Integral>
static inline bool byteSizeForIntegralArray(const ColumnArray * col_arr, const IDataType * data_type, const ColumnPtr & column, ColumnUInt64::Container & vec_res)
{
return (byteSizeForIntegralArrayExpanded<Integral>(col_arr, data_type, column, vec_res) || ...);
}
template <class NestedType>
static bool byteSizeForIntegralArrayExpanded(const ColumnArray * col_arr, const IDataType * /*data_type*/, const ColumnPtr & /*column*/, ColumnUInt64::Container & vec_res)
{
const ColumnVector<NestedType> * col_nested = checkAndGetColumn<ColumnVector<NestedType>>(&(col_arr->getData()));
if (!col_nested)
return false;
size_t vec_size = vec_res.size();
const auto & offsets = col_arr->getOffsets();
ColumnArray::Offset prev_offset = 0;
for (size_t i = 0; i < vec_size; ++i)
{
UInt64 byte_size = 0;
byteSizeByTypeId(TypeId<NestedType>::value, byte_size);
size_t array_size = offsets[i] - prev_offset;
vec_res[i] += array_size * byte_size + sizeof(offsets[0]);
prev_offset = offsets[i];
}
return true;
}
2020-12-24 17:15:11 +00:00
static inline bool byteSizeForConstArray(const IDataType * /*data_type*/, const ColumnPtr & column, ColumnUInt64::Container & vec_res) {
const ColumnConst * col_arr = checkAndGetColumnConst<ColumnArray>(column.get());
if (!col_arr)
return false;
size_t vec_size = vec_res.size();
const UInt64 byte_size = col_arr->byteSize();
for (size_t i = 0; i < vec_size; ++i)
vec_res[i] += byte_size;
return true;
}
2020-12-23 19:39:02 +00:00
static bool byteSizeForStringArray(const ColumnArray * col_arr, const ColumnPtr & /*column*/, ColumnUInt64::Container & vec_res)
{
const ColumnString * col_nested = checkAndGetColumn<ColumnString>(&(col_arr->getData()));
if (!col_nested)
return false;
size_t vec_size = vec_res.size();
const auto & offsets = col_arr->getOffsets();
const auto & string_offsets = col_nested->getOffsets();
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < vec_size; ++i)
{
UInt64 byte_size = 0;
size_t array_size = offsets[i] - current_offset;
for (size_t j = 0; j < array_size; ++j)
{
const ColumnArray::Offset string_pos = current_offset + j == 0
? 0
: string_offsets[current_offset + j - 1];
byte_size += string_offsets[current_offset + j] - string_pos;
2020-12-21 01:54:04 +00:00
}
2020-12-23 19:39:02 +00:00
vec_res[i] += byte_size + sizeof(offsets[0]) + array_size * sizeof(string_offsets[0]);
current_offset = offsets[i];
}
return true;
}
static bool byteSizeForGenericArray(const ColumnArray * col_arr, const ColumnPtr & /*column*/, ColumnUInt64::Container & vec_res)
{
const IColumn & col_nested = col_arr->getData();
size_t vec_size = vec_res.size();
const auto & offsets = col_arr->getOffsets();
ColumnArray::Offset prev_offset = 0;
for (size_t i = 0; i < vec_size; ++i)
{
size_t array_size = offsets[i] - prev_offset;
UInt64 byte_size = 0;
for (size_t j = 0; j < array_size; ++j)
byte_size += col_nested.getDataAt(prev_offset + j).size;
vec_res[i] += byte_size;
prev_offset = offsets[i];
2020-12-21 01:54:04 +00:00
}
2020-12-23 19:39:02 +00:00
return true;
2020-12-21 01:54:04 +00:00
}
2020-12-23 19:39:02 +00:00
2020-12-21 01:54:04 +00:00
};
}
void registerFunctionByteSize(FunctionFactory & factory)
{
factory.registerFunction<FunctionByteSize>();
}
}