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>();
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|