mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
dbms: Server: Adding nullable type support to functions manipulating arrays. [#METR-19266]
This commit is contained in:
parent
bd73b2d165
commit
5a7b98d29a
@ -1,8 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Common/Exception.h>
|
||||
#include <string>
|
||||
#include <cstring>
|
||||
|
||||
namespace DB { namespace ErrorCodes {
|
||||
|
||||
extern const int LOGICAL_ERROR;
|
||||
|
||||
}}
|
||||
|
||||
namespace detail
|
||||
{
|
||||
@ -22,7 +28,8 @@ inline bool endsWith(const std::string & s, const std::string & suffix)
|
||||
}
|
||||
|
||||
|
||||
/// strlen evaluated compile-time.
|
||||
/// With GCC, strlen is evaluated compile time if we pass it a constant
|
||||
/// string that is known at compile time.
|
||||
inline bool startsWith(const std::string & s, const char * prefix)
|
||||
{
|
||||
return detail::startsWith(s, prefix, strlen(prefix));
|
||||
@ -32,3 +39,32 @@ inline bool endsWith(const std::string & s, const char * suffix)
|
||||
{
|
||||
return detail::endsWith(s, suffix, strlen(suffix));
|
||||
}
|
||||
|
||||
/// Given an integer, return the adequate suffix for
|
||||
/// printing an ordinal number.
|
||||
template <typename T>
|
||||
std::string getOrdinalSuffix(T n)
|
||||
{
|
||||
static_assert(std::is_integral<T>::value, "Integer value required");
|
||||
|
||||
auto val = n % 10;
|
||||
|
||||
bool is_th;
|
||||
if ((val >= 1) && (val <= 3))
|
||||
is_th = (n > 10) && (((n / 10) % 10) == 1);
|
||||
else
|
||||
is_th = true;
|
||||
|
||||
if (is_th)
|
||||
return "th";
|
||||
else
|
||||
{
|
||||
switch (val)
|
||||
{
|
||||
case 1: return "st";
|
||||
case 2: return "nd";
|
||||
case 3: return "rd";
|
||||
default: throw DB::Exception{"Internal error", DB::ErrorCodes::LOGICAL_ERROR};
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -357,6 +357,7 @@ struct ArrayIndexNumImpl
|
||||
const PaddedPODArray<UInt8> * null_map_data,
|
||||
const PaddedPODArray<UInt8> * null_map_item)
|
||||
{
|
||||
/// Processing is split into 4 cases.
|
||||
if ((null_map_data == nullptr) && (null_map_item == nullptr))
|
||||
vectorCase1(data, offsets, value, result);
|
||||
else if ((null_map_data == nullptr) && (null_map_item != nullptr))
|
||||
@ -368,7 +369,7 @@ struct ArrayIndexNumImpl
|
||||
}
|
||||
};
|
||||
|
||||
/// Specialization that catches developer errors.
|
||||
/// Specialization that catches internal errors.
|
||||
template <typename T, typename IndexConv>
|
||||
struct ArrayIndexNumImpl<T, Null, IndexConv>
|
||||
{
|
||||
@ -398,11 +399,7 @@ struct ArrayIndexNumNullImpl
|
||||
result.resize(size);
|
||||
|
||||
if (null_map_data == nullptr)
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
result[i] = 0;
|
||||
return;
|
||||
}
|
||||
|
||||
const auto & null_map_ref = *null_map_data;
|
||||
|
||||
@ -440,6 +437,11 @@ struct ArrayIndexStringNullImpl
|
||||
const auto size = offsets.size();
|
||||
result.resize(size);
|
||||
|
||||
if (null_map_data == nullptr)
|
||||
return;
|
||||
|
||||
const auto & null_map_ref = *null_map_data;
|
||||
|
||||
ColumnArray::Offset_t current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
@ -449,7 +451,7 @@ struct ArrayIndexStringNullImpl
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1;
|
||||
if (null_map_data && ((*null_map_data)[k] == 1))
|
||||
if (null_map_ref[k] == 1)
|
||||
{
|
||||
if (!IndexConv::apply(j, current))
|
||||
break;
|
||||
@ -1069,20 +1071,22 @@ private:
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, ColumnUInt32::Container_t & res_values);
|
||||
bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values);
|
||||
|
||||
bool executeString(const ColumnArray * array, ColumnUInt32::Container_t & res_values);
|
||||
bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values);
|
||||
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result);
|
||||
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
const ConstColumnPlainPtrs & null_maps,
|
||||
ColumnUInt32::Container_t & res_values);
|
||||
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
const ConstColumnPlainPtrs & null_maps,
|
||||
ColumnUInt32::Container_t & res_values);
|
||||
};
|
||||
|
||||
|
@ -1335,6 +1335,7 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
|
||||
Columns array_columns(arguments.size());
|
||||
const ColumnArray::Offsets_t * offsets = nullptr;
|
||||
ConstColumnPlainPtrs data_columns(arguments.size());
|
||||
ConstColumnPlainPtrs null_maps(arguments.size());
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
@ -1346,22 +1347,34 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
|
||||
block.getByPosition(arguments[i]).column.get());
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
|
||||
+ " of " + toString(i + 1) + "-th argument of function " + getName(),
|
||||
+ " of " + toString(i + 1) + getOrdinalSuffix(i + 1) + " argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = typeid_cast<const ColumnArray *>(array_ptr.get());
|
||||
array = static_cast<const ColumnArray *>(array_ptr.get());
|
||||
}
|
||||
|
||||
array_columns[i] = array_ptr;
|
||||
|
||||
const ColumnArray::Offsets_t & offsets_i = array->getOffsets();
|
||||
if (!i)
|
||||
if (i == 0)
|
||||
offsets = &offsets_i;
|
||||
else if (offsets_i != *offsets)
|
||||
throw Exception("Lengths of all arrays passsed to " + getName() + " must be equal.",
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
|
||||
data_columns[i] = &array->getData();
|
||||
if (data_columns[i]->isNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(*data_columns[i]);
|
||||
data_columns[i] = nullable_col.getNestedColumn().get();
|
||||
null_maps[i] = nullable_col.getNullValuesByteMap().get();
|
||||
}
|
||||
else
|
||||
null_maps[i] = nullptr;
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = typeid_cast<const ColumnArray *>(array_columns[0].get());
|
||||
const ColumnArray * first_array = static_cast<const ColumnArray *>(array_columns[0].get());
|
||||
const IColumn * first_null_map = null_maps[0];
|
||||
auto res = std::make_shared<ColumnUInt32>();
|
||||
block.getByPosition(result).column = res;
|
||||
|
||||
@ -1370,32 +1383,43 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!( executeNumber<UInt8> (first_array, res_values)
|
||||
|| executeNumber<UInt16> (first_array, res_values)
|
||||
|| executeNumber<UInt32> (first_array, res_values)
|
||||
|| executeNumber<UInt64> (first_array, res_values)
|
||||
|| executeNumber<Int8> (first_array, res_values)
|
||||
|| executeNumber<Int16> (first_array, res_values)
|
||||
|| executeNumber<Int32> (first_array, res_values)
|
||||
|| executeNumber<Int64> (first_array, res_values)
|
||||
|| executeNumber<Float32> (first_array, res_values)
|
||||
|| executeNumber<Float64> (first_array, res_values)
|
||||
|| executeString (first_array, res_values)))
|
||||
if (!( executeNumber<UInt8> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt16> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt32> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<UInt64> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int8> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int16> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int32> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<Int64> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<Float32> (first_array, first_null_map, res_values)
|
||||
|| executeNumber<Float64> (first_array, first_null_map, res_values)
|
||||
|| executeString (first_array, first_null_map, res_values)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!execute128bit(*offsets, data_columns, res_values))
|
||||
executeHashed(*offsets, data_columns, res_values);
|
||||
if (!execute128bit(*offsets, data_columns, null_maps, res_values))
|
||||
executeHashed(*offsets, data_columns, null_maps, res_values);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool FunctionArrayUniq::executeNumber(const ColumnArray * array, ColumnUInt32::Container_t & res_values)
|
||||
bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
const ColumnVector<T> * nested = typeid_cast<const ColumnVector<T> *>(&array->getData());
|
||||
const IColumn * inner_col;
|
||||
|
||||
const auto & array_data = array->getData();
|
||||
if (array_data.isNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(array_data);
|
||||
inner_col = nullable_col.getNestedColumn().get();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnVector<T> * nested = typeid_cast<const ColumnVector<T> *>(inner_col);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets_t & offsets = array->getOffsets();
|
||||
@ -1404,22 +1428,32 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, ColumnUInt32::C
|
||||
typedef ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(T)> > Set;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
bool found_null = false;
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
set.insert(values[j]);
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
found_null = true;
|
||||
else
|
||||
set.insert(values[j]);
|
||||
}
|
||||
|
||||
res_values[i] = set.size();
|
||||
res_values[i] = set.size() + (found_null ? 1 : 0);
|
||||
prev_off = off;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayUniq::executeString(const ColumnArray * array, ColumnUInt32::Container_t & res_values)
|
||||
bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
const ColumnString * nested = typeid_cast<const ColumnString *>(&array->getData());
|
||||
if (!nested)
|
||||
@ -1429,16 +1463,26 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, ColumnUInt32::C
|
||||
typedef ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1 << INITIAL_SIZE_DEGREE) * sizeof(StringRef)> > Set;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
Set set;
|
||||
size_t prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
bool found_null = false;
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
set.insert(nested->getDataAt(j));
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
found_null = true;
|
||||
else
|
||||
set.insert(nested->getDataAt(j));
|
||||
}
|
||||
|
||||
res_values[i] = set.size();
|
||||
res_values[i] = set.size() + (found_null ? 1 : 0);
|
||||
prev_off = off;
|
||||
}
|
||||
return true;
|
||||
@ -1462,6 +1506,7 @@ bool FunctionArrayUniq::executeConst(Block & block, const ColumnNumbers & argume
|
||||
bool FunctionArrayUniq::execute128bit(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
const ConstColumnPlainPtrs & null_maps,
|
||||
ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
@ -1499,6 +1544,7 @@ bool FunctionArrayUniq::execute128bit(
|
||||
void FunctionArrayUniq::executeHashed(
|
||||
const ColumnArray::Offsets_t & offsets,
|
||||
const ConstColumnPlainPtrs & columns,
|
||||
const ConstColumnPlainPtrs & null_maps,
|
||||
ColumnUInt32::Container_t & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
Loading…
Reference in New Issue
Block a user