2013-10-26 19:00:13 +00:00
|
|
|
#include <math.h>
|
|
|
|
|
2014-08-22 00:57:20 +00:00
|
|
|
#include <DB/Functions/FunctionFactory.h>
|
2013-10-26 19:00:13 +00:00
|
|
|
#include <DB/Functions/FunctionsArithmetic.h>
|
|
|
|
#include <DB/Functions/FunctionsMiscellaneous.h>
|
2016-09-21 11:00:04 +00:00
|
|
|
#include <DB/Functions/DataTypeTraits.h>
|
2015-12-21 13:15:13 +00:00
|
|
|
#include <DB/DataTypes/DataTypeEnum.h>
|
2016-08-17 13:38:33 +00:00
|
|
|
#include <DB/DataTypes/NullSymbol.h>
|
|
|
|
#include <DB/DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DB/Columns/ColumnNullable.h>
|
2015-12-21 13:15:13 +00:00
|
|
|
#include <ext/enumerate.hpp>
|
2013-10-26 19:00:13 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
|
|
template <typename T>
|
2016-04-15 00:33:21 +00:00
|
|
|
static void numWidthVector(const PaddedPODArray<T> & a, PaddedPODArray<UInt64> & c)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
size_t size = a.size();
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
if (a[i] >= 0)
|
|
|
|
c[i] = a[i] ? 1 + log10(a[i]) : 1;
|
2014-01-08 16:33:28 +00:00
|
|
|
else if (std::is_signed<T>::value && a[i] == std::numeric_limits<T>::min())
|
2013-10-26 19:00:13 +00:00
|
|
|
c[i] = 2 + log10(std::numeric_limits<T>::max());
|
|
|
|
else
|
|
|
|
c[i] = 2 + log10(-a[i]);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
static void numWidthConstant(T a, UInt64 & c)
|
|
|
|
{
|
|
|
|
if (a >= 0)
|
|
|
|
c = a ? 1 + log10(a) : 1;
|
2014-01-08 16:33:28 +00:00
|
|
|
else if (std::is_signed<T>::value && a == std::numeric_limits<T>::min())
|
2013-10-26 19:00:13 +00:00
|
|
|
c = 2 + log10(std::numeric_limits<T>::max());
|
|
|
|
else
|
|
|
|
c = 2 + log10(-a);
|
|
|
|
}
|
|
|
|
|
2014-12-10 11:11:41 +00:00
|
|
|
inline UInt64 floatWidth(const double x)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2015-11-30 15:15:45 +00:00
|
|
|
DoubleConverter<false>::BufferType buffer;
|
|
|
|
double_conversion::StringBuilder builder{buffer, sizeof(buffer)};
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2015-11-30 15:15:45 +00:00
|
|
|
const auto result = DoubleConverter<false>::instance().ToShortest(x, &builder);
|
2014-12-10 11:11:41 +00:00
|
|
|
|
|
|
|
if (!result)
|
2015-07-16 20:33:05 +00:00
|
|
|
throw Exception("Cannot print double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
|
|
|
|
|
|
|
|
return builder.position();
|
|
|
|
}
|
|
|
|
|
|
|
|
inline UInt64 floatWidth(const float x)
|
|
|
|
{
|
2015-11-30 15:15:45 +00:00
|
|
|
DoubleConverter<false>::BufferType buffer;
|
|
|
|
double_conversion::StringBuilder builder{buffer, sizeof(buffer)};
|
2015-07-16 20:33:05 +00:00
|
|
|
|
2015-11-30 15:15:45 +00:00
|
|
|
const auto result = DoubleConverter<false>::instance().ToShortestSingle(x, &builder);
|
2015-07-16 20:33:05 +00:00
|
|
|
|
|
|
|
if (!result)
|
|
|
|
throw Exception("Cannot print float number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2014-12-10 11:11:41 +00:00
|
|
|
return builder.position();
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
2016-04-15 00:33:21 +00:00
|
|
|
static void floatWidthVector(const PaddedPODArray<T> & a, PaddedPODArray<UInt64> & c)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
size_t size = a.size();
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
c[i] = floatWidth(a[i]);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
static void floatWidthConstant(T a, UInt64 & c)
|
|
|
|
{
|
|
|
|
c = floatWidth(a);
|
|
|
|
}
|
|
|
|
|
2016-04-15 00:33:21 +00:00
|
|
|
template <> inline void numWidthVector<Float64>(const PaddedPODArray<Float64> & a, PaddedPODArray<UInt64> & c) { floatWidthVector(a, c); }
|
|
|
|
template <> inline void numWidthVector<Float32>(const PaddedPODArray<Float32> & a, PaddedPODArray<UInt64> & c) { floatWidthVector(a, c); }
|
2013-10-26 19:00:13 +00:00
|
|
|
template <> inline void numWidthConstant<Float64>(Float64 a, UInt64 & c) { floatWidthConstant(a, c); }
|
|
|
|
template <> inline void numWidthConstant<Float32>(Float32 a, UInt64 & c) { floatWidthConstant(a, c); }
|
|
|
|
|
|
|
|
|
2016-04-15 00:33:21 +00:00
|
|
|
static inline void stringWidthVector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, PaddedPODArray<UInt64> & res)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
size_t size = offsets.size();
|
|
|
|
|
|
|
|
size_t prev_offset = 0;
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
res[i] = stringWidth(&data[prev_offset], &data[offsets[i] - 1]);
|
|
|
|
prev_offset = offsets[i];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-04-15 00:33:21 +00:00
|
|
|
static inline void stringWidthFixedVector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray<UInt64> & res)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
size_t size = data.size() / n;
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
res[i] = stringWidth(&data[i * n], &data[(i + 1) * n]);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
namespace VisibleWidth
|
|
|
|
{
|
|
|
|
template <typename T>
|
|
|
|
static bool executeConstNumber(Block & block, const ColumnPtr & column, size_t result)
|
|
|
|
{
|
2016-06-21 21:43:43 +00:00
|
|
|
if (const ColumnConst<T> * col = typeid_cast<const ColumnConst<T> *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
UInt64 res = 0;
|
|
|
|
numWidthConstant(col->getData(), res);
|
2016-05-28 05:31:36 +00:00
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(column->size(), res);
|
2013-10-26 19:00:13 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
static bool executeNumber(Block & block, const ColumnPtr & column, size_t result)
|
|
|
|
{
|
2016-06-21 21:43:43 +00:00
|
|
|
if (const ColumnVector<T> * col = typeid_cast<const ColumnVector<T> *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
auto res = std::make_shared<ColumnUInt64>(column->size());
|
2013-10-26 19:00:13 +00:00
|
|
|
block.getByPosition(result).column = res;
|
2015-03-05 05:42:42 +00:00
|
|
|
numWidthVector(col->getData(), res->getData());
|
2013-10-26 19:00:13 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
|
|
|
}
|
2015-12-21 13:15:13 +00:00
|
|
|
|
2015-12-28 17:04:29 +00:00
|
|
|
template <typename DataTypeEnum>
|
2015-12-21 13:15:13 +00:00
|
|
|
static bool executeEnum(Block & block, const DataTypePtr & type_ptr, const ColumnPtr & column, const size_t result)
|
|
|
|
{
|
2015-12-28 17:04:29 +00:00
|
|
|
if (const auto type = typeid_cast<const DataTypeEnum *>(type_ptr.get()))
|
2015-12-21 13:15:13 +00:00
|
|
|
{
|
2015-12-28 17:04:29 +00:00
|
|
|
if (const auto col = typeid_cast<const typename DataTypeEnum::ColumnType *>(column.get()))
|
2015-12-21 13:15:13 +00:00
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
const auto res = std::make_shared<ColumnUInt64>(col->size());
|
|
|
|
block.getByPosition(result).column = res;
|
2015-12-21 13:15:13 +00:00
|
|
|
|
|
|
|
const auto & in = col->getData();
|
|
|
|
auto & out = res->getData();
|
|
|
|
|
|
|
|
for (const auto & idx_num : ext::enumerate(in))
|
2015-12-30 11:53:12 +00:00
|
|
|
{
|
2016-02-01 17:30:46 +00:00
|
|
|
StringRef name = type->getNameForValue(idx_num.second);
|
|
|
|
out[idx_num.first] = stringWidth(
|
|
|
|
reinterpret_cast<const UInt8 *>(name.data),
|
|
|
|
reinterpret_cast<const UInt8 *>(name.data) + name.size);
|
2015-12-30 11:53:12 +00:00
|
|
|
}
|
2015-12-21 13:15:13 +00:00
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
2015-12-28 17:04:29 +00:00
|
|
|
else if (const auto col = typeid_cast<const typename DataTypeEnum::ConstColumnType *>(column.get()))
|
2015-12-21 13:15:13 +00:00
|
|
|
{
|
2016-02-01 17:30:46 +00:00
|
|
|
StringRef name = type->getNameForValue(col->getData());
|
2015-12-30 11:53:12 +00:00
|
|
|
|
2016-05-28 05:31:36 +00:00
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(
|
2016-02-01 17:30:46 +00:00
|
|
|
col->size(), stringWidth(
|
|
|
|
reinterpret_cast<const UInt8 *>(name.data),
|
2016-05-28 05:31:36 +00:00
|
|
|
reinterpret_cast<const UInt8 *>(name.data) + name.size));
|
2015-12-21 13:15:13 +00:00
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2016-07-06 09:47:55 +00:00
|
|
|
void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
|
2016-08-17 13:38:33 +00:00
|
|
|
{
|
|
|
|
auto & element = block.getByPosition(arguments[0]);
|
|
|
|
|
|
|
|
auto & res_element = block.getByPosition(result);
|
|
|
|
auto & res_col = res_element.column;
|
|
|
|
|
|
|
|
size_t row_count = block.rowsInFirstColumn();
|
|
|
|
|
|
|
|
if (element.column->isNull())
|
|
|
|
{
|
|
|
|
/// The input column has the Null type.
|
2016-08-17 16:31:00 +00:00
|
|
|
res_col = std::make_shared<ColumnConstUInt64>(row_count, strlen(NullSymbol::Escaped::name));
|
2016-08-17 13:38:33 +00:00
|
|
|
}
|
|
|
|
else if (element.column->isNullable())
|
|
|
|
{
|
|
|
|
/// Perform visibleWidth on a block that holds the nested column
|
|
|
|
/// of the input column.
|
|
|
|
auto & nullable_col = static_cast<ColumnNullable &>(*element.column);
|
|
|
|
auto & nested_col = nullable_col.getNestedColumn();
|
|
|
|
|
|
|
|
auto & nullable_type = static_cast<DataTypeNullable &>(*element.type);
|
|
|
|
auto & nested_type = nullable_type.getNestedType();
|
|
|
|
|
|
|
|
Block block_with_nested_col =
|
|
|
|
{
|
|
|
|
{
|
|
|
|
nested_col,
|
|
|
|
nested_type,
|
|
|
|
element.name
|
|
|
|
},
|
|
|
|
|
|
|
|
{
|
|
|
|
nullptr,
|
|
|
|
res_element.type,
|
|
|
|
""
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
perform(block_with_nested_col, {0, 1}, 1);
|
|
|
|
|
|
|
|
/// Create the result. If any row of the input column holds a NULL value,
|
|
|
|
/// we assign the corresponding row of the result the length of the NULL
|
|
|
|
/// symbol.
|
|
|
|
res_col = std::make_shared<ColumnUInt64>(row_count);
|
|
|
|
auto & res_data = static_cast<ColumnUInt64 &>(*res_col).getData();
|
|
|
|
|
|
|
|
const auto & src = static_cast<const ColumnUInt64 &>(
|
|
|
|
*block_with_nested_col.unsafeGetByPosition(1).column
|
|
|
|
).getData();
|
|
|
|
|
|
|
|
for (size_t row = 0; row < row_count; ++row)
|
|
|
|
{
|
|
|
|
if (nullable_col.isNullAt(row))
|
2016-08-17 16:31:00 +00:00
|
|
|
res_data[row] = strlen(NullSymbol::Escaped::name);
|
2016-08-17 13:38:33 +00:00
|
|
|
else
|
|
|
|
res_data[row] = src[row];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// The input column has an ordinary type.
|
|
|
|
perform(block, arguments, result);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void FunctionVisibleWidth::perform(Block & block, const ColumnNumbers & arguments, size_t result)
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
|
|
|
const DataTypePtr type = block.getByPosition(arguments[0]).type;
|
|
|
|
size_t rows = column->size();
|
|
|
|
|
2016-06-21 21:43:43 +00:00
|
|
|
if (typeid_cast<const DataTypeDate *>(type.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, strlen("0000-00-00"));
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (typeid_cast<const DataTypeDateTime *>(type.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, strlen("0000-00-00 00:00:00"));
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2015-12-28 17:04:29 +00:00
|
|
|
else if (VisibleWidth::executeEnum<DataTypeEnum8>(block, type, column, result)
|
|
|
|
|| VisibleWidth::executeEnum<DataTypeEnum16>(block, type, column, result))
|
2015-12-21 13:15:13 +00:00
|
|
|
{
|
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
else if (VisibleWidth::executeConstNumber<UInt8>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<UInt16>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<UInt32>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<UInt64>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<Int8>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<Int16>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<Int32>(block, column, result)
|
|
|
|
|| VisibleWidth::executeConstNumber<Int64>(block, column, result)
|
2015-07-16 20:33:05 +00:00
|
|
|
|| VisibleWidth::executeConstNumber<Float32>(block, column, result)
|
2013-10-26 19:00:13 +00:00
|
|
|
|| VisibleWidth::executeConstNumber<Float64>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<UInt8>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<UInt16>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<UInt32>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<UInt64>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<Int8>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<Int16>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<Int32>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<Int64>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<Float32>(block, column, result)
|
|
|
|
|| VisibleWidth::executeNumber<Float64>(block, column, result))
|
|
|
|
{
|
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (const ColumnString * col = typeid_cast<const ColumnString *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
auto res = std::make_shared<ColumnUInt64>(rows);
|
2013-10-26 19:00:13 +00:00
|
|
|
block.getByPosition(result).column = res;
|
2015-03-05 05:42:42 +00:00
|
|
|
stringWidthVector(col->getChars(), col->getOffsets(), res->getData());
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
auto res = std::make_shared<ColumnUInt64>(rows);
|
2013-10-26 19:00:13 +00:00
|
|
|
block.getByPosition(result).column = res;
|
2015-03-05 05:42:42 +00:00
|
|
|
stringWidthFixedVector(col->getChars(), col->getN(), res->getData());
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
UInt64 res = 0;
|
2016-05-28 05:31:36 +00:00
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, res);
|
2015-03-05 05:42:42 +00:00
|
|
|
stringWidthConstant(col->getData(), res);
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (const ColumnArray * col = typeid_cast<const ColumnArray *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-08-17 06:25:01 +00:00
|
|
|
/// Calculate visible width of elements of array.
|
2013-10-26 19:00:13 +00:00
|
|
|
Block nested_block;
|
2015-07-17 01:27:35 +00:00
|
|
|
ColumnWithTypeAndName nested_values;
|
2014-06-26 00:58:14 +00:00
|
|
|
nested_values.type = typeid_cast<const DataTypeArray &>(*type).getNestedType();
|
2013-10-26 19:00:13 +00:00
|
|
|
nested_values.column = col->getDataPtr();
|
|
|
|
nested_block.insert(nested_values);
|
|
|
|
|
2015-07-17 01:27:35 +00:00
|
|
|
ColumnWithTypeAndName nested_result;
|
2016-05-28 07:48:40 +00:00
|
|
|
nested_result.type = std::make_shared<DataTypeUInt64>();
|
2013-10-26 19:00:13 +00:00
|
|
|
nested_block.insert(nested_result);
|
|
|
|
|
2016-09-21 11:00:04 +00:00
|
|
|
executeImpl(nested_block, {0}, 1);
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2016-08-17 06:25:01 +00:00
|
|
|
/// Then accumulate and place into result.
|
2016-05-28 05:31:36 +00:00
|
|
|
auto res = std::make_shared<ColumnUInt64>(rows);
|
2015-03-05 05:42:42 +00:00
|
|
|
block.getByPosition(result).column = res;
|
2013-10-26 19:00:13 +00:00
|
|
|
ColumnUInt64::Container_t & vec = res->getData();
|
|
|
|
|
2016-09-21 11:00:04 +00:00
|
|
|
/// If the elements of the array are nullable, we have to check whether
|
|
|
|
/// an element is a NULL for it is not surrounded by a pair of quotes.
|
|
|
|
const PaddedPODArray<UInt8> * null_map = nullptr;
|
|
|
|
if (nested_values.type->isNullable())
|
|
|
|
{
|
|
|
|
const auto & nullable_col = static_cast<const ColumnNullable &>(col->getData());
|
|
|
|
null_map = &static_cast<const ColumnUInt8 &>(*nullable_col.getNullValuesByteMap()).getData();
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto & observed_type = DataTypeTraits::removeNullable(nested_values.type);
|
|
|
|
|
2016-08-17 06:25:01 +00:00
|
|
|
size_t additional_symbols = 0; /// Quotes.
|
2016-09-21 11:00:04 +00:00
|
|
|
if (typeid_cast<const DataTypeDate *>(observed_type.get())
|
|
|
|
|| typeid_cast<const DataTypeDateTime *>(observed_type.get())
|
|
|
|
|| typeid_cast<const DataTypeString *>(observed_type.get())
|
2016-10-19 15:00:56 +00:00
|
|
|
|| typeid_cast<const DataTypeFixedString *>(observed_type.get())
|
|
|
|
|| typeid_cast<const DataTypeEnum8 *>(observed_type.get())
|
|
|
|
|| typeid_cast<const DataTypeEnum16 *>(observed_type.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
additional_symbols = 2;
|
|
|
|
|
2016-06-21 21:43:43 +00:00
|
|
|
if (ColumnUInt64 * nested_result_column = typeid_cast<ColumnUInt64 *>(nested_block.getByPosition(1).column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
ColumnUInt64::Container_t & nested_res = nested_result_column->getData();
|
|
|
|
|
|
|
|
size_t j = 0;
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
{
|
2016-10-19 15:00:56 +00:00
|
|
|
/** If empty array, then two characters: [];
|
|
|
|
* if not, then character '[', and one extra character for each element: either ',' or ']'.
|
2013-10-26 19:00:13 +00:00
|
|
|
*/
|
2016-09-21 11:00:04 +00:00
|
|
|
vec[i] = (j == col->getOffsets()[i]) ? 2 : 1;
|
2013-10-26 19:00:13 +00:00
|
|
|
|
|
|
|
for (; j < col->getOffsets()[i]; ++j)
|
2016-09-21 11:00:04 +00:00
|
|
|
{
|
|
|
|
size_t effective_additional_symbols;
|
|
|
|
if ((null_map != nullptr) && ((*null_map)[j] == 1))
|
|
|
|
{
|
|
|
|
/// The NULL value is not quoted.
|
|
|
|
effective_additional_symbols = 0;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
effective_additional_symbols = additional_symbols;
|
|
|
|
|
|
|
|
vec[i] += 1 + effective_additional_symbols + nested_res[j];
|
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (ColumnConstUInt64 * nested_result_column = typeid_cast<ColumnConstUInt64 *>(nested_block.getByPosition(1).column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-09-21 11:00:04 +00:00
|
|
|
size_t j = 0;
|
2013-10-26 19:00:13 +00:00
|
|
|
for (size_t i = 0; i < rows; ++i)
|
2016-09-21 11:00:04 +00:00
|
|
|
{
|
|
|
|
size_t width = 0;
|
|
|
|
|
|
|
|
for (; j < col->getOffsets()[i]; ++j)
|
|
|
|
{
|
|
|
|
size_t effective_additional_symbols;
|
|
|
|
if ((null_map != nullptr) && ((*null_map)[j] == 1))
|
|
|
|
{
|
|
|
|
/// The NULL value is not quoted.
|
|
|
|
effective_additional_symbols = 0;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
effective_additional_symbols = additional_symbols;
|
|
|
|
|
|
|
|
width += 1 + effective_additional_symbols + nested_result_column->getData();
|
|
|
|
}
|
|
|
|
|
|
|
|
vec[i] = 1 + std::max(static_cast<size_t>(1), width);
|
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (const ColumnTuple * col = typeid_cast<const ColumnTuple *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
2016-08-17 06:25:01 +00:00
|
|
|
/// Calculate visible width for each nested column separately, and then accumulate.
|
2013-10-26 19:00:13 +00:00
|
|
|
Block nested_block = col->getData();
|
|
|
|
size_t columns = nested_block.columns();
|
|
|
|
|
|
|
|
FunctionPlus func_plus;
|
|
|
|
|
|
|
|
for (size_t i = 0; i < columns; ++i)
|
|
|
|
{
|
|
|
|
nested_block.getByPosition(i).type = static_cast<const DataTypeTuple &>(*type).getElements()[i];
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2016-08-17 06:25:01 +00:00
|
|
|
/** nested_block will consist of following columns:
|
2013-10-26 19:00:13 +00:00
|
|
|
* x1, x2, x3... , width1, width2, width1 + width2, width3, width1 + width2 + width3, ...
|
|
|
|
*/
|
|
|
|
|
2015-07-17 01:27:35 +00:00
|
|
|
ColumnWithTypeAndName nested_result;
|
2016-05-28 07:48:40 +00:00
|
|
|
nested_result.type = std::make_shared<DataTypeUInt64>();
|
2013-10-26 19:00:13 +00:00
|
|
|
nested_block.insert(nested_result);
|
|
|
|
|
|
|
|
ColumnNumbers nested_argument_numbers(1, i);
|
|
|
|
execute(nested_block, nested_argument_numbers, nested_block.columns() - 1);
|
|
|
|
|
|
|
|
if (i != 0)
|
|
|
|
{
|
2015-07-17 01:27:35 +00:00
|
|
|
ColumnWithTypeAndName plus_result;
|
2016-05-28 07:48:40 +00:00
|
|
|
plus_result.type = std::make_shared<DataTypeUInt64>();
|
2013-10-26 19:00:13 +00:00
|
|
|
nested_block.insert(plus_result);
|
|
|
|
|
|
|
|
ColumnNumbers plus_argument_numbers(2);
|
|
|
|
plus_argument_numbers[0] = nested_block.columns() - 3;
|
|
|
|
plus_argument_numbers[1] = nested_block.columns() - 2;
|
|
|
|
func_plus.execute(nested_block, plus_argument_numbers, nested_block.columns() - 1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-08-17 06:25:01 +00:00
|
|
|
/// Add also number of characters for quotes and commas.
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2016-08-17 06:25:01 +00:00
|
|
|
size_t additional_symbols = columns - 1; /// Commas.
|
2013-10-26 19:00:13 +00:00
|
|
|
for (size_t i = 0; i < columns; ++i)
|
|
|
|
{
|
2016-06-21 21:43:43 +00:00
|
|
|
if (typeid_cast<const DataTypeDate *>(nested_block.getByPosition(i).type.get())
|
|
|
|
|| typeid_cast<const DataTypeDateTime *>(nested_block.getByPosition(i).type.get())
|
|
|
|
|| typeid_cast<const DataTypeString *>(nested_block.getByPosition(i).type.get())
|
2016-08-17 06:25:01 +00:00
|
|
|
|| typeid_cast<const DataTypeFixedString *>(nested_block.getByPosition(i).type.get())
|
|
|
|
|| typeid_cast<const DataTypeEnum8 *>(nested_block.getByPosition(i).type.get())
|
|
|
|
|| typeid_cast<const DataTypeEnum16 *>(nested_block.getByPosition(i).type.get()))
|
|
|
|
additional_symbols += 2; /// Quotes.
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
|
2014-11-23 04:00:18 +00:00
|
|
|
ColumnPtr & nested_result_column = nested_block.getByPosition(nested_block.columns() - 1).column;
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2014-11-23 04:00:18 +00:00
|
|
|
if (nested_result_column->isConst())
|
|
|
|
{
|
|
|
|
ColumnConstUInt64 & nested_result_column_const = typeid_cast<ColumnConstUInt64 &>(*nested_result_column);
|
|
|
|
if (nested_result_column_const.size())
|
|
|
|
nested_result_column_const.getData() += 2 + additional_symbols;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
ColumnUInt64 & nested_result_column_vec = typeid_cast<ColumnUInt64 &>(*nested_result_column);
|
|
|
|
ColumnUInt64::Container_t & nested_res = nested_result_column_vec.getData();
|
|
|
|
|
|
|
|
for (size_t i = 0; i < rows; ++i)
|
|
|
|
nested_res[i] += 2 + additional_symbols;
|
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
|
2014-11-23 04:00:18 +00:00
|
|
|
block.getByPosition(result).column = nested_result_column;
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-07-10 07:24:24 +00:00
|
|
|
else if (typeid_cast<const ColumnConstArray *>(column.get())
|
|
|
|
|| typeid_cast<const ColumnConstTuple *>(column.get()))
|
2013-10-26 19:00:13 +00:00
|
|
|
{
|
|
|
|
String s;
|
|
|
|
{
|
|
|
|
WriteBufferFromString wb(s);
|
2016-07-10 07:24:24 +00:00
|
|
|
type->serializeTextEscaped(*column->cut(0, 1)->convertToFullColumnIfConst(), 0, wb);
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
|
|
|
|
2016-05-28 05:31:36 +00:00
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, s.size());
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2016-06-21 21:43:43 +00:00
|
|
|
else if (typeid_cast<const ColumnAggregateFunction *>(column.get()))
|
|
|
|
{
|
|
|
|
/** Return obviously wrong (arbitary) value for states of aggregate functions.
|
|
|
|
* Result of visibleWidth is used for presentation purposes,
|
|
|
|
* and state of aggregate function is presented as unreadable sequence of bytes,
|
|
|
|
* so using wrong calculation of its displayed width don't make presentation much worse.
|
|
|
|
*/
|
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt64>(rows, 10);
|
|
|
|
}
|
2013-10-26 19:00:13 +00:00
|
|
|
else
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
+ " of argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
|
2015-04-28 04:47:27 +00:00
|
|
|
|
2016-12-10 06:12:21 +00:00
|
|
|
void FunctionHasColumnInTable::getReturnTypeAndPrerequisitesImpl(
|
2016-11-15 10:57:11 +00:00
|
|
|
const ColumnsWithTypeAndName & arguments,
|
|
|
|
DataTypePtr & out_return_type,
|
|
|
|
ExpressionActions::Actions & out_prerequisites)
|
|
|
|
{
|
|
|
|
if (arguments.size() != number_of_arguments)
|
|
|
|
throw Exception("Function " + getName() + " requires exactly three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
|
|
static const std::string arg_pos_description[] = {"First", "Second", "Third"};
|
|
|
|
for (size_t i = 0; i < number_of_arguments; ++i)
|
|
|
|
{
|
|
|
|
const ColumnWithTypeAndName & argument = arguments[i];
|
|
|
|
|
|
|
|
const ColumnConstString * column = typeid_cast<const ColumnConstString *>(argument.column.get());
|
|
|
|
if (!column)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
arg_pos_description[i] + " argument for function " + getName() + " must be const String.",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
out_return_type = std::make_shared<DataTypeUInt8>();
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-12-10 06:12:21 +00:00
|
|
|
void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
|
2016-11-15 10:57:11 +00:00
|
|
|
{
|
|
|
|
auto get_string_from_block =
|
2016-11-16 11:30:50 +00:00
|
|
|
[&](size_t column_pos) -> const String &
|
2016-11-15 10:57:11 +00:00
|
|
|
{
|
|
|
|
DB::ColumnPtr column = block.getByPosition(column_pos).column;
|
|
|
|
const ColumnConstString * const_column = typeid_cast<const ColumnConstString *>(column.get());
|
|
|
|
return const_column->getData();
|
|
|
|
};
|
|
|
|
|
2016-11-16 11:30:50 +00:00
|
|
|
const DB::String & database_name = get_string_from_block(arguments[0]);
|
|
|
|
const DB::String & table_name = get_string_from_block(arguments[1]);
|
|
|
|
const DB::String & column_name = get_string_from_block(arguments[2]);
|
2016-11-15 10:57:11 +00:00
|
|
|
|
2016-11-16 11:30:50 +00:00
|
|
|
const DB::StoragePtr & table = global_context.getTable(database_name, table_name);
|
|
|
|
const bool has_column = table->hasColumn(column_name);
|
2016-11-15 10:57:11 +00:00
|
|
|
|
|
|
|
block.getByPosition(result).column = std::make_shared<ColumnConstUInt8>(
|
|
|
|
block.rowsInFirstColumn(), has_column);
|
|
|
|
}
|
|
|
|
|
2013-10-26 19:00:13 +00:00
|
|
|
}
|
2014-08-22 00:57:20 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
|
|
|
{
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionCurrentDatabase>();
|
|
|
|
factory.registerFunction<FunctionHostName>();
|
|
|
|
factory.registerFunction<FunctionVisibleWidth>();
|
|
|
|
factory.registerFunction<FunctionToTypeName>();
|
2016-07-10 02:14:23 +00:00
|
|
|
factory.registerFunction<FunctionToColumnTypeName>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionBlockSize>();
|
2016-05-17 18:25:47 +00:00
|
|
|
factory.registerFunction<FunctionBlockNumber>();
|
2016-07-08 21:46:18 +00:00
|
|
|
factory.registerFunction<FunctionRowNumberInBlock>();
|
|
|
|
factory.registerFunction<FunctionRowNumberInAllBlocks>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionSleep>();
|
|
|
|
factory.registerFunction<FunctionMaterialize>();
|
|
|
|
factory.registerFunction<FunctionIgnore>();
|
2016-04-15 23:10:29 +00:00
|
|
|
factory.registerFunction<FunctionIndexHint>();
|
2015-07-13 21:27:08 +00:00
|
|
|
factory.registerFunction<FunctionIdentity>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionArrayJoin>();
|
2015-06-19 02:34:47 +00:00
|
|
|
factory.registerFunction<FunctionReplicate>();
|
2014-11-12 17:23:26 +00:00
|
|
|
factory.registerFunction<FunctionBar>();
|
2016-11-15 10:57:11 +00:00
|
|
|
factory.registerFunction<FunctionHasColumnInTable>();
|
2014-11-12 17:23:26 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionTuple>();
|
|
|
|
factory.registerFunction<FunctionTupleElement>();
|
|
|
|
factory.registerFunction<FunctionIn<false, false>>();
|
|
|
|
factory.registerFunction<FunctionIn<false, true>>();
|
|
|
|
factory.registerFunction<FunctionIn<true, false>>();
|
|
|
|
factory.registerFunction<FunctionIn<true, true>>();
|
2015-04-03 13:30:00 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionIsFinite>();
|
|
|
|
factory.registerFunction<FunctionIsInfinite>();
|
2015-04-03 14:52:33 +00:00
|
|
|
factory.registerFunction<FunctionIsNaN>();
|
2015-06-03 13:11:59 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionVersion>();
|
2016-04-09 07:47:08 +00:00
|
|
|
factory.registerFunction<FunctionUptime>();
|
2015-06-06 00:28:37 +00:00
|
|
|
|
|
|
|
factory.registerFunction<FunctionRunningAccumulate>();
|
2016-09-16 22:00:30 +00:00
|
|
|
factory.registerFunction<FunctionRunningDifference>();
|
2015-06-06 00:28:37 +00:00
|
|
|
factory.registerFunction<FunctionFinalizeAggregation>();
|
2014-08-22 00:57:20 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|