mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #3085 from yandex/every_function_in_own_file_3
Every function in its own file, part 3
This commit is contained in:
commit
6e0330b3a4
@ -32,46 +32,6 @@ generate_function_register(Arithmetic
|
||||
FunctionIntExp10
|
||||
)
|
||||
|
||||
generate_function_register(Array
|
||||
FunctionArray
|
||||
FunctionArrayElement
|
||||
FunctionHas
|
||||
FunctionIndexOf
|
||||
FunctionCountEqual
|
||||
FunctionArrayEnumerate
|
||||
FunctionArrayEnumerateUniq
|
||||
FunctionArrayEnumerateDense
|
||||
FunctionArrayUniq
|
||||
FunctionArrayDistinct
|
||||
FunctionEmptyArrayUInt8
|
||||
FunctionEmptyArrayUInt16
|
||||
FunctionEmptyArrayUInt32
|
||||
FunctionEmptyArrayUInt64
|
||||
FunctionEmptyArrayInt8
|
||||
FunctionEmptyArrayInt16
|
||||
FunctionEmptyArrayInt32
|
||||
FunctionEmptyArrayInt64
|
||||
FunctionEmptyArrayFloat32
|
||||
FunctionEmptyArrayFloat64
|
||||
FunctionEmptyArrayDate
|
||||
FunctionEmptyArrayDateTime
|
||||
FunctionEmptyArrayString
|
||||
FunctionEmptyArrayToSingle
|
||||
FunctionRange
|
||||
FunctionArrayReduce
|
||||
FunctionArrayReverse
|
||||
FunctionArrayConcat
|
||||
FunctionArraySlice
|
||||
FunctionArrayPushBack
|
||||
FunctionArrayPushFront
|
||||
FunctionArrayPopBack
|
||||
FunctionArrayPopFront
|
||||
FunctionArrayHasAll
|
||||
FunctionArrayHasAny
|
||||
FunctionArrayIntersect
|
||||
FunctionArrayResize
|
||||
)
|
||||
|
||||
generate_function_register(Projection
|
||||
FunctionOneOrZero
|
||||
FunctionProject
|
||||
|
59
dbms/src/Functions/EmptyImpl.h
Normal file
59
dbms/src/Functions/EmptyImpl.h
Normal file
@ -0,0 +1,59 @@
|
||||
#include <cstring>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
template <bool negative = false>
|
||||
struct EmptyImpl
|
||||
{
|
||||
/// If the function will return constant value for FixedString data type.
|
||||
static constexpr auto is_fixed_to_constant = false;
|
||||
|
||||
static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
ColumnString::Offset prev_offset = 1;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = negative ^ (offsets[i] == prev_offset);
|
||||
prev_offset = offsets[i] + 1;
|
||||
}
|
||||
}
|
||||
|
||||
/// Only make sense if is_fixed_to_constant.
|
||||
static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt8 & /*res*/)
|
||||
{
|
||||
throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
std::vector<char> empty_chars(n);
|
||||
size_t size = data.size() / n;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n));
|
||||
}
|
||||
|
||||
static void array(const ColumnString::Offsets & offsets, PaddedPODArray<UInt8> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = negative ^ (offsets[i] == prev_offset);
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
139
dbms/src/Functions/FunctionStartsEndsWith.h
Normal file
139
dbms/src/Functions/FunctionStartsEndsWith.h
Normal file
@ -0,0 +1,139 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using namespace GatherUtils;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
struct NameStartsWith
|
||||
{
|
||||
static constexpr auto name = "startsWith";
|
||||
};
|
||||
struct NameEndsWith
|
||||
{
|
||||
static constexpr auto name = "endsWith";
|
||||
};
|
||||
|
||||
template <typename Name>
|
||||
class FunctionStartsEndsWith : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionStartsEndsWith>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isStringOrFixedString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * needle_column = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
auto col_res = ColumnVector<UInt8>::create();
|
||||
typename ColumnVector<UInt8>::Container & vec_res = col_res->getData();
|
||||
|
||||
vec_res.resize(input_rows_count);
|
||||
|
||||
if (const ColumnString * haystack = checkAndGetColumn<ColumnString>(haystack_column))
|
||||
dispatch<StringSource>(StringSource(*haystack), needle_column, vec_res);
|
||||
else if (const ColumnFixedString * haystack = checkAndGetColumn<ColumnFixedString>(haystack_column))
|
||||
dispatch<FixedStringSource>(FixedStringSource(*haystack), needle_column, vec_res);
|
||||
else if (const ColumnConst * haystack = checkAndGetColumnConst<ColumnString>(haystack_column))
|
||||
dispatch<ConstSource<StringSource>>(ConstSource<StringSource>(*haystack), needle_column, vec_res);
|
||||
else if (const ColumnConst * haystack = checkAndGetColumnConst<ColumnFixedString>(haystack_column))
|
||||
dispatch<ConstSource<FixedStringSource>>(ConstSource<FixedStringSource>(*haystack), needle_column, vec_res);
|
||||
else
|
||||
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename HaystackSource>
|
||||
void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray<UInt8> & res_data) const
|
||||
{
|
||||
if (const ColumnString * needle = checkAndGetColumn<ColumnString>(needle_column))
|
||||
execute<HaystackSource, StringSource>(haystack_source, StringSource(*needle), res_data);
|
||||
else if (const ColumnFixedString * needle = checkAndGetColumn<ColumnFixedString>(needle_column))
|
||||
execute<HaystackSource, FixedStringSource>(haystack_source, FixedStringSource(*needle), res_data);
|
||||
else if (const ColumnConst * needle = checkAndGetColumnConst<ColumnString>(needle_column))
|
||||
execute<HaystackSource, ConstSource<StringSource>>(haystack_source, ConstSource<StringSource>(*needle), res_data);
|
||||
else if (const ColumnConst * needle = checkAndGetColumnConst<ColumnFixedString>(needle_column))
|
||||
execute<HaystackSource, ConstSource<FixedStringSource>>(haystack_source, ConstSource<FixedStringSource>(*needle), res_data);
|
||||
else
|
||||
throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
template <typename HaystackSource, typename NeedleSource>
|
||||
static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray<UInt8> & res_data)
|
||||
{
|
||||
size_t row_num = 0;
|
||||
|
||||
while (!haystack_source.isEnd())
|
||||
{
|
||||
auto haystack = haystack_source.getWhole();
|
||||
auto needle = needle_source.getWhole();
|
||||
|
||||
if (needle.size > haystack.size)
|
||||
{
|
||||
res_data[row_num] = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if constexpr (std::is_same_v<Name, NameStartsWith>)
|
||||
{
|
||||
res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size);
|
||||
}
|
||||
else /// endsWith
|
||||
{
|
||||
res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size);
|
||||
}
|
||||
}
|
||||
|
||||
haystack_source.next();
|
||||
needle_source.next();
|
||||
++row_num;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
101
dbms/src/Functions/FunctionStringOrArrayToT.h
Normal file
101
dbms/src/Functions/FunctionStringOrArrayToT.h
Normal file
@ -0,0 +1,101 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename Impl, typename Name, typename ResultType>
|
||||
class FunctionStringOrArrayToT : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionStringOrArrayToT>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0])
|
||||
&& !isArray(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeNumber<ResultType>>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->size());
|
||||
Impl::vector(col->getChars(), col->getOffsets(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
if (Impl::is_fixed_to_constant)
|
||||
{
|
||||
ResultType res = 0;
|
||||
Impl::vector_fixed_to_constant(col->getChars(), col->getN(), res);
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col->size(), toField(res));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->size());
|
||||
Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
}
|
||||
else if (const ColumnArray * col = checkAndGetColumn<ColumnArray>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
|
||||
typename ColumnVector<ResultType>::Container & vec_res = col_res->getData();
|
||||
vec_res.resize(col->size());
|
||||
Impl::array(col->getOffsets(), vec_res);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
76
dbms/src/Functions/FunctionStringToString.h
Normal file
76
dbms/src/Functions/FunctionStringToString.h
Normal file
@ -0,0 +1,76 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
template <typename Impl, typename Name, bool is_injective = false>
|
||||
class FunctionStringToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionStringToString>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return is_injective;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(col->getN());
|
||||
Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -1,8 +1,9 @@
|
||||
#include <Functions/FunctionsConditional.h>
|
||||
#include <Functions/FunctionsArray.h>
|
||||
#include <Functions/FunctionsTransform.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <vector>
|
||||
|
||||
@ -251,22 +252,15 @@ DataTypePtr FunctionCaseWithExpression::getReturnTypeImpl(const DataTypes & args
|
||||
/// See the comments in executeImpl() to understand why we actually have to
|
||||
/// get the return type of a transform function.
|
||||
|
||||
/// Get the return types of the arrays that we pass to the transform function.
|
||||
ColumnsWithTypeAndName src_array_types;
|
||||
ColumnsWithTypeAndName dst_array_types;
|
||||
/// Get the types of the arrays that we pass to the transform function.
|
||||
DataTypes src_array_types;
|
||||
DataTypes dst_array_types;
|
||||
|
||||
for (size_t i = 1; i < (args.size() - 1); ++i)
|
||||
{
|
||||
if ((i % 2) != 0)
|
||||
src_array_types.push_back({nullptr, args[i], {}});
|
||||
else
|
||||
dst_array_types.push_back({nullptr, args[i], {}});
|
||||
}
|
||||
for (size_t i = 1; i < args.size() - 1; ++i)
|
||||
((i % 2) ? src_array_types : dst_array_types).push_back(args[i]);
|
||||
|
||||
FunctionArray fun_array{context};
|
||||
|
||||
DataTypePtr src_array_type = fun_array.getReturnType(src_array_types);
|
||||
DataTypePtr dst_array_type = fun_array.getReturnType(dst_array_types);
|
||||
DataTypePtr src_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(src_array_types));
|
||||
DataTypePtr dst_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(dst_array_types));
|
||||
|
||||
/// Finally get the return type of the transform function.
|
||||
FunctionTransform fun_transform;
|
||||
@ -291,29 +285,31 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers
|
||||
|
||||
/// Create the arrays required by the transform function.
|
||||
ColumnNumbers src_array_args;
|
||||
ColumnsWithTypeAndName src_array_types;
|
||||
ColumnsWithTypeAndName src_array_elems;
|
||||
DataTypes src_array_types;
|
||||
|
||||
ColumnNumbers dst_array_args;
|
||||
ColumnsWithTypeAndName dst_array_types;
|
||||
ColumnsWithTypeAndName dst_array_elems;
|
||||
DataTypes dst_array_types;
|
||||
|
||||
for (size_t i = 1; i < (args.size() - 1); ++i)
|
||||
{
|
||||
if ((i % 2) != 0)
|
||||
if (i % 2)
|
||||
{
|
||||
src_array_args.push_back(args[i]);
|
||||
src_array_types.push_back(block.getByPosition(args[i]));
|
||||
src_array_elems.push_back(block.getByPosition(args[i]));
|
||||
src_array_types.push_back(block.getByPosition(args[i]).type);
|
||||
}
|
||||
else
|
||||
{
|
||||
dst_array_args.push_back(args[i]);
|
||||
dst_array_types.push_back(block.getByPosition(args[i]));
|
||||
dst_array_elems.push_back(block.getByPosition(args[i]));
|
||||
dst_array_types.push_back(block.getByPosition(args[i]).type);
|
||||
}
|
||||
}
|
||||
|
||||
FunctionArray fun_array{context};
|
||||
|
||||
DataTypePtr src_array_type = fun_array.getReturnType(src_array_types);
|
||||
DataTypePtr dst_array_type = fun_array.getReturnType(dst_array_types);
|
||||
DataTypePtr src_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(src_array_types));
|
||||
DataTypePtr dst_array_type = std::make_shared<DataTypeArray>(getLeastSupertype(dst_array_types));
|
||||
|
||||
Block temp_block = block;
|
||||
|
||||
@ -323,8 +319,10 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers
|
||||
size_t dst_array_pos = temp_block.columns();
|
||||
temp_block.insert({nullptr, dst_array_type, ""});
|
||||
|
||||
fun_array.execute(temp_block, src_array_args, src_array_pos, input_rows_count);
|
||||
fun_array.execute(temp_block, dst_array_args, dst_array_pos, input_rows_count);
|
||||
auto fun_array = FunctionFactory::instance().get("array", context);
|
||||
|
||||
fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count);
|
||||
fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count);
|
||||
|
||||
/// Execute transform.
|
||||
FunctionTransform fun_transform;
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -1,204 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
#include <Poco/Unicode.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
/** String functions
|
||||
*
|
||||
* length, empty, notEmpty,
|
||||
* concat, substring, lower, upper, reverse
|
||||
* lengthUTF8, substringUTF8, lowerUTF8, upperUTF8, reverseUTF8
|
||||
*
|
||||
* s -> UInt8: empty, notEmpty
|
||||
* s -> UInt64: length, lengthUTF8
|
||||
* s -> s: lower, upper, lowerUTF8, upperUTF8, reverse, reverseUTF8
|
||||
* s, s -> s: concat
|
||||
* s, c1, c2 -> s: substring, substringUTF8
|
||||
* s, c1, c2, s2 -> s: replace, replaceUTF8
|
||||
*
|
||||
* The search functions for strings and regular expressions are located separately.
|
||||
* URL functions are located separately.
|
||||
* String encoding functions, converting to other types are located separately.
|
||||
*
|
||||
* The functions length, empty, notEmpty, reverse also work with arrays.
|
||||
*/
|
||||
|
||||
|
||||
/// xor or do nothing
|
||||
template <bool>
|
||||
UInt8 xor_or_identity(const UInt8 c, const int mask)
|
||||
{
|
||||
return c ^ mask;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline UInt8 xor_or_identity<false>(const UInt8 c, const int)
|
||||
{
|
||||
return c;
|
||||
}
|
||||
|
||||
/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array
|
||||
template <bool to_lower>
|
||||
inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst)
|
||||
{
|
||||
if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
|
||||
{
|
||||
/// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x10);
|
||||
}
|
||||
else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
|
||||
{
|
||||
/// ѐёђѓєѕіїјљњћќѝўџ
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x10);
|
||||
}
|
||||
else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
|
||||
{
|
||||
/// А-П
|
||||
*dst++ = *src++;
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
|
||||
}
|
||||
else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu))
|
||||
{
|
||||
/// а-п
|
||||
*dst++ = *src++;
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
|
||||
}
|
||||
else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu))
|
||||
{
|
||||
/// Р-Я
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
|
||||
}
|
||||
else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
|
||||
{
|
||||
/// р-я
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case.
|
||||
* Note: It is assumed that after the character is converted to another case,
|
||||
* the length of its multibyte sequence in UTF-8 does not change.
|
||||
* Otherwise, the behavior is undefined.
|
||||
*/
|
||||
template <char not_case_lower_bound,
|
||||
char not_case_upper_bound,
|
||||
int to_case(int),
|
||||
void cyrillic_to_case(const UInt8 *&, UInt8 *&)>
|
||||
struct LowerUpperUTF8Impl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars_t & res_data,
|
||||
ColumnString::Offsets & res_offsets);
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data);
|
||||
|
||||
static void constant(const std::string & data, std::string & res_data);
|
||||
|
||||
/** Converts a single code point starting at `src` to desired case, storing result starting at `dst`.
|
||||
* `src` and `dst` are incremented by corresponding sequence lengths. */
|
||||
static void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst);
|
||||
|
||||
private:
|
||||
static constexpr auto ascii_upper_bound = '\x7f';
|
||||
static constexpr auto flip_case_mask = 'A' ^ 'a';
|
||||
|
||||
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst);
|
||||
};
|
||||
|
||||
|
||||
template <typename Impl, typename Name, bool is_injective = false>
|
||||
class FunctionStringToString : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionStringToString>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return is_injective;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(col->getN());
|
||||
Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
struct NameLowerUTF8
|
||||
{
|
||||
static constexpr auto name = "lowerUTF8";
|
||||
};
|
||||
struct NameUpperUTF8
|
||||
{
|
||||
static constexpr auto name = "upperUTF8";
|
||||
};
|
||||
|
||||
|
||||
using FunctionLowerUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>;
|
||||
using FunctionUpperUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>;
|
||||
|
||||
}
|
@ -7,7 +7,7 @@
|
||||
#include <common/find_first_symbols.h>
|
||||
#include <common/StringRef.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/FunctionsString.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include <Functions/FunctionsStringArray.h>
|
||||
#include <port/memrchr.h>
|
||||
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsVisitParam.h>
|
||||
#include <Functions/FunctionsStringSearch.h>
|
||||
#include <Functions/FunctionsString.h>
|
||||
#include <Functions/FunctionsURL.h>
|
||||
|
||||
|
||||
@ -26,7 +25,6 @@ using FunctionVisitParamExtractRaw = FunctionsStringSearchToString<ExtractParamT
|
||||
using FunctionVisitParamExtractString = FunctionsStringSearchToString<ExtractParamToStringImpl<ExtractString>, NameVisitParamExtractString>;
|
||||
|
||||
|
||||
|
||||
void registerFunctionsVisitParam(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionVisitParamHas>();
|
||||
|
@ -37,15 +37,12 @@ std::unique_ptr<IArraySink> createArraySink(ColumnArray & col, size_t column_siz
|
||||
void concat(const std::vector<std::unique_ptr<IArraySource>> & sources, IArraySink & sink);
|
||||
|
||||
void sliceFromLeftConstantOffsetUnbounded(IArraySource & src, IArraySink & sink, size_t offset);
|
||||
|
||||
void sliceFromLeftConstantOffsetBounded(IArraySource & src, IArraySink & sink, size_t offset, ssize_t length);
|
||||
|
||||
void sliceFromRightConstantOffsetUnbounded(IArraySource & src, IArraySink & sink, size_t offset);
|
||||
|
||||
void sliceFromRightConstantOffsetBounded(IArraySource & src, IArraySink & sink, size_t offset, ssize_t length);
|
||||
|
||||
void sliceDynamicOffsetUnbounded(IArraySource & src, IArraySink & sink, const IColumn & offset_column);
|
||||
|
||||
void sliceDynamicOffsetBounded(IArraySource & src, IArraySink & sink, const IColumn & offset_column, const IColumn & length_column);
|
||||
|
||||
void sliceHas(IArraySource & first, IArraySource & second, bool all, ColumnUInt8 & result);
|
||||
@ -55,5 +52,6 @@ void push(IArraySource & array_source, IValueSource & value_source, IArraySink &
|
||||
void resizeDynamicSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, const IColumn & size_column);
|
||||
|
||||
void resizeConstantSize(IArraySource & array_source, IValueSource & value_source, IArraySink & sink, ssize_t size);
|
||||
|
||||
}
|
||||
|
||||
|
67
dbms/src/Functions/LowerUpperImpl.h
Normal file
67
dbms/src/Functions/LowerUpperImpl.h
Normal file
@ -0,0 +1,67 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <char not_case_lower_bound, char not_case_upper_bound>
|
||||
struct LowerUpperImpl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars_t & res_data,
|
||||
ColumnString::Offsets & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
array(data.data(), data.data() + data.size(), res_data.data());
|
||||
}
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(data.data(), data.data() + data.size(), res_data.data());
|
||||
}
|
||||
|
||||
private:
|
||||
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
|
||||
{
|
||||
const auto flip_case_mask = 'A' ^ 'a';
|
||||
|
||||
#if __SSE2__
|
||||
const auto bytes_sse = sizeof(__m128i);
|
||||
const auto src_end_sse = src_end - (src_end - src) % bytes_sse;
|
||||
|
||||
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
|
||||
const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1);
|
||||
const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask);
|
||||
|
||||
for (; src < src_end_sse; src += bytes_sse, dst += bytes_sse)
|
||||
{
|
||||
/// load 16 sequential 8-bit characters
|
||||
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
|
||||
|
||||
/// find which 8-bit sequences belong to range [case_lower_bound, case_upper_bound]
|
||||
const auto is_not_case
|
||||
= _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound));
|
||||
|
||||
/// keep `flip_case_mask` only where necessary, zero out elsewhere
|
||||
const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case);
|
||||
|
||||
/// flip case by applying calculated mask
|
||||
const auto cased_chars = _mm_xor_si128(chars, xor_mask);
|
||||
|
||||
/// store result back to destination
|
||||
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
|
||||
}
|
||||
#endif
|
||||
|
||||
for (; src < src_end; ++src, ++dst)
|
||||
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
|
||||
*dst = *src ^ flip_case_mask;
|
||||
else
|
||||
*dst = *src;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
229
dbms/src/Functions/LowerUpperUTF8Impl.h
Normal file
229
dbms/src/Functions/LowerUpperUTF8Impl.h
Normal file
@ -0,0 +1,229 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
|
||||
#if __SSE2__
|
||||
#include <emmintrin.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
/// xor or do nothing
|
||||
template <bool>
|
||||
UInt8 xor_or_identity(const UInt8 c, const int mask)
|
||||
{
|
||||
return c ^ mask;
|
||||
}
|
||||
|
||||
template <>
|
||||
inline UInt8 xor_or_identity<false>(const UInt8 c, const int)
|
||||
{
|
||||
return c;
|
||||
}
|
||||
|
||||
/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array
|
||||
template <bool to_lower>
|
||||
inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst)
|
||||
{
|
||||
if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
|
||||
{
|
||||
/// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x10);
|
||||
}
|
||||
else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
|
||||
{
|
||||
/// ѐёђѓєѕіїјљњћќѝўџ
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x10);
|
||||
}
|
||||
else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
|
||||
{
|
||||
/// А-П
|
||||
*dst++ = *src++;
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
|
||||
}
|
||||
else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu))
|
||||
{
|
||||
/// а-п
|
||||
*dst++ = *src++;
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
|
||||
}
|
||||
else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu))
|
||||
{
|
||||
/// Р-Я
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
|
||||
}
|
||||
else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
|
||||
{
|
||||
/// р-я
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
|
||||
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case.
|
||||
* Note: It is assumed that after the character is converted to another case,
|
||||
* the length of its multibyte sequence in UTF-8 does not change.
|
||||
* Otherwise, the behavior is undefined.
|
||||
*/
|
||||
template <char not_case_lower_bound,
|
||||
char not_case_upper_bound,
|
||||
int to_case(int),
|
||||
void cyrillic_to_case(const UInt8 *&, UInt8 *&)>
|
||||
struct LowerUpperUTF8Impl
|
||||
{
|
||||
static void vector(
|
||||
const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars_t & res_data,
|
||||
ColumnString::Offsets & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
array(data.data(), data.data() + data.size(), res_data.data());
|
||||
}
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(data.data(), data.data() + data.size(), res_data.data());
|
||||
}
|
||||
|
||||
static void constant(const std::string & data, std::string & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
array(reinterpret_cast<const UInt8 *>(data.data()),
|
||||
reinterpret_cast<const UInt8 *>(data.data() + data.size()),
|
||||
reinterpret_cast<UInt8 *>(res_data.data()));
|
||||
}
|
||||
|
||||
/** Converts a single code point starting at `src` to desired case, storing result starting at `dst`.
|
||||
* `src` and `dst` are incremented by corresponding sequence lengths. */
|
||||
static void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst)
|
||||
{
|
||||
if (src[0] <= ascii_upper_bound)
|
||||
{
|
||||
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
|
||||
*dst++ = *src++ ^ flip_case_mask;
|
||||
else
|
||||
*dst++ = *src++;
|
||||
}
|
||||
else if (src + 1 < src_end
|
||||
&& ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
|
||||
{
|
||||
cyrillic_to_case(src, dst);
|
||||
}
|
||||
else if (src + 1 < src_end && src[0] == 0xC2u)
|
||||
{
|
||||
/// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
|
||||
*dst++ = *src++;
|
||||
*dst++ = *src++;
|
||||
}
|
||||
else if (src + 2 < src_end && src[0] == 0xE2u)
|
||||
{
|
||||
/// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
|
||||
*dst++ = *src++;
|
||||
*dst++ = *src++;
|
||||
*dst++ = *src++;
|
||||
}
|
||||
else
|
||||
{
|
||||
static const Poco::UTF8Encoding utf8;
|
||||
|
||||
if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src))
|
||||
{
|
||||
src += chars;
|
||||
dst += chars;
|
||||
}
|
||||
else
|
||||
{
|
||||
++src;
|
||||
++dst;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
static constexpr auto ascii_upper_bound = '\x7f';
|
||||
static constexpr auto flip_case_mask = 'A' ^ 'a';
|
||||
|
||||
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst)
|
||||
{
|
||||
#if __SSE2__
|
||||
const auto bytes_sse = sizeof(__m128i);
|
||||
auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
|
||||
|
||||
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
|
||||
const auto v_zero = _mm_setzero_si128();
|
||||
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
|
||||
const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1);
|
||||
const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask);
|
||||
|
||||
while (src < src_end_sse)
|
||||
{
|
||||
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
|
||||
|
||||
/// check for ASCII
|
||||
const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero);
|
||||
const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii);
|
||||
|
||||
/// ASCII
|
||||
if (mask_is_not_ascii == 0)
|
||||
{
|
||||
const auto is_not_case
|
||||
= _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound));
|
||||
const auto mask_is_not_case = _mm_movemask_epi8(is_not_case);
|
||||
|
||||
/// everything in correct case ASCII
|
||||
if (mask_is_not_case == 0)
|
||||
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars);
|
||||
else
|
||||
{
|
||||
/// ASCII in mixed case
|
||||
/// keep `flip_case_mask` only where necessary, zero out elsewhere
|
||||
const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case);
|
||||
|
||||
/// flip case by applying calculated mask
|
||||
const auto cased_chars = _mm_xor_si128(chars, xor_mask);
|
||||
|
||||
/// store result back to destination
|
||||
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
|
||||
}
|
||||
|
||||
src += bytes_sse;
|
||||
dst += bytes_sse;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// UTF-8
|
||||
const auto expected_end = src + bytes_sse;
|
||||
|
||||
while (src < expected_end)
|
||||
toCase(src, src_end, dst);
|
||||
|
||||
/// adjust src_end_sse by pushing it forward or backward
|
||||
const auto diff = src - expected_end;
|
||||
if (diff != 0)
|
||||
{
|
||||
if (src_end_sse + diff < src_end)
|
||||
src_end_sse += diff;
|
||||
else
|
||||
src_end_sse -= bytes_sse - diff;
|
||||
}
|
||||
}
|
||||
}
|
||||
#endif
|
||||
/// handle remaining symbols
|
||||
while (src < src_end)
|
||||
toCase(src, src_end, dst);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
116
dbms/src/Functions/appendTrailingCharIfAbsent.cpp
Normal file
116
dbms/src/Functions/appendTrailingCharIfAbsent.cpp
Normal file
@ -0,0 +1,116 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
class FunctionAppendTrailingCharIfAbsent : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "appendTrailingCharIfAbsent";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionAppendTrailingCharIfAbsent>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const auto & column = block.getByPosition(arguments[0]).column;
|
||||
const auto & column_char = block.getByPosition(arguments[1]).column;
|
||||
|
||||
if (!checkColumnConst<ColumnString>(column_char.get()))
|
||||
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
String trailing_char_str = static_cast<const ColumnConst &>(*column_char).getValue<String>();
|
||||
|
||||
if (trailing_char_str.size() != 1)
|
||||
throw Exception{"Second argument of function " + getName() + " must be a one-character string", ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
if (const auto col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
const auto & src_data = col->getChars();
|
||||
const auto & src_offsets = col->getOffsets();
|
||||
|
||||
auto & dst_data = col_res->getChars();
|
||||
auto & dst_offsets = col_res->getOffsets();
|
||||
|
||||
const auto size = src_offsets.size();
|
||||
dst_data.resize(src_data.size() + size);
|
||||
dst_offsets.resize(size);
|
||||
|
||||
ColumnString::Offset src_offset{};
|
||||
ColumnString::Offset dst_offset{};
|
||||
|
||||
for (const auto i : ext::range(0, size))
|
||||
{
|
||||
const auto src_length = src_offsets[i] - src_offset;
|
||||
memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length);
|
||||
src_offset = src_offsets[i];
|
||||
dst_offset += src_length;
|
||||
|
||||
if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front())
|
||||
{
|
||||
dst_data[dst_offset - 1] = trailing_char_str.front();
|
||||
dst_data[dst_offset] = 0;
|
||||
++dst_offset;
|
||||
}
|
||||
|
||||
dst_offsets[i] = dst_offset;
|
||||
}
|
||||
|
||||
dst_data.resize_assume_reserved(dst_offset);
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionAppendTrailingCharIfAbsent>();
|
||||
}
|
||||
|
||||
}
|
119
dbms/src/Functions/array.cpp
Normal file
119
dbms/src/Functions/array.cpp
Normal file
@ -0,0 +1,119 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// array(c1, c2, ...) - create an array.
|
||||
class FunctionArray : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "array";
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FunctionArray>(context);
|
||||
}
|
||||
|
||||
FunctionArray(const Context & context)
|
||||
: context(context)
|
||||
{
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype(arguments));
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
size_t num_elements = arguments.size();
|
||||
|
||||
if (num_elements == 0)
|
||||
{
|
||||
/// We should return constant empty array.
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
const DataTypePtr & return_type = block.getByPosition(result).type;
|
||||
const DataTypePtr & elem_type = static_cast<const DataTypeArray &>(*return_type).getNestedType();
|
||||
|
||||
size_t block_size = input_rows_count;
|
||||
|
||||
/** If part of columns have not same type as common type of all elements of array,
|
||||
* then convert them to common type.
|
||||
* If part of columns are constants,
|
||||
* then convert them to full columns.
|
||||
*/
|
||||
|
||||
Columns columns_holder(num_elements);
|
||||
const IColumn * columns[num_elements];
|
||||
|
||||
for (size_t i = 0; i < num_elements; ++i)
|
||||
{
|
||||
const auto & arg = block.getByPosition(arguments[i]);
|
||||
|
||||
ColumnPtr preprocessed_column = arg.column;
|
||||
|
||||
if (!arg.type->equals(*elem_type))
|
||||
preprocessed_column = castColumn(arg, elem_type, context);
|
||||
|
||||
if (ColumnPtr materialized_column = preprocessed_column->convertToFullColumnIfConst())
|
||||
preprocessed_column = materialized_column;
|
||||
|
||||
columns_holder[i] = std::move(preprocessed_column);
|
||||
columns[i] = columns_holder[i].get();
|
||||
}
|
||||
|
||||
/// Create and fill the result array.
|
||||
|
||||
auto out = ColumnArray::create(elem_type->createColumn());
|
||||
IColumn & out_data = out->getData();
|
||||
IColumn::Offsets & out_offsets = out->getOffsets();
|
||||
|
||||
out_data.reserve(block_size * num_elements);
|
||||
out_offsets.resize(block_size);
|
||||
|
||||
IColumn::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < block_size; ++i)
|
||||
{
|
||||
for (size_t j = 0; j < num_elements; ++j)
|
||||
out_data.insertFrom(*columns[j], i);
|
||||
|
||||
current_offset += num_elements;
|
||||
out_offsets[i] = current_offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(out);
|
||||
}
|
||||
|
||||
private:
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool addField(DataTypePtr type_res, const Field & f, Array & arr) const;
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionArray(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArray>();
|
||||
}
|
||||
|
||||
}
|
117
dbms/src/Functions/arrayConcat.cpp
Normal file
117
dbms/src/Functions/arrayConcat.cpp
Normal file
@ -0,0 +1,117 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/// arrayConcat(arr1, ...) - concatenate arrays.
|
||||
class FunctionArrayConcat : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayConcat";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayConcat>(context); }
|
||||
FunctionArrayConcat(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.empty())
|
||||
throw Exception{"Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
for (auto i : ext::range(0, arguments.size()))
|
||||
{
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return getLeastSupertype(arguments);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const DataTypePtr & return_type = block.getByPosition(result).type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
size_t rows = input_rows_count;
|
||||
size_t num_args = arguments.size();
|
||||
|
||||
Columns preprocessed_columns(num_args);
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]);
|
||||
ColumnPtr preprocessed_column = arg.column;
|
||||
|
||||
if (!arg.type->equals(*return_type))
|
||||
preprocessed_column = castColumn(arg, return_type, context);
|
||||
|
||||
preprocessed_columns[i] = std::move(preprocessed_column);
|
||||
}
|
||||
|
||||
std::vector<std::unique_ptr<GatherUtils::IArraySource>> sources;
|
||||
|
||||
for (auto & argument_column : preprocessed_columns)
|
||||
{
|
||||
bool is_const = false;
|
||||
|
||||
if (auto argument_column_const = typeid_cast<const ColumnConst *>(argument_column.get()))
|
||||
{
|
||||
is_const = true;
|
||||
argument_column = argument_column_const->getDataColumnPtr();
|
||||
}
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(argument_column.get()))
|
||||
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows));
|
||||
else
|
||||
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
auto sink = GatherUtils::createArraySink(typeid_cast<ColumnArray &>(*result_column), rows);
|
||||
GatherUtils::concat(sources, *sink);
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionArrayConcat(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayConcat>();
|
||||
}
|
||||
|
||||
}
|
302
dbms/src/Functions/arrayDistinct.cpp
Normal file
302
dbms/src/Functions/arrayDistinct.cpp
Normal file
@ -0,0 +1,302 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/HashTable/ClearableHashSet.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/// Find different elements in an array.
|
||||
class FunctionArrayDistinct : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayDistinct";
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionArrayDistinct>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument for function " + getName() + " must be array but it "
|
||||
" has type " + arguments[0]->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
auto nested_type = removeNullable(array_type->getNestedType());
|
||||
|
||||
return std::make_shared<DataTypeArray>(nested_type);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col);
|
||||
|
||||
bool executeString(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col);
|
||||
|
||||
void executeHashed(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col);
|
||||
};
|
||||
|
||||
|
||||
void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[0]).column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
|
||||
const auto & return_type = block.getByPosition(result).type;
|
||||
|
||||
auto res_ptr = return_type->createColumn();
|
||||
ColumnArray & res = static_cast<ColumnArray &>(*res_ptr);
|
||||
|
||||
const IColumn & src_data = array->getData();
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
IColumn & res_data = res.getData();
|
||||
ColumnArray::Offsets & res_offsets = res.getOffsets();
|
||||
|
||||
const ColumnNullable * nullable_col = nullptr;
|
||||
|
||||
const IColumn * inner_col;
|
||||
|
||||
if (src_data.isColumnNullable())
|
||||
{
|
||||
nullable_col = static_cast<const ColumnNullable *>(&src_data);
|
||||
inner_col = &nullable_col->getNestedColumn();
|
||||
}
|
||||
else
|
||||
{
|
||||
inner_col = &src_data;
|
||||
}
|
||||
|
||||
if (!(executeNumber<UInt8>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<UInt16>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<UInt32>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<UInt64>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<Int8>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<Int16>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<Int32>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<Int64>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<Float32>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeNumber<Float64>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|
||||
|| executeString(*inner_col, offsets, res_data, res_offsets, nullable_col)))
|
||||
executeHashed(*inner_col, offsets, res_data, res_offsets, nullable_col);
|
||||
|
||||
block.getByPosition(result).column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool FunctionArrayDistinct::executeNumber(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col)
|
||||
{
|
||||
const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data);
|
||||
|
||||
if (!src_data_concrete)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
const PaddedPODArray<T> & values = src_data_concrete->getData();
|
||||
PaddedPODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
|
||||
|
||||
const PaddedPODArray<UInt8> * src_null_map = nullptr;
|
||||
|
||||
if (nullable_col)
|
||||
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
|
||||
|
||||
using Set = ClearableHashSet<T,
|
||||
DefaultHash<T>,
|
||||
HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
Set set;
|
||||
|
||||
ColumnArray::Offset prev_src_offset = 0;
|
||||
ColumnArray::Offset res_offset = 0;
|
||||
|
||||
for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
|
||||
ColumnArray::Offset curr_src_offset = src_offsets[i];
|
||||
for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j)
|
||||
{
|
||||
if (nullable_col && (*src_null_map)[j])
|
||||
continue;
|
||||
|
||||
if (set.find(values[j]) == set.end())
|
||||
{
|
||||
res_data.emplace_back(values[j]);
|
||||
set.insert(values[j]);
|
||||
}
|
||||
}
|
||||
|
||||
res_offset += set.size();
|
||||
res_offsets.emplace_back(res_offset);
|
||||
|
||||
prev_src_offset = curr_src_offset;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayDistinct::executeString(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col)
|
||||
{
|
||||
const ColumnString * src_data_concrete = checkAndGetColumn<ColumnString>(&src_data);
|
||||
|
||||
if (!src_data_concrete)
|
||||
return false;
|
||||
|
||||
ColumnString & res_data_column_string = typeid_cast<ColumnString &>(res_data_col);
|
||||
|
||||
using Set = ClearableHashSet<StringRef,
|
||||
StringRefHash,
|
||||
HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * src_null_map = nullptr;
|
||||
|
||||
if (nullable_col)
|
||||
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
|
||||
|
||||
Set set;
|
||||
|
||||
ColumnArray::Offset prev_src_offset = 0;
|
||||
ColumnArray::Offset res_offset = 0;
|
||||
|
||||
for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
|
||||
ColumnArray::Offset curr_src_offset = src_offsets[i];
|
||||
for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j)
|
||||
{
|
||||
if (nullable_col && (*src_null_map)[j])
|
||||
continue;
|
||||
|
||||
StringRef str_ref = src_data_concrete->getDataAt(j);
|
||||
|
||||
if (set.find(str_ref) == set.end())
|
||||
{
|
||||
set.insert(str_ref);
|
||||
res_data_column_string.insertData(str_ref.data, str_ref.size);
|
||||
}
|
||||
}
|
||||
|
||||
res_offset += set.size();
|
||||
res_offsets.emplace_back(res_offset);
|
||||
|
||||
prev_src_offset = curr_src_offset;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void FunctionArrayDistinct::executeHashed(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col)
|
||||
{
|
||||
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * src_null_map = nullptr;
|
||||
|
||||
if (nullable_col)
|
||||
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
|
||||
|
||||
Set set;
|
||||
|
||||
ColumnArray::Offset prev_src_offset = 0;
|
||||
ColumnArray::Offset res_offset = 0;
|
||||
|
||||
for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
|
||||
ColumnArray::Offset curr_src_offset = src_offsets[i];
|
||||
for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j)
|
||||
{
|
||||
if (nullable_col && (*src_null_map)[j])
|
||||
continue;
|
||||
|
||||
UInt128 hash;
|
||||
SipHash hash_function;
|
||||
src_data.updateHashWithValue(j, hash_function);
|
||||
hash_function.get128(reinterpret_cast<char *>(&hash));
|
||||
|
||||
if (set.find(hash) == set.end())
|
||||
{
|
||||
set.insert(hash);
|
||||
res_data_col.insertFrom(src_data, j);
|
||||
}
|
||||
}
|
||||
|
||||
res_offset += set.size();
|
||||
res_offsets.emplace_back(res_offset);
|
||||
|
||||
prev_src_offset = curr_src_offset;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionArrayDistinct(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayDistinct>();
|
||||
}
|
||||
|
||||
}
|
886
dbms/src/Functions/arrayElement.cpp
Normal file
886
dbms/src/Functions/arrayElement.cpp
Normal file
@ -0,0 +1,886 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ZERO_ARRAY_OR_TUPLE_INDEX;
|
||||
}
|
||||
|
||||
namespace ArrayImpl
|
||||
{
|
||||
class NullMapBuilder;
|
||||
}
|
||||
|
||||
/** arrayElement(arr, i) - get the array element by index. If index is not constant and out of range - return default value of data type.
|
||||
* The index begins with 1. Also, the index can be negative - then it is counted from the end of the array.
|
||||
*/
|
||||
class FunctionArrayElement : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayElement";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
void perform(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
|
||||
|
||||
template <typename DataType>
|
||||
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType, typename DataType>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
|
||||
size_t input_rows_count);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
|
||||
|
||||
/** For a tuple array, the function is evaluated component-wise for each element of the tuple.
|
||||
*/
|
||||
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count);
|
||||
};
|
||||
|
||||
|
||||
namespace ArrayImpl
|
||||
{
|
||||
|
||||
class NullMapBuilder
|
||||
{
|
||||
public:
|
||||
operator bool() const { return src_null_map; }
|
||||
bool operator!() const { return !src_null_map; }
|
||||
|
||||
void initSource(const UInt8 * src_null_map_)
|
||||
{
|
||||
src_null_map = src_null_map_;
|
||||
}
|
||||
|
||||
void initSink(size_t size)
|
||||
{
|
||||
auto sink = ColumnUInt8::create(size);
|
||||
sink_null_map = sink->getData().data();
|
||||
sink_null_map_holder = std::move(sink);
|
||||
}
|
||||
|
||||
void update(size_t from)
|
||||
{
|
||||
sink_null_map[index] = bool(src_null_map && src_null_map[from]);
|
||||
++index;
|
||||
}
|
||||
|
||||
void update()
|
||||
{
|
||||
sink_null_map[index] = bool(src_null_map);
|
||||
++index;
|
||||
}
|
||||
|
||||
ColumnPtr getNullMapColumnPtr() && { return std::move(sink_null_map_holder); }
|
||||
|
||||
private:
|
||||
const UInt8 * src_null_map = nullptr;
|
||||
UInt8 * sink_null_map = nullptr;
|
||||
MutableColumnPtr sink_null_map_holder;
|
||||
size_t index = 0;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
struct ArrayElementNumImpl
|
||||
{
|
||||
/** Implementation for constant index.
|
||||
* If negative = false - index is from beginning of array, started from 0.
|
||||
* If negative = true - index is from end of array, started from 0.
|
||||
*/
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const PaddedPODArray<T> & data, const ColumnArray::Offsets & offsets,
|
||||
const ColumnArray::Offset index,
|
||||
PaddedPODArray<T> & result, ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.resize(size);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
if (index < array_size)
|
||||
{
|
||||
size_t j = !negative ? (current_offset + index) : (offsets[i] - index - 1);
|
||||
result[i] = data[j];
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else
|
||||
{
|
||||
result[i] = T();
|
||||
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/** Implementation for non-constant index.
|
||||
*/
|
||||
template <typename TIndex>
|
||||
static void vector(
|
||||
const PaddedPODArray<T> & data, const ColumnArray::Offsets & offsets,
|
||||
const PaddedPODArray<TIndex> & indices,
|
||||
PaddedPODArray<T> & result, ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.resize(size);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
TIndex index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
{
|
||||
size_t j = current_offset + index - 1;
|
||||
result[i] = data[j];
|
||||
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
{
|
||||
size_t j = offsets[i] + index;
|
||||
result[i] = data[j];
|
||||
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else
|
||||
{
|
||||
result[i] = T();
|
||||
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct ArrayElementStringImpl
|
||||
{
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets,
|
||||
const ColumnArray::Offset index,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result_offsets.resize(size);
|
||||
result_data.reserve(data.size());
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
ColumnArray::Offset current_result_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
if (index < array_size)
|
||||
{
|
||||
size_t adjusted_index = !negative ? index : (array_size - index - 1);
|
||||
|
||||
size_t j = current_offset + adjusted_index;
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
|
||||
ColumnArray::Offset string_pos = current_offset == 0 && adjusted_index == 0
|
||||
? 0
|
||||
: string_offsets[current_offset + adjusted_index - 1];
|
||||
|
||||
ColumnArray::Offset string_size = string_offsets[current_offset + adjusted_index] - string_pos;
|
||||
|
||||
result_data.resize(current_result_offset + string_size);
|
||||
memcpySmallAllowReadWriteOverflow15(&result_data[current_result_offset], &data[string_pos], string_size);
|
||||
current_result_offset += string_size;
|
||||
result_offsets[i] = current_result_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Insert an empty row.
|
||||
result_data.resize(current_result_offset + 1);
|
||||
result_data[current_result_offset] = 0;
|
||||
current_result_offset += 1;
|
||||
result_offsets[i] = current_result_offset;
|
||||
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/** Implementation for non-constant index.
|
||||
*/
|
||||
template <typename TIndex>
|
||||
static void vector(
|
||||
const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets,
|
||||
const PaddedPODArray<TIndex> & indices,
|
||||
ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result_offsets.resize(size);
|
||||
result_data.reserve(data.size());
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
ColumnArray::Offset current_result_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
size_t adjusted_index; /// index in array from zero
|
||||
|
||||
TIndex index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
adjusted_index = index - 1;
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
adjusted_index = array_size + index;
|
||||
else
|
||||
adjusted_index = array_size; /// means no element should be taken
|
||||
|
||||
if (adjusted_index < array_size)
|
||||
{
|
||||
size_t j = current_offset + adjusted_index;
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
|
||||
ColumnArray::Offset string_pos = current_offset == 0 && adjusted_index == 0
|
||||
? 0
|
||||
: string_offsets[current_offset + adjusted_index - 1];
|
||||
|
||||
ColumnArray::Offset string_size = string_offsets[current_offset + adjusted_index] - string_pos;
|
||||
|
||||
result_data.resize(current_result_offset + string_size);
|
||||
memcpySmallAllowReadWriteOverflow15(&result_data[current_result_offset], &data[string_pos], string_size);
|
||||
current_result_offset += string_size;
|
||||
result_offsets[i] = current_result_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Insert empty string
|
||||
result_data.resize(current_result_offset + 1);
|
||||
result_data[current_result_offset] = 0;
|
||||
current_result_offset += 1;
|
||||
result_offsets[i] = current_result_offset;
|
||||
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/// Generic implementation for other nested types.
|
||||
struct ArrayElementGenericImpl
|
||||
{
|
||||
template <bool negative>
|
||||
static void vectorConst(
|
||||
const IColumn & data, const ColumnArray::Offsets & offsets,
|
||||
const ColumnArray::Offset index,
|
||||
IColumn & result, ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.reserve(size);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
if (index < array_size)
|
||||
{
|
||||
size_t j = !negative ? current_offset + index : offsets[i] - index - 1;
|
||||
result.insertFrom(data, j);
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else
|
||||
{
|
||||
result.insertDefault();
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
/** Implementation for non-constant index.
|
||||
*/
|
||||
template <typename TIndex>
|
||||
static void vector(
|
||||
const IColumn & data, const ColumnArray::Offsets & offsets,
|
||||
const PaddedPODArray<TIndex> & indices,
|
||||
IColumn & result, ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.reserve(size);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
|
||||
TIndex index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
{
|
||||
size_t j = current_offset + index - 1;
|
||||
result.insertFrom(data, j);
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
{
|
||||
size_t j = offsets[i] + index;
|
||||
result.insertFrom(data, j);
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else
|
||||
{
|
||||
result.insertDefault();
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
FunctionPtr FunctionArrayElement::create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionArrayElement>();
|
||||
}
|
||||
|
||||
|
||||
template <typename DataType>
|
||||
bool FunctionArrayElement::executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnVector<DataType> * col_nested = checkAndGetColumn<ColumnVector<DataType>>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
auto col_res = ColumnVector<DataType>::create();
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementNumImpl<DataType>::template vectorConst<false>(
|
||||
col_nested->getData(), col_array->getOffsets(), safeGet<UInt64>(index) - 1, col_res->getData(), builder);
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementNumImpl<DataType>::template vectorConst<true>(
|
||||
col_nested->getData(), col_array->getOffsets(), -safeGet<Int64>(index) - 1, col_res->getData(), builder);
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType, typename DataType>
|
||||
bool FunctionArrayElement::executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnVector<DataType> * col_nested = checkAndGetColumn<ColumnVector<DataType>>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
auto col_res = ColumnVector<DataType>::create();
|
||||
|
||||
ArrayElementNumImpl<DataType>::template vector<IndexType>(
|
||||
col_nested->getData(), col_array->getOffsets(), indices, col_res->getData(), builder);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnString * col_nested = checkAndGetColumn<ColumnString>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementStringImpl::vectorConst<false>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
safeGet<UInt64>(index) - 1,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets(),
|
||||
builder);
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementStringImpl::vectorConst<true>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
-safeGet<Int64>(index) - 1,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets(),
|
||||
builder);
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnString * col_nested = checkAndGetColumn<ColumnString>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
ArrayElementStringImpl::vector<IndexType>(
|
||||
col_nested->getChars(),
|
||||
col_array->getOffsets(),
|
||||
col_nested->getOffsets(),
|
||||
indices,
|
||||
col_res->getChars(),
|
||||
col_res->getOffsets(),
|
||||
builder);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const auto & col_nested = col_array->getData();
|
||||
auto col_res = col_nested.cloneEmpty();
|
||||
|
||||
if (index.getType() == Field::Types::UInt64)
|
||||
ArrayElementGenericImpl::vectorConst<false>(
|
||||
col_nested, col_array->getOffsets(), safeGet<UInt64>(index) - 1, *col_res, builder);
|
||||
else if (index.getType() == Field::Types::Int64)
|
||||
ArrayElementGenericImpl::vectorConst<true>(
|
||||
col_nested, col_array->getOffsets(), -safeGet<Int64>(index) - 1, *col_res, builder);
|
||||
else
|
||||
throw Exception("Illegal type of array index", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const auto & col_nested = col_array->getData();
|
||||
auto col_res = col_nested.cloneEmpty();
|
||||
|
||||
ArrayElementGenericImpl::vector<IndexType>(
|
||||
col_nested, col_array->getOffsets(), indices, *col_res, builder);
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
|
||||
size_t input_rows_count)
|
||||
{
|
||||
const ColumnArray * col_array = checkAndGetColumnConstData<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
auto res = block.getByPosition(result).type->createColumn();
|
||||
|
||||
size_t rows = input_rows_count;
|
||||
const IColumn & array_elements = col_array->getData();
|
||||
size_t array_size = array_elements.size();
|
||||
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
IndexType index = indices[i];
|
||||
if (index > 0 && static_cast<size_t>(index) <= array_size)
|
||||
{
|
||||
size_t j = index - 1;
|
||||
res->insertFrom(array_elements, j);
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else if (index < 0 && static_cast<size_t>(-index) <= array_size)
|
||||
{
|
||||
size_t j = array_size + index;
|
||||
res->insertFrom(array_elements, j);
|
||||
if (builder)
|
||||
builder.update(j);
|
||||
}
|
||||
else
|
||||
{
|
||||
res->insertDefault();
|
||||
if (builder)
|
||||
builder.update();
|
||||
}
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count)
|
||||
{
|
||||
auto index = checkAndGetColumn<ColumnVector<IndexType>>(block.getByPosition(arguments[1]).column.get());
|
||||
|
||||
if (!index)
|
||||
return false;
|
||||
|
||||
const auto & index_data = index->getData();
|
||||
|
||||
if (builder)
|
||||
builder.initSink(index_data.size());
|
||||
|
||||
if (!( executeNumber<IndexType, UInt8>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt16>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt32>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, UInt64>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int8>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int16>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int32>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Int64>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Float32>(block, arguments, result, index_data, builder)
|
||||
|| executeNumber<IndexType, Float64>(block, arguments, result, index_data, builder)
|
||||
|| executeConst<IndexType>(block, arguments, result, index_data, builder, input_rows_count)
|
||||
|| executeString<IndexType>(block, arguments, result, index_data, builder)
|
||||
|| executeGeneric<IndexType>(block, arguments, result, index_data, builder)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
const ColumnArray * col_array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
|
||||
|
||||
if (!col_array)
|
||||
return false;
|
||||
|
||||
const ColumnTuple * col_nested = typeid_cast<const ColumnTuple *>(&col_array->getData());
|
||||
|
||||
if (!col_nested)
|
||||
return false;
|
||||
|
||||
const Columns & tuple_columns = col_nested->getColumns();
|
||||
size_t tuple_size = tuple_columns.size();
|
||||
|
||||
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(
|
||||
*typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType()).getElements();
|
||||
|
||||
/** We will calculate the function for the tuple of the internals of the array.
|
||||
* To do this, create a temporary block.
|
||||
* It will consist of the following columns
|
||||
* - the index of the array to be taken;
|
||||
* - an array of the first elements of the tuples;
|
||||
* - the result of taking the elements by the index for an array of the first elements of the tuples;
|
||||
* - array of the second elements of the tuples;
|
||||
* - result of taking elements by index for an array of second elements of tuples;
|
||||
* ...
|
||||
*/
|
||||
Block block_of_temporary_results;
|
||||
block_of_temporary_results.insert(block.getByPosition(arguments[1]));
|
||||
|
||||
/// results of taking elements by index for arrays from each element of the tuples;
|
||||
Columns result_tuple_columns;
|
||||
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
ColumnWithTypeAndName array_of_tuple_section;
|
||||
array_of_tuple_section.column = ColumnArray::create(tuple_columns[i], col_array->getOffsetsPtr());
|
||||
array_of_tuple_section.type = std::make_shared<DataTypeArray>(tuple_types[i]);
|
||||
block_of_temporary_results.insert(array_of_tuple_section);
|
||||
|
||||
ColumnWithTypeAndName array_elements_of_tuple_section;
|
||||
array_elements_of_tuple_section.type = getReturnTypeImpl(
|
||||
{block_of_temporary_results.getByPosition(i * 2 + 1).type, block_of_temporary_results.getByPosition(0).type});
|
||||
block_of_temporary_results.insert(array_elements_of_tuple_section);
|
||||
|
||||
executeImpl(block_of_temporary_results, ColumnNumbers{i * 2 + 1, 0}, i * 2 + 2, input_rows_count);
|
||||
|
||||
result_tuple_columns.emplace_back(std::move(block_of_temporary_results.getByPosition(i * 2 + 2).column));
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnTuple::create(result_tuple_columns);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
String FunctionArrayElement::getName() const
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments) const
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isInteger(arguments[1]))
|
||||
throw Exception("Second argument for function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return array_type->getNestedType();
|
||||
}
|
||||
|
||||
void FunctionArrayElement::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
/// Check nullability.
|
||||
bool is_array_of_nullable = false;
|
||||
|
||||
const ColumnArray * col_array = nullptr;
|
||||
const ColumnArray * col_const_array = nullptr;
|
||||
|
||||
col_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
if (col_array)
|
||||
is_array_of_nullable = col_array->getData().isColumnNullable();
|
||||
else
|
||||
{
|
||||
col_const_array = checkAndGetColumnConstData<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
if (col_const_array)
|
||||
is_array_of_nullable = col_const_array->getData().isColumnNullable();
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
if (!is_array_of_nullable)
|
||||
{
|
||||
ArrayImpl::NullMapBuilder builder;
|
||||
perform(block, arguments, result, builder, input_rows_count);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Perform initializations.
|
||||
ArrayImpl::NullMapBuilder builder;
|
||||
Block source_block;
|
||||
|
||||
const auto & input_type = typeid_cast<const DataTypeNullable &>(*typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType()).getNestedType();
|
||||
const auto & tmp_ret_type = typeid_cast<const DataTypeNullable &>(*block.getByPosition(result).type).getNestedType();
|
||||
|
||||
if (col_array)
|
||||
{
|
||||
const auto & nullable_col = typeid_cast<const ColumnNullable &>(col_array->getData());
|
||||
const auto & nested_col = nullable_col.getNestedColumnPtr();
|
||||
|
||||
/// Put nested_col inside a ColumnArray.
|
||||
source_block =
|
||||
{
|
||||
{
|
||||
ColumnArray::create(nested_col, col_array->getOffsetsPtr()),
|
||||
std::make_shared<DataTypeArray>(input_type),
|
||||
""
|
||||
},
|
||||
block.getByPosition(arguments[1]),
|
||||
{
|
||||
nullptr,
|
||||
tmp_ret_type,
|
||||
""
|
||||
}
|
||||
};
|
||||
|
||||
builder.initSource(nullable_col.getNullMapData().data());
|
||||
}
|
||||
else
|
||||
{
|
||||
/// ColumnConst(ColumnArray(ColumnNullable(...)))
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(col_const_array->getData());
|
||||
const auto & nested_col = nullable_col.getNestedColumnPtr();
|
||||
|
||||
source_block =
|
||||
{
|
||||
{
|
||||
ColumnConst::create(ColumnArray::create(nested_col, col_const_array->getOffsetsPtr()), input_rows_count),
|
||||
std::make_shared<DataTypeArray>(input_type),
|
||||
""
|
||||
},
|
||||
block.getByPosition(arguments[1]),
|
||||
{
|
||||
nullptr,
|
||||
tmp_ret_type,
|
||||
""
|
||||
}
|
||||
};
|
||||
|
||||
builder.initSource(nullable_col.getNullMapData().data());
|
||||
}
|
||||
|
||||
perform(source_block, {0, 1}, 2, builder, input_rows_count);
|
||||
|
||||
/// Store the result.
|
||||
const ColumnWithTypeAndName & source_col = source_block.getByPosition(2);
|
||||
ColumnWithTypeAndName & dest_col = block.getByPosition(result);
|
||||
dest_col.column = ColumnNullable::create(source_col.column, builder ? std::move(builder).getNullMapColumnPtr() : ColumnUInt8::create());
|
||||
}
|
||||
}
|
||||
|
||||
void FunctionArrayElement::perform(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count)
|
||||
{
|
||||
if (executeTuple(block, arguments, result, input_rows_count))
|
||||
{
|
||||
}
|
||||
else if (!block.getByPosition(arguments[1]).column->isColumnConst())
|
||||
{
|
||||
if (!(executeArgument<UInt8>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt16>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt32>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<UInt64>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int8>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int16>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int32>(block, arguments, result, builder, input_rows_count)
|
||||
|| executeArgument<Int64>(block, arguments, result, builder, input_rows_count)))
|
||||
throw Exception("Second argument for function " + getName() + " must must have UInt or Int type.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
Field index = (*block.getByPosition(arguments[1]).column)[0];
|
||||
|
||||
if (builder)
|
||||
builder.initSink(input_rows_count);
|
||||
|
||||
if (index == UInt64(0))
|
||||
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
|
||||
if (!( executeNumberConst<UInt8>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt16>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt32>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<UInt64>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int8>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int16>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int32>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Int64>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Float32>(block, arguments, result, index, builder)
|
||||
|| executeNumberConst<Float64>(block, arguments, result, index, builder)
|
||||
|| executeStringConst (block, arguments, result, index, builder)
|
||||
|| executeGenericConst (block, arguments, result, index, builder)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionArrayElement(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayElement>();
|
||||
}
|
||||
|
||||
}
|
85
dbms/src/Functions/arrayEnumerate.cpp
Normal file
85
dbms/src/Functions/arrayEnumerate.cpp
Normal file
@ -0,0 +1,85 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
/// arrayEnumerate(arr) - Returns the array [1,2,3,..., length(arr)]
|
||||
class FunctionArrayEnumerate : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayEnumerate";
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionArrayEnumerate>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
|
||||
{
|
||||
if (const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
auto res_nested = ColumnUInt32::create();
|
||||
|
||||
ColumnUInt32::Container & res_values = res_nested->getData();
|
||||
res_values.resize(array->getData().size());
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
res_values[j] = j - prev_off + 1;
|
||||
|
||||
prev_off = off;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), array->getOffsetsPtr());
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionArrayEnumerate(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayEnumerate>();
|
||||
}
|
||||
|
||||
}
|
22
dbms/src/Functions/arrayEnumerateDense.cpp
Normal file
22
dbms/src/Functions/arrayEnumerateDense.cpp
Normal file
@ -0,0 +1,22 @@
|
||||
#include <Functions/arrayEnumerateExtended.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class FunctionArrayEnumerateDense : public FunctionArrayEnumerateExtended<FunctionArrayEnumerateDense>
|
||||
{
|
||||
using Base = FunctionArrayEnumerateExtended<FunctionArrayEnumerateDense>;
|
||||
public:
|
||||
static constexpr auto name = "arrayEnumerateDense";
|
||||
using Base::create;
|
||||
};
|
||||
|
||||
void registerFunctionArrayEnumerateDense(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayEnumerateDense>();
|
||||
}
|
||||
|
||||
}
|
484
dbms/src/Functions/arrayEnumerateExtended.h
Normal file
484
dbms/src/Functions/arrayEnumerateExtended.h
Normal file
@ -0,0 +1,484 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
class FunctionArrayEnumerateUniq;
|
||||
class FunctionArrayEnumerateDense;
|
||||
|
||||
template <typename Derived>
|
||||
class FunctionArrayEnumerateExtended : public IFunction
|
||||
{
|
||||
public:
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<Derived>(); }
|
||||
|
||||
String getName() const override { return Derived::name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() == 0)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be at least 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("All arguments for function " + getName() + " must be arrays but argument " +
|
||||
toString(i + 1) + " has type " + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns);
|
||||
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values);
|
||||
};
|
||||
|
||||
|
||||
template <typename Derived>
|
||||
void FunctionArrayEnumerateExtended<Derived>::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
const ColumnArray::Offsets * offsets = nullptr;
|
||||
ColumnRawPtrs data_columns;
|
||||
data_columns.reserve(arguments.size());
|
||||
|
||||
bool has_nullable_columns = false;
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(
|
||||
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(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
}
|
||||
|
||||
const ColumnArray::Offsets & offsets_i = array->getOffsets();
|
||||
if (i == 0)
|
||||
offsets = &offsets_i;
|
||||
else if (offsets_i != *offsets)
|
||||
throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.",
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
|
||||
auto * array_data = &array->getData();
|
||||
data_columns.push_back(array_data);
|
||||
}
|
||||
|
||||
size_t num_columns = data_columns.size();
|
||||
ColumnRawPtrs original_data_columns(num_columns);
|
||||
ColumnRawPtrs null_maps(num_columns);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
original_data_columns[i] = data_columns[i];
|
||||
|
||||
if (data_columns[i]->isColumnNullable())
|
||||
{
|
||||
has_nullable_columns = true;
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(*data_columns[i]);
|
||||
data_columns[i] = &nullable_col.getNestedColumn();
|
||||
null_maps[i] = &nullable_col.getNullMapColumn();
|
||||
}
|
||||
else
|
||||
null_maps[i] = nullptr;
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments.at(0)).column.get());
|
||||
const IColumn * first_null_map = null_maps[0];
|
||||
auto res_nested = ColumnUInt32::create();
|
||||
|
||||
ColumnUInt32::Container & res_values = res_nested->getData();
|
||||
if (!offsets->empty())
|
||||
res_values.resize(offsets->back());
|
||||
|
||||
if (num_columns == 1)
|
||||
{
|
||||
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)))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), first_array->getOffsetsPtr());
|
||||
}
|
||||
|
||||
|
||||
template <typename Derived>
|
||||
template <typename T>
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
{
|
||||
const IColumn * inner_col;
|
||||
|
||||
const auto & array_data = array->getData();
|
||||
if (array_data.isColumnNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnVector<T> * nested = checkAndGetColumn<ColumnVector<T>>(inner_col);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container & values = nested->getData();
|
||||
|
||||
using ValuesToIndices = ClearableHashMap<T, UInt32, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
|
||||
{
|
||||
// Unique
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
UInt32 null_count = 0;
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
res_values[j] = ++null_count;
|
||||
else
|
||||
res_values[j] = ++indices[values[j]];
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// Dense
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
size_t rank = 0;
|
||||
UInt32 null_index = 0;
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
{
|
||||
if (!null_index)
|
||||
null_index = ++rank;
|
||||
res_values[j] = null_index;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & idx = indices[values[j]];
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Derived>
|
||||
bool FunctionArrayEnumerateExtended<Derived>::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
{
|
||||
const IColumn * inner_col;
|
||||
|
||||
const auto & array_data = array->getData();
|
||||
if (array_data.isColumnNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnString * nested = checkAndGetColumn<ColumnString>(inner_col);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
size_t prev_off = 0;
|
||||
using ValuesToIndices = ClearableHashMap<StringRef, UInt32, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
ValuesToIndices indices;
|
||||
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
|
||||
{
|
||||
// Unique
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
UInt32 null_count = 0;
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
res_values[j] = ++null_count;
|
||||
else
|
||||
res_values[j] = ++indices[nested->getDataAt(j)];
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// Dense
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
size_t rank = 0;
|
||||
UInt32 null_index = 0;
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
{
|
||||
if (!null_index)
|
||||
null_index = ++rank;
|
||||
res_values[j] = null_index;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto & idx = indices[nested->getDataAt(j)];
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Derived>
|
||||
bool FunctionArrayEnumerateExtended<Derived>::execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
size_t keys_bytes = 0;
|
||||
Sizes key_sizes(count);
|
||||
|
||||
for (size_t j = 0; j < count; ++j)
|
||||
{
|
||||
if (!columns[j]->isFixedAndContiguous())
|
||||
return false;
|
||||
key_sizes[j] = columns[j]->sizeOfValueIfFixed();
|
||||
keys_bytes += key_sizes[j];
|
||||
}
|
||||
if (has_nullable_columns)
|
||||
keys_bytes += std::tuple_size<KeysNullMap<UInt128>>::value;
|
||||
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
|
||||
{
|
||||
// Unique
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (has_nullable_columns)
|
||||
{
|
||||
KeysNullMap<UInt128> bitmap{};
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (null_maps[i])
|
||||
{
|
||||
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[i]).getData();
|
||||
if (null_map[j] == 1)
|
||||
{
|
||||
size_t bucket = i / 8;
|
||||
size_t offset = i % 8;
|
||||
bitmap[bucket] |= UInt8(1) << offset;
|
||||
}
|
||||
}
|
||||
}
|
||||
res_values[j] = ++indices[packFixed<UInt128>(j, count, columns, key_sizes, bitmap)];
|
||||
}
|
||||
else
|
||||
res_values[j] = ++indices[packFixed<UInt128>(j, count, columns, key_sizes)];
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// Dense
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
size_t off = offsets[i];
|
||||
size_t rank = 0;
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (has_nullable_columns)
|
||||
{
|
||||
KeysNullMap<UInt128> bitmap{};
|
||||
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (null_maps[i])
|
||||
{
|
||||
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[i]).getData();
|
||||
if (null_map[j] == 1)
|
||||
{
|
||||
size_t bucket = i / 8;
|
||||
size_t offset = i % 8;
|
||||
bitmap[bucket] |= UInt8(1) << offset;
|
||||
}
|
||||
}
|
||||
}
|
||||
auto &idx = indices[packFixed<UInt128>(j, count, columns, key_sizes, bitmap)];
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto &idx = indices[packFixed<UInt128>(j, count, columns, key_sizes)];;
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Derived>
|
||||
void FunctionArrayEnumerateExtended<Derived>::executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
using ValuesToIndices = ClearableHashMap<UInt128, UInt32, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
ValuesToIndices indices;
|
||||
size_t prev_off = 0;
|
||||
if constexpr (std::is_same_v<Derived, FunctionArrayEnumerateUniq>)
|
||||
{
|
||||
// Unique
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
size_t off = offsets[i];
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
res_values[j] = ++indices[hash128(j, count, columns)];
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
// Dense
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
indices.clear();
|
||||
size_t off = offsets[i];
|
||||
size_t rank = 0;
|
||||
for (size_t j = prev_off; j < off; ++j)
|
||||
{
|
||||
auto & idx = indices[hash128(j, count, columns)];
|
||||
if (!idx)
|
||||
idx = ++rank;
|
||||
res_values[j] = idx;
|
||||
}
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
28
dbms/src/Functions/arrayEnumerateUniq.cpp
Normal file
28
dbms/src/Functions/arrayEnumerateUniq.cpp
Normal file
@ -0,0 +1,28 @@
|
||||
#include <Functions/arrayEnumerateExtended.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** arrayEnumerateUniq(arr)
|
||||
* - outputs an array parallel (having same size) to this, where for each element specified
|
||||
* how many times this element was encountered before (including this element) among elements with the same value.
|
||||
* For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
|
||||
* arrayEnumerateUniq(arr1, arr2...)
|
||||
* - for tuples from elements in the corresponding positions in several arrays.
|
||||
*/
|
||||
class FunctionArrayEnumerateUniq : public FunctionArrayEnumerateExtended<FunctionArrayEnumerateUniq>
|
||||
{
|
||||
using Base = FunctionArrayEnumerateExtended<FunctionArrayEnumerateUniq>;
|
||||
public:
|
||||
static constexpr auto name = "arrayEnumerateUniq";
|
||||
using Base::create;
|
||||
};
|
||||
|
||||
void registerFunctionArrayEnumerateUniq(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayEnumerateUniq>();
|
||||
}
|
||||
|
||||
}
|
@ -1,27 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
#include <ext/range.h>
|
||||
|
||||
#include <unordered_map>
|
||||
#include <numeric>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -30,141 +17,10 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/** Array functions:
|
||||
*
|
||||
* array(c1, c2, ...) - create an array.
|
||||
* arrayElement(arr, i) - get the array element by index. If index is not constant and out of range - return default value of data type.
|
||||
* The index begins with 1. Also, the index can be negative - then it is counted from the end of the array.
|
||||
* has(arr, x) - whether there is an element x in the array.
|
||||
* indexOf(arr, x) - returns the index of the element x (starting with 1), if it exists in the array, or 0 if it is not.
|
||||
* arrayEnumerate(arr) - Returns the array [1,2,3,..., length(arr)]
|
||||
*
|
||||
* arrayUniq(arr) - counts the number of different elements in the array,
|
||||
* arrayUniq(arr1, arr2, ...) - counts the number of different tuples from the elements in the corresponding positions in several arrays.
|
||||
*
|
||||
* arrayDistinct(arr) - retrun different elements in an array
|
||||
*
|
||||
* arrayEnumerateUniq(arr)
|
||||
* - outputs an array parallel (having same size) to this, where for each element specified
|
||||
* how many times this element was encountered before (including this element) among elements with the same value.
|
||||
* For example: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1]
|
||||
* arrayEnumerateUniq(arr1, arr2...)
|
||||
* - for tuples from elements in the corresponding positions in several arrays.
|
||||
*
|
||||
* emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value.
|
||||
*
|
||||
* arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...`
|
||||
* If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function.
|
||||
*
|
||||
* arrayConcat(arr1, ...) - concatenate arrays.
|
||||
*
|
||||
* arraySlice(arr, offset, length) - make slice of array. Offsets and length may be < 0 or Null
|
||||
* - if offset < 0, indexation from right element
|
||||
* - if length < 0, length = len(array) - (positive_index(offset) - 1) + length
|
||||
* indexation:
|
||||
* [ 1, 2, 3, 4, 5, 6]
|
||||
* [-6, -5, -4, -3, -2, -1]
|
||||
* examples:
|
||||
* arraySlice([1, 2, 3, 4, 5, 6], -4, 2) -> [3, 4]
|
||||
* arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5] (6 - (2 - 1) + (-1) = 4)
|
||||
* arraySlice([1, 2, 3, 4, 5, 6], -5, -1) = arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5]
|
||||
*
|
||||
* arrayPushBack(arr, x), arrayPushFront(arr, x)
|
||||
* arrayPopBack(arr), arrayPopFront(arr)
|
||||
*/
|
||||
|
||||
|
||||
class FunctionArray : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "array";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
FunctionArray(const Context & context);
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
String getName() const override;
|
||||
|
||||
bool addField(DataTypePtr type_res, const Field & f, Array & arr) const;
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
namespace ArrayImpl
|
||||
{
|
||||
class NullMapBuilder;
|
||||
}
|
||||
|
||||
class FunctionArrayElement : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayElement";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
void perform(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
|
||||
|
||||
template <typename DataType>
|
||||
bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType, typename DataType>
|
||||
bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray<IndexType> & indices,
|
||||
ArrayImpl::NullMapBuilder & builder);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
const PaddedPODArray <IndexType> & indices, ArrayImpl::NullMapBuilder & builder,
|
||||
size_t input_rows_count);
|
||||
|
||||
template <typename IndexType>
|
||||
bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
ArrayImpl::NullMapBuilder & builder, size_t input_rows_count);
|
||||
|
||||
/** For a tuple array, the function is evaluated component-wise for each element of the tuple.
|
||||
*/
|
||||
bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count);
|
||||
};
|
||||
|
||||
|
||||
/// For has.
|
||||
struct IndexToOne
|
||||
{
|
||||
@ -1159,587 +1015,4 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayEnumerate : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayEnumerate";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
};
|
||||
|
||||
|
||||
/// Counts the number of different elements in the array, or the number of different tuples from the elements at the corresponding positions in several arrays.
|
||||
/// NOTE The implementation partially matches arrayEnumerateUniq.
|
||||
class FunctionArrayUniq : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayUniq";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns);
|
||||
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values);
|
||||
};
|
||||
|
||||
|
||||
/// Find different elements in an array.
|
||||
class FunctionArrayDistinct : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayDistinct";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col);
|
||||
|
||||
bool executeString(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col);
|
||||
|
||||
void executeHashed(
|
||||
const IColumn & src_data,
|
||||
const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
ColumnArray::Offsets & res_offsets,
|
||||
const ColumnNullable * nullable_col);
|
||||
};
|
||||
|
||||
|
||||
template <typename Derived>
|
||||
class FunctionArrayEnumerateExtended : public IFunction
|
||||
{
|
||||
public:
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns);
|
||||
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values);
|
||||
};
|
||||
|
||||
class FunctionArrayEnumerateUniq : public FunctionArrayEnumerateExtended<FunctionArrayEnumerateUniq>
|
||||
{
|
||||
using Base = FunctionArrayEnumerateExtended<FunctionArrayEnumerateUniq>;
|
||||
public:
|
||||
static constexpr auto name = "arrayEnumerateUniq";
|
||||
using Base::create;
|
||||
};
|
||||
|
||||
class FunctionArrayEnumerateDense : public FunctionArrayEnumerateExtended<FunctionArrayEnumerateDense>
|
||||
{
|
||||
using Base = FunctionArrayEnumerateExtended<FunctionArrayEnumerateDense>;
|
||||
public:
|
||||
static constexpr auto name = "arrayEnumerateDense";
|
||||
using Base::create;
|
||||
};
|
||||
|
||||
template <typename Type> struct TypeToColumnType { using ColumnType = ColumnVector<Type>; };
|
||||
template <> struct TypeToColumnType<String> { using ColumnType = ColumnString; };
|
||||
|
||||
template <typename DataType> struct DataTypeToName : TypeName<typename DataType::FieldType> { };
|
||||
template <> struct DataTypeToName<DataTypeDate> { static std::string get() { return "Date"; } };
|
||||
template <> struct DataTypeToName<DataTypeDateTime> { static std::string get() { return "DateTime"; } };
|
||||
|
||||
template <typename DataType>
|
||||
struct FunctionEmptyArray : public IFunction
|
||||
{
|
||||
static constexpr auto base_name = "emptyArray";
|
||||
static const String name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionEmptyArray>(); }
|
||||
|
||||
private:
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||
{
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataType>());
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
using UnderlyingColumnType = typename TypeToColumnType<typename DataType::FieldType>::ColumnType;
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(
|
||||
UnderlyingColumnType::create(),
|
||||
ColumnArray::ColumnOffsets::create(input_rows_count, 0));
|
||||
}
|
||||
};
|
||||
|
||||
template <typename DataType>
|
||||
const String FunctionEmptyArray<DataType>::name = FunctionEmptyArray::base_name + String(DataTypeToName<DataType>::get());
|
||||
|
||||
class FunctionRange : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "range";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRange>(); }
|
||||
|
||||
private:
|
||||
String getName() const override;
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
template <typename T>
|
||||
bool executeInternal(Block & block, const IColumn * arg, const size_t result);
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
};
|
||||
|
||||
|
||||
class FunctionEmptyArrayToSingle : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "emptyArrayToSingle";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayReverse";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
size_t input_rows_count);
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col);
|
||||
|
||||
bool executeFixedString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col);
|
||||
|
||||
bool executeString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col);
|
||||
};
|
||||
|
||||
|
||||
class IAggregateFunction;
|
||||
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
|
||||
|
||||
/** Applies an aggregate function to array and returns its result.
|
||||
* If aggregate function has multiple arguments, then this function can be applied to multiple arrays of the same size.
|
||||
*/
|
||||
class FunctionArrayReduce : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayReduce";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
private:
|
||||
/// lazy initialization in getReturnTypeImpl
|
||||
/// TODO: init in FunctionBuilder
|
||||
mutable AggregateFunctionPtr aggregate_function;
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayConcat : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayConcat";
|
||||
static FunctionPtr create(const Context & context);
|
||||
FunctionArrayConcat(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
class FunctionArraySlice : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arraySlice";
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayPush : public IFunction
|
||||
{
|
||||
public:
|
||||
FunctionArrayPush(const Context & context, bool push_front, const char * name)
|
||||
: context(context), push_front(push_front), name(name) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
bool push_front;
|
||||
const char * name;
|
||||
};
|
||||
|
||||
class FunctionArrayPushFront : public FunctionArrayPush
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPushFront";
|
||||
|
||||
static FunctionPtr create(const Context & context);
|
||||
FunctionArrayPushFront(const Context & context) : FunctionArrayPush(context, true, name) {}
|
||||
};
|
||||
|
||||
class FunctionArrayPushBack : public FunctionArrayPush
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPushBack";
|
||||
|
||||
static FunctionPtr create(const Context & context);
|
||||
FunctionArrayPushBack(const Context & context) : FunctionArrayPush(context, false, name) {}
|
||||
};
|
||||
|
||||
class FunctionArrayPop : public IFunction
|
||||
{
|
||||
public:
|
||||
FunctionArrayPop(bool pop_front, const char * name) : pop_front(pop_front), name(name) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
private:
|
||||
bool pop_front;
|
||||
const char * name;
|
||||
};
|
||||
|
||||
class FunctionArrayPopFront : public FunctionArrayPop
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPopFront";
|
||||
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
FunctionArrayPopFront() : FunctionArrayPop(true, name) {}
|
||||
};
|
||||
|
||||
class FunctionArrayPopBack : public FunctionArrayPop
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPopBack";
|
||||
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
FunctionArrayPopBack() : FunctionArrayPop(false, name) {}
|
||||
};
|
||||
|
||||
class FunctionArrayIntersect : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayIntersect";
|
||||
static FunctionPtr create(const Context & context);
|
||||
FunctionArrayIntersect(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
struct UnpackedArrays
|
||||
{
|
||||
std::vector<char> is_const;
|
||||
std::vector<const NullMap *> null_maps;
|
||||
std::vector<const ColumnArray::ColumnOffsets::Container *> offsets;
|
||||
ColumnRawPtrs nested_columns;
|
||||
|
||||
UnpackedArrays() = default;
|
||||
};
|
||||
|
||||
/// Cast column to data_type removing nullable if data_type hasn't.
|
||||
/// It's expected that column can represent data_type after removing some NullMap's.
|
||||
ColumnPtr castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const;
|
||||
Columns castColumns(Block & block, const ColumnNumbers & arguments,
|
||||
const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls) const;
|
||||
UnpackedArrays prepareArrays(const Columns & columns) const;
|
||||
|
||||
template <typename Map, typename ColumnType, bool is_numeric_column>
|
||||
static ColumnPtr execute(const UnpackedArrays & arrays, MutableColumnPtr result_data);
|
||||
|
||||
struct NumberExecutor
|
||||
{
|
||||
const UnpackedArrays & arrays;
|
||||
const DataTypePtr & data_type;
|
||||
ColumnPtr & result;
|
||||
|
||||
NumberExecutor(const UnpackedArrays & arrays, const DataTypePtr & data_type, ColumnPtr & result)
|
||||
: arrays(arrays), data_type(data_type), result(result) {}
|
||||
|
||||
template <typename T, size_t>
|
||||
void operator()();
|
||||
};
|
||||
};
|
||||
|
||||
class FunctionArrayHasAllAny : public IFunction
|
||||
{
|
||||
public:
|
||||
FunctionArrayHasAllAny(const Context & context, bool all, const char * name)
|
||||
: context(context), all(all), name(name) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
bool all;
|
||||
const char * name;
|
||||
};
|
||||
|
||||
class FunctionArrayHasAll : public FunctionArrayHasAllAny
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hasAll";
|
||||
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
FunctionArrayHasAll(const Context & context) : FunctionArrayHasAllAny(context, true, name) {}
|
||||
};
|
||||
|
||||
class FunctionArrayHasAny : public FunctionArrayHasAllAny
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hasAny";
|
||||
|
||||
static FunctionPtr create(const Context & context);
|
||||
|
||||
FunctionArrayHasAny(const Context & context) : FunctionArrayHasAllAny(context, false, name) {}
|
||||
};
|
||||
|
||||
|
||||
class FunctionArrayResize : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayResize";
|
||||
static FunctionPtr create(const Context & context);
|
||||
FunctionArrayResize(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override;
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
struct NameHas { static constexpr auto name = "has"; };
|
||||
struct NameIndexOf { static constexpr auto name = "indexOf"; };
|
||||
struct NameCountEqual { static constexpr auto name = "countEqual"; };
|
||||
|
||||
using FunctionHas = FunctionArrayIndex<IndexToOne, NameHas>;
|
||||
using FunctionIndexOf = FunctionArrayIndex<IndexIdentity, NameIndexOf>;
|
||||
using FunctionCountEqual = FunctionArrayIndex<IndexCount, NameCountEqual>;
|
||||
|
||||
using FunctionEmptyArrayUInt8 = FunctionEmptyArray<DataTypeUInt8>;
|
||||
using FunctionEmptyArrayUInt16 = FunctionEmptyArray<DataTypeUInt16>;
|
||||
using FunctionEmptyArrayUInt32 = FunctionEmptyArray<DataTypeUInt32>;
|
||||
using FunctionEmptyArrayUInt64 = FunctionEmptyArray<DataTypeUInt64>;
|
||||
using FunctionEmptyArrayInt8 = FunctionEmptyArray<DataTypeInt8>;
|
||||
using FunctionEmptyArrayInt16 = FunctionEmptyArray<DataTypeInt16>;
|
||||
using FunctionEmptyArrayInt32 = FunctionEmptyArray<DataTypeInt32>;
|
||||
using FunctionEmptyArrayInt64 = FunctionEmptyArray<DataTypeInt64>;
|
||||
using FunctionEmptyArrayFloat32 = FunctionEmptyArray<DataTypeFloat32>;
|
||||
using FunctionEmptyArrayFloat64 = FunctionEmptyArray<DataTypeFloat64>;
|
||||
using FunctionEmptyArrayDate = FunctionEmptyArray<DataTypeDate>;
|
||||
using FunctionEmptyArrayDateTime = FunctionEmptyArray<DataTypeDateTime>;
|
||||
using FunctionEmptyArrayString = FunctionEmptyArray<DataTypeString>;
|
||||
|
||||
|
||||
}
|
457
dbms/src/Functions/arrayIntersect.cpp
Normal file
457
dbms/src/Functions/arrayIntersect.cpp
Normal file
@ -0,0 +1,457 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/getMostSubtype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/HashTable/ClearableHashMap.h>
|
||||
#include <Core/TypeListNumber.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
class FunctionArrayIntersect : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayIntersect";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayIntersect>(context); }
|
||||
FunctionArrayIntersect(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
struct UnpackedArrays
|
||||
{
|
||||
std::vector<char> is_const;
|
||||
std::vector<const NullMap *> null_maps;
|
||||
std::vector<const ColumnArray::ColumnOffsets::Container *> offsets;
|
||||
ColumnRawPtrs nested_columns;
|
||||
|
||||
UnpackedArrays() = default;
|
||||
};
|
||||
|
||||
/// Cast column to data_type removing nullable if data_type hasn't.
|
||||
/// It's expected that column can represent data_type after removing some NullMap's.
|
||||
ColumnPtr castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const;
|
||||
Columns castColumns(Block & block, const ColumnNumbers & arguments,
|
||||
const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls) const;
|
||||
UnpackedArrays prepareArrays(const Columns & columns) const;
|
||||
|
||||
template <typename Map, typename ColumnType, bool is_numeric_column>
|
||||
static ColumnPtr execute(const UnpackedArrays & arrays, MutableColumnPtr result_data);
|
||||
|
||||
struct NumberExecutor
|
||||
{
|
||||
const UnpackedArrays & arrays;
|
||||
const DataTypePtr & data_type;
|
||||
ColumnPtr & result;
|
||||
|
||||
NumberExecutor(const UnpackedArrays & arrays, const DataTypePtr & data_type, ColumnPtr & result)
|
||||
: arrays(arrays), data_type(data_type), result(result) {}
|
||||
|
||||
template <typename T, size_t>
|
||||
void operator()();
|
||||
};
|
||||
};
|
||||
|
||||
|
||||
DataTypePtr FunctionArrayIntersect::getReturnTypeImpl(const DataTypes & arguments) const
|
||||
{
|
||||
DataTypes nested_types;
|
||||
nested_types.reserve(arguments.size());
|
||||
|
||||
bool has_nothing = false;
|
||||
|
||||
if (arguments.empty())
|
||||
throw Exception{"Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
for (auto i : ext::range(0, arguments.size()))
|
||||
{
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
const auto & nested_type = array_type->getNestedType();
|
||||
|
||||
if (typeid_cast<const DataTypeNothing *>(nested_type.get()))
|
||||
has_nothing = true;
|
||||
else
|
||||
nested_types.push_back(nested_type);
|
||||
}
|
||||
|
||||
DataTypePtr result_type;
|
||||
|
||||
if (!nested_types.empty())
|
||||
result_type = getMostSubtype(nested_types, true);
|
||||
|
||||
if (has_nothing)
|
||||
result_type = std::make_shared<DataTypeNothing>();
|
||||
|
||||
return std::make_shared<DataTypeArray>(result_type);
|
||||
}
|
||||
|
||||
ColumnPtr FunctionArrayIntersect::castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const
|
||||
{
|
||||
if (auto column_nullable = checkAndGetColumn<ColumnNullable>(column.get()))
|
||||
{
|
||||
auto nullable_type = checkAndGetDataType<DataTypeNullable>(data_type.get());
|
||||
const auto & nested = column_nullable->getNestedColumnPtr();
|
||||
if (nullable_type)
|
||||
{
|
||||
auto casted_column = castRemoveNullable(nested, nullable_type->getNestedType());
|
||||
return ColumnNullable::create(casted_column, column_nullable->getNullMapColumnPtr());
|
||||
}
|
||||
return castRemoveNullable(nested, data_type);
|
||||
}
|
||||
else if (auto column_array = checkAndGetColumn<ColumnArray>(column.get()))
|
||||
{
|
||||
auto array_type = checkAndGetDataType<DataTypeArray>(data_type.get());
|
||||
if (!array_type)
|
||||
throw Exception{"Cannot cast array column to column with type "
|
||||
+ data_type->getName() + " in function " + getName(), ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
auto casted_column = castRemoveNullable(column_array->getDataPtr(), array_type->getNestedType());
|
||||
return ColumnArray::create(casted_column, column_array->getOffsetsPtr());
|
||||
}
|
||||
else if (auto column_tuple = checkAndGetColumn<ColumnTuple>(column.get()))
|
||||
{
|
||||
auto tuple_type = checkAndGetDataType<DataTypeTuple>(data_type.get());
|
||||
|
||||
if (!tuple_type)
|
||||
throw Exception{"Cannot cast tuple column to type "
|
||||
+ data_type->getName() + " in function " + getName(), ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
auto columns_number = column_tuple->getColumns().size();
|
||||
Columns columns(columns_number);
|
||||
|
||||
const auto & types = tuple_type->getElements();
|
||||
|
||||
for (auto i : ext::range(0, columns_number))
|
||||
{
|
||||
columns[i] = castRemoveNullable(column_tuple->getColumnPtr(i), types[i]);
|
||||
}
|
||||
return ColumnTuple::create(columns);
|
||||
}
|
||||
|
||||
return column;
|
||||
}
|
||||
|
||||
Columns FunctionArrayIntersect::castColumns(
|
||||
Block & block, const ColumnNumbers & arguments, const DataTypePtr & return_type,
|
||||
const DataTypePtr & return_type_with_nulls) const
|
||||
{
|
||||
size_t num_args = arguments.size();
|
||||
Columns columns(num_args);
|
||||
|
||||
auto type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
|
||||
auto & type_nested = type_array->getNestedType();
|
||||
auto type_not_nullable_nested = removeNullable(type_nested);
|
||||
|
||||
const bool is_numeric_or_string = isNumber(type_not_nullable_nested)
|
||||
|| isDateOrDateTime(type_not_nullable_nested)
|
||||
|| isStringOrFixedString(type_not_nullable_nested);
|
||||
|
||||
DataTypePtr nullable_return_type;
|
||||
|
||||
if (is_numeric_or_string)
|
||||
{
|
||||
auto type_nullable_nested = makeNullable(type_nested);
|
||||
nullable_return_type = std::make_shared<DataTypeArray>(type_nullable_nested);
|
||||
}
|
||||
|
||||
const bool nested_is_nullable = type_nested->isNullable();
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]);
|
||||
auto & column = columns[i];
|
||||
|
||||
if (is_numeric_or_string)
|
||||
{
|
||||
/// Cast to Array(T) or Array(Nullable(T)).
|
||||
if (nested_is_nullable)
|
||||
{
|
||||
if (arg.type->equals(*return_type))
|
||||
column = arg.column;
|
||||
else
|
||||
column = castColumn(arg, return_type, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If result has array type Array(T) still cast Array(Nullable(U)) to Array(Nullable(T))
|
||||
/// because cannot cast Nullable(T) to T.
|
||||
if (arg.type->equals(*return_type) || arg.type->equals(*nullable_return_type))
|
||||
column = arg.column;
|
||||
else if (static_cast<const DataTypeArray &>(*arg.type).getNestedType()->isNullable())
|
||||
column = castColumn(arg, nullable_return_type, context);
|
||||
else
|
||||
column = castColumn(arg, return_type, context);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// return_type_with_nulls is the most common subtype with possible nullable parts.
|
||||
if (arg.type->equals(*return_type_with_nulls))
|
||||
column = arg.column;
|
||||
else
|
||||
column = castColumn(arg, return_type_with_nulls, context);
|
||||
}
|
||||
}
|
||||
|
||||
return columns;
|
||||
}
|
||||
|
||||
FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(const Columns & columns) const
|
||||
{
|
||||
UnpackedArrays arrays;
|
||||
|
||||
size_t columns_number = columns.size();
|
||||
arrays.is_const.assign(columns_number, false);
|
||||
arrays.null_maps.resize(columns_number);
|
||||
arrays.offsets.resize(columns_number);
|
||||
arrays.nested_columns.resize(columns_number);
|
||||
|
||||
for (auto i : ext::range(0, columns_number))
|
||||
{
|
||||
auto argument_column = columns[i].get();
|
||||
if (auto argument_column_const = typeid_cast<const ColumnConst *>(argument_column))
|
||||
{
|
||||
arrays.is_const[i] = true;
|
||||
argument_column = argument_column_const->getDataColumnPtr().get();
|
||||
}
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(argument_column))
|
||||
{
|
||||
arrays.offsets[i] = &argument_column_array->getOffsets();
|
||||
arrays.nested_columns[i] = &argument_column_array->getData();
|
||||
if (auto column_nullable = typeid_cast<const ColumnNullable *>(arrays.nested_columns[i]))
|
||||
{
|
||||
arrays.null_maps[i] = &column_nullable->getNullMapData();
|
||||
arrays.nested_columns[i] = &column_nullable->getNestedColumn();
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
return arrays;
|
||||
}
|
||||
|
||||
void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
const auto & return_type = block.getByPosition(result).type;
|
||||
auto return_type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
|
||||
|
||||
if (!return_type)
|
||||
throw Exception{"Return type for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
const auto & nested_return_type = return_type_array->getNestedType();
|
||||
|
||||
if (typeid_cast<const DataTypeNothing *>(nested_return_type.get()))
|
||||
{
|
||||
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto num_args = arguments.size();
|
||||
DataTypes data_types;
|
||||
data_types.reserve(num_args);
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
data_types.push_back(block.getByPosition(arguments[i]).type);
|
||||
|
||||
auto return_type_with_nulls = getMostSubtype(data_types, true, true);
|
||||
|
||||
Columns columns = castColumns(block, arguments, return_type, return_type_with_nulls);
|
||||
|
||||
UnpackedArrays arrays = prepareArrays(columns);
|
||||
|
||||
ColumnPtr result_column;
|
||||
auto not_nullable_nested_return_type = removeNullable(nested_return_type);
|
||||
TypeListNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
|
||||
|
||||
using DateMap = ClearableHashMap<DataTypeDate::FieldType, size_t, DefaultHash<DataTypeDate::FieldType>,
|
||||
HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDate::FieldType)>>;
|
||||
|
||||
using DateTimeMap = ClearableHashMap<DataTypeDateTime::FieldType, size_t, DefaultHash<DataTypeDateTime::FieldType>,
|
||||
HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(DataTypeDateTime::FieldType)>>;
|
||||
|
||||
using StringMap = ClearableHashMap<StringRef, size_t, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
if (!result_column)
|
||||
{
|
||||
auto column = not_nullable_nested_return_type->createColumn();
|
||||
WhichDataType which(not_nullable_nested_return_type);
|
||||
|
||||
if (which.isDate())
|
||||
result_column = execute<DateMap, ColumnVector<DataTypeDate::FieldType>, true>(arrays, std::move(column));
|
||||
else if (which.isDateTime())
|
||||
result_column = execute<DateTimeMap, ColumnVector<DataTypeDateTime::FieldType>, true>(arrays, std::move(column));
|
||||
else if(which.isString())
|
||||
result_column = execute<StringMap, ColumnString, false>(arrays, std::move(column));
|
||||
else if(which.isFixedString())
|
||||
result_column = execute<StringMap, ColumnFixedString, false>(arrays, std::move(column));
|
||||
else
|
||||
{
|
||||
column = static_cast<const DataTypeArray &>(*return_type_with_nulls).getNestedType()->createColumn();
|
||||
result_column = castRemoveNullable(execute<StringMap, IColumn, false>(arrays, std::move(column)), return_type);
|
||||
}
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
template <typename T, size_t>
|
||||
void FunctionArrayIntersect::NumberExecutor::operator()()
|
||||
{
|
||||
using Map = ClearableHashMap<T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
if (!result && typeid_cast<const DataTypeNumber<T> *>(data_type.get()))
|
||||
result = execute<Map, ColumnVector<T>, true>(arrays, ColumnVector<T>::create());
|
||||
}
|
||||
|
||||
template <typename Map, typename ColumnType, bool is_numeric_column>
|
||||
ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr)
|
||||
{
|
||||
auto args = arrays.nested_columns.size();
|
||||
auto rows = arrays.offsets.front()->size();
|
||||
|
||||
bool all_nullable = true;
|
||||
|
||||
std::vector<const ColumnType *> columns;
|
||||
columns.reserve(args);
|
||||
for (auto arg : ext::range(0, args))
|
||||
{
|
||||
if constexpr (std::is_same<ColumnType, IColumn>::value)
|
||||
columns.push_back(arrays.nested_columns[arg]);
|
||||
else
|
||||
columns.push_back(checkAndGetColumn<ColumnType>(arrays.nested_columns[arg]));
|
||||
|
||||
if (!columns.back())
|
||||
throw Exception("Unexpected array type for function arrayIntersect", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!arrays.null_maps[arg])
|
||||
all_nullable = false;
|
||||
}
|
||||
|
||||
auto & result_data = static_cast<ColumnType &>(*result_data_ptr);
|
||||
auto result_offsets_ptr = ColumnArray::ColumnOffsets::create(rows);
|
||||
auto & result_offsets = static_cast<ColumnArray::ColumnOffsets &>(*result_offsets_ptr);
|
||||
auto null_map_column = ColumnUInt8::create();
|
||||
NullMap & null_map = static_cast<ColumnUInt8 &>(*null_map_column).getData();
|
||||
|
||||
Arena arena;
|
||||
|
||||
Map map;
|
||||
std::vector<size_t> prev_off(args, 0);
|
||||
size_t result_offset = 0;
|
||||
for (auto row : ext::range(0, rows))
|
||||
{
|
||||
map.clear();
|
||||
|
||||
bool all_has_nullable = all_nullable;
|
||||
|
||||
for (auto arg : ext::range(0, args))
|
||||
{
|
||||
bool current_has_nullable = false;
|
||||
size_t off = (*arrays.offsets[arg])[row];
|
||||
for (auto i : ext::range(prev_off[arg], off))
|
||||
{
|
||||
if (arrays.null_maps[arg] && (*arrays.null_maps[arg])[i])
|
||||
current_has_nullable = true;
|
||||
else
|
||||
{
|
||||
if constexpr (is_numeric_column)
|
||||
++map[columns[arg]->getElement(i)];
|
||||
else if constexpr (std::is_same<ColumnType, ColumnString>::value || std::is_same<ColumnType, ColumnFixedString>::value)
|
||||
++map[columns[arg]->getDataAt(i)];
|
||||
else
|
||||
{
|
||||
const char * data = nullptr;
|
||||
++map[columns[arg]->serializeValueIntoArena(i, arena, data)];
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
prev_off[arg] = off;
|
||||
if (!current_has_nullable)
|
||||
all_has_nullable = false;
|
||||
}
|
||||
|
||||
if (all_has_nullable)
|
||||
{
|
||||
++result_offset;
|
||||
result_data.insertDefault();
|
||||
null_map.push_back(1);
|
||||
}
|
||||
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
if (pair.second == args)
|
||||
{
|
||||
++result_offset;
|
||||
if constexpr (is_numeric_column)
|
||||
result_data.insert(pair.first);
|
||||
else if constexpr (std::is_same<ColumnType, ColumnString>::value || std::is_same<ColumnType, ColumnFixedString>::value)
|
||||
result_data.insertData(pair.first.data, pair.first.size);
|
||||
else
|
||||
result_data.deserializeAndInsertFromArena(pair.first.data);
|
||||
|
||||
if (all_nullable)
|
||||
null_map.push_back(0);
|
||||
}
|
||||
}
|
||||
result_offsets.getElement(row) = result_offset;
|
||||
}
|
||||
|
||||
ColumnPtr result_column = std::move(result_data_ptr);
|
||||
if (all_nullable)
|
||||
result_column = ColumnNullable::create(result_column, std::move(null_map_column));
|
||||
return ColumnArray::create(result_column, std::move(result_offsets_ptr));
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionArrayIntersect(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayIntersect>();
|
||||
}
|
||||
|
||||
}
|
82
dbms/src/Functions/arrayPop.h
Normal file
82
dbms/src/Functions/arrayPop.h
Normal file
@ -0,0 +1,82 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
class FunctionArrayPop : public IFunction
|
||||
{
|
||||
public:
|
||||
FunctionArrayPop(bool pop_front, const char * name) : pop_front(pop_front), name(name) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments[0]->onlyNull())
|
||||
return arguments[0];
|
||||
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto & return_type = block.getByPosition(result).type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
const auto & array_column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> source;
|
||||
|
||||
size_t size = array_column->size();
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(array_column.get()))
|
||||
source = GatherUtils::createArraySource(*argument_column_array, false, size);
|
||||
else
|
||||
throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
auto sink = GatherUtils::createArraySink(typeid_cast<ColumnArray &>(*result_column), size);
|
||||
|
||||
if (pop_front)
|
||||
GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, *sink, 1);
|
||||
else
|
||||
GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, 0, -1);
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
private:
|
||||
bool pop_front;
|
||||
const char * name;
|
||||
};
|
||||
|
||||
}
|
21
dbms/src/Functions/arrayPopBack.cpp
Normal file
21
dbms/src/Functions/arrayPopBack.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Functions/arrayPop.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionArrayPopBack : public FunctionArrayPop
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPopBack";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayPopBack>(); }
|
||||
FunctionArrayPopBack() : FunctionArrayPop(false, name) {}
|
||||
};
|
||||
|
||||
void registerFunctionArrayPopBack(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayPopBack>();
|
||||
}
|
||||
|
||||
}
|
21
dbms/src/Functions/arrayPopFront.cpp
Normal file
21
dbms/src/Functions/arrayPopFront.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Functions/arrayPop.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionArrayPopFront : public FunctionArrayPop
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPopFront";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayPopFront>(); }
|
||||
FunctionArrayPopFront() : FunctionArrayPop(true, name) {}
|
||||
};
|
||||
|
||||
void registerFunctionArrayPopFront(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayPopFront>();
|
||||
}
|
||||
|
||||
}
|
114
dbms/src/Functions/arrayPush.h
Normal file
114
dbms/src/Functions/arrayPush.h
Normal file
@ -0,0 +1,114 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
class FunctionArrayPush : public IFunction
|
||||
{
|
||||
public:
|
||||
FunctionArrayPush(const Context & context, bool push_front, const char * name)
|
||||
: context(context), push_front(push_front), name(name) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments[0]->onlyNull())
|
||||
return arguments[0];
|
||||
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
auto nested_type = array_type->getNestedType();
|
||||
|
||||
DataTypes types = {nested_type, arguments[1]};
|
||||
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype(types));
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto & return_type = block.getByPosition(result).type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
auto array_column = block.getByPosition(arguments[0]).column;
|
||||
auto appended_column = block.getByPosition(arguments[1]).column;
|
||||
|
||||
if (!block.getByPosition(arguments[0]).type->equals(*return_type))
|
||||
array_column = castColumn(block.getByPosition(arguments[0]), return_type, context);
|
||||
|
||||
const DataTypePtr & return_nested_type = typeid_cast<const DataTypeArray &>(*return_type).getNestedType();
|
||||
if (!block.getByPosition(arguments[1]).type->equals(*return_nested_type))
|
||||
appended_column = castColumn(block.getByPosition(arguments[1]), return_nested_type, context);
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> array_source;
|
||||
std::unique_ptr<GatherUtils::IValueSource> value_source;
|
||||
|
||||
size_t size = array_column->size();
|
||||
bool is_const = false;
|
||||
|
||||
if (auto const_array_column = typeid_cast<const ColumnConst *>(array_column.get()))
|
||||
{
|
||||
is_const = true;
|
||||
array_column = const_array_column->getDataColumnPtr();
|
||||
}
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(array_column.get()))
|
||||
array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size);
|
||||
else
|
||||
throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
|
||||
bool is_appended_const = false;
|
||||
if (auto const_appended_column = typeid_cast<const ColumnConst *>(appended_column.get()))
|
||||
{
|
||||
is_appended_const = true;
|
||||
appended_column = const_appended_column->getDataColumnPtr();
|
||||
}
|
||||
|
||||
value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size);
|
||||
|
||||
auto sink = GatherUtils::createArraySink(typeid_cast<ColumnArray &>(*result_column), size);
|
||||
|
||||
GatherUtils::push(*array_source, *value_source, *sink, push_front);
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
bool push_front;
|
||||
const char * name;
|
||||
};
|
||||
|
||||
}
|
21
dbms/src/Functions/arrayPushBack.cpp
Normal file
21
dbms/src/Functions/arrayPushBack.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Functions/arrayPush.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionArrayPushBack : public FunctionArrayPush
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPushBack";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayPushBack>(context); }
|
||||
FunctionArrayPushBack(const Context & context) : FunctionArrayPush(context, false, name) {}
|
||||
};
|
||||
|
||||
void registerFunctionArrayPushBack(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayPushBack>();
|
||||
}
|
||||
|
||||
}
|
23
dbms/src/Functions/arrayPushFront.cpp
Normal file
23
dbms/src/Functions/arrayPushFront.cpp
Normal file
@ -0,0 +1,23 @@
|
||||
#include <Functions/arrayPush.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class FunctionArrayPushFront : public FunctionArrayPush
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayPushFront";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayPushFront>(context); }
|
||||
FunctionArrayPushFront(const Context & context) : FunctionArrayPush(context, true, name) {}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionArrayPushFront(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayPushFront>();
|
||||
}
|
||||
|
||||
}
|
195
dbms/src/Functions/arrayReduce.cpp
Normal file
195
dbms/src/Functions/arrayReduce.cpp
Normal file
@ -0,0 +1,195 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/Arena.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/** Applies an aggregate function to array and returns its result.
|
||||
* If aggregate function has multiple arguments, then this function can be applied to multiple arrays of the same size.
|
||||
*
|
||||
* arrayReduce('agg', arr1, ...) - apply the aggregate function `agg` to arrays `arr1...`
|
||||
* If multiple arrays passed, then elements on corresponding positions are passed as multiple arguments to the aggregate function.
|
||||
*/
|
||||
class FunctionArrayReduce : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayReduce";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayReduce>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// lazy initialization in getReturnTypeImpl
|
||||
/// TODO: init in FunctionBuilder
|
||||
mutable AggregateFunctionPtr aggregate_function;
|
||||
};
|
||||
|
||||
|
||||
DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
/// The first argument is a constant string with the name of the aggregate function
|
||||
/// (possibly with parameters in parentheses, for example: "quantile(0.99)").
|
||||
|
||||
if (arguments.size() < 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be at least 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const ColumnConst * aggregate_function_name_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
||||
if (!aggregate_function_name_column)
|
||||
throw Exception("First argument for function " + getName() + " must be constant string: name of aggregate function.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
DataTypes argument_types(arguments.size() - 1);
|
||||
for (size_t i = 1, size = arguments.size(); i < size; ++i)
|
||||
{
|
||||
const DataTypeArray * arg = checkAndGetDataType<DataTypeArray>(arguments[i].type.get());
|
||||
if (!arg)
|
||||
throw Exception("Argument " + toString(i) + " for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[i].type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
argument_types[i - 1] = arg->getNestedType();
|
||||
}
|
||||
|
||||
if (!aggregate_function)
|
||||
{
|
||||
String aggregate_function_name_with_params = aggregate_function_name_column->getValue<String>();
|
||||
|
||||
if (aggregate_function_name_with_params.empty())
|
||||
throw Exception("First argument for function " + getName() + " (name of aggregate function) cannot be empty.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
String aggregate_function_name;
|
||||
Array params_row;
|
||||
getAggregateFunctionNameAndParametersArray(aggregate_function_name_with_params,
|
||||
aggregate_function_name, params_row, "function " + getName());
|
||||
|
||||
aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, argument_types, params_row);
|
||||
}
|
||||
|
||||
return aggregate_function->getReturnType();
|
||||
}
|
||||
|
||||
|
||||
void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
IAggregateFunction & agg_func = *aggregate_function.get();
|
||||
AlignedBuffer place_holder(agg_func.sizeOfData(), agg_func.alignOfData());
|
||||
AggregateDataPtr place = place_holder.data();
|
||||
|
||||
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;
|
||||
|
||||
size_t rows = input_rows_count;
|
||||
|
||||
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
|
||||
std::vector<ColumnPtr> materialized_columns;
|
||||
|
||||
const size_t num_arguments_columns = arguments.size() - 1;
|
||||
|
||||
std::vector<const IColumn *> aggregate_arguments_vec(num_arguments_columns);
|
||||
const ColumnArray::Offsets * offsets = nullptr;
|
||||
|
||||
for (size_t i = 0; i < num_arguments_columns; ++i)
|
||||
{
|
||||
const IColumn * col = block.getByPosition(arguments[i + 1]).column.get();
|
||||
const ColumnArray::Offsets * offsets_i = nullptr;
|
||||
if (const ColumnArray * arr = checkAndGetColumn<ColumnArray>(col))
|
||||
{
|
||||
aggregate_arguments_vec[i] = &arr->getData();
|
||||
offsets_i = &arr->getOffsets();
|
||||
}
|
||||
else if (const ColumnConst * const_arr = checkAndGetColumnConst<ColumnArray>(col))
|
||||
{
|
||||
materialized_columns.emplace_back(const_arr->convertToFullColumn());
|
||||
const auto & arr = typeid_cast<const ColumnArray &>(*materialized_columns.back().get());
|
||||
aggregate_arguments_vec[i] = &arr.getData();
|
||||
offsets_i = &arr.getOffsets();
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + col->getName() + " as argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (i == 0)
|
||||
offsets = offsets_i;
|
||||
else if (*offsets_i != *offsets)
|
||||
throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.",
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
}
|
||||
const IColumn ** aggregate_arguments = aggregate_arguments_vec.data();
|
||||
|
||||
MutableColumnPtr result_holder = block.getByPosition(result).type->createColumn();
|
||||
IColumn & res_col = *result_holder;
|
||||
|
||||
/// AggregateFunction's states should be inserted into column using specific way
|
||||
auto res_col_aggregate_function = typeid_cast<ColumnAggregateFunction *>(&res_col);
|
||||
|
||||
if (!res_col_aggregate_function && agg_func.isState())
|
||||
throw Exception("State function " + agg_func.getName() + " inserts results into non-state column "
|
||||
+ block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
agg_func.create(place);
|
||||
ColumnArray::Offset next_offset = (*offsets)[i];
|
||||
|
||||
try
|
||||
{
|
||||
for (size_t j = current_offset; j < next_offset; ++j)
|
||||
agg_func.add(place, aggregate_arguments, j, arena.get());
|
||||
|
||||
if (!res_col_aggregate_function)
|
||||
agg_func.insertResultInto(place, res_col);
|
||||
else
|
||||
res_col_aggregate_function->insertFrom(place);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
agg_func.destroy(place);
|
||||
throw;
|
||||
}
|
||||
|
||||
agg_func.destroy(place);
|
||||
current_offset = next_offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(result_holder);
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionArrayReduce(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayReduce>();
|
||||
}
|
||||
|
||||
}
|
148
dbms/src/Functions/arrayResize.cpp
Normal file
148
dbms/src/Functions/arrayResize.cpp
Normal file
@ -0,0 +1,148 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
class FunctionArrayResize : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayResize";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayResize>(context); }
|
||||
FunctionArrayResize(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const size_t number_of_arguments = arguments.size();
|
||||
|
||||
if (number_of_arguments < 2 || number_of_arguments > 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(number_of_arguments) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments[0]->onlyNull())
|
||||
return arguments[0];
|
||||
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (WhichDataType(array_type->getNestedType()).isNothing())
|
||||
throw Exception("Function " + getName() + " cannot resize " + array_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isInteger(removeNullable(arguments[1])) && !arguments[1]->onlyNull())
|
||||
throw Exception(
|
||||
"Argument " + toString(1) + " for function " + getName() + " must be integer but it has type "
|
||||
+ arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (number_of_arguments == 2)
|
||||
return arguments[0];
|
||||
else /* if (number_of_arguments == 3) */
|
||||
return std::make_shared<DataTypeArray>(getLeastSupertype({array_type->getNestedType(), arguments[2]}));
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto & return_type = block.getByPosition(result).type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
auto array_column = block.getByPosition(arguments[0]).column;
|
||||
auto size_column = block.getByPosition(arguments[1]).column;
|
||||
|
||||
if (!block.getByPosition(arguments[0]).type->equals(*return_type))
|
||||
array_column = castColumn(block.getByPosition(arguments[0]), return_type, context);
|
||||
|
||||
const DataTypePtr & return_nested_type = typeid_cast<const DataTypeArray &>(*return_type).getNestedType();
|
||||
size_t size = array_column->size();
|
||||
|
||||
ColumnPtr appended_column;
|
||||
if (arguments.size() == 3)
|
||||
{
|
||||
appended_column = block.getByPosition(arguments[2]).column;
|
||||
if (!block.getByPosition(arguments[2]).type->equals(*return_nested_type))
|
||||
appended_column = castColumn(block.getByPosition(arguments[2]), return_nested_type, context);
|
||||
}
|
||||
else
|
||||
appended_column = return_nested_type->createColumnConstWithDefaultValue(size);
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> array_source;
|
||||
std::unique_ptr<GatherUtils::IValueSource> value_source;
|
||||
|
||||
bool is_const = false;
|
||||
|
||||
if (auto const_array_column = typeid_cast<const ColumnConst *>(array_column.get()))
|
||||
{
|
||||
is_const = true;
|
||||
array_column = const_array_column->getDataColumnPtr();
|
||||
}
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(array_column.get()))
|
||||
array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size);
|
||||
else
|
||||
throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
|
||||
bool is_appended_const = false;
|
||||
if (auto const_appended_column = typeid_cast<const ColumnConst *>(appended_column.get()))
|
||||
{
|
||||
is_appended_const = true;
|
||||
appended_column = const_appended_column->getDataColumnPtr();
|
||||
}
|
||||
|
||||
value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size);
|
||||
|
||||
auto sink = GatherUtils::createArraySink(typeid_cast<ColumnArray &>(*result_column), size);
|
||||
|
||||
if (size_column->isColumnConst())
|
||||
GatherUtils::resizeConstantSize(*array_source, *value_source, *sink, size_column->getInt(0));
|
||||
else
|
||||
GatherUtils::resizeDynamicSize(*array_source, *value_source, *sink, *size_column);
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionArrayResize(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayResize>();
|
||||
}
|
||||
|
||||
}
|
365
dbms/src/Functions/arrayReverse.cpp
Normal file
365
dbms/src/Functions/arrayReverse.cpp
Normal file
@ -0,0 +1,365 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/// TODO Add generic implementation.
|
||||
|
||||
class FunctionArrayReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayReverse";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayReverse>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument for function " + getName() + " must be array.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
size_t input_rows_count);
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col);
|
||||
|
||||
bool executeFixedString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col);
|
||||
|
||||
bool executeString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col);
|
||||
};
|
||||
|
||||
|
||||
void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
if (executeConst(block, arguments, result, input_rows_count))
|
||||
return;
|
||||
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
auto res_ptr = array->cloneEmpty();
|
||||
ColumnArray & res = static_cast<ColumnArray &>(*res_ptr);
|
||||
res.getOffsetsPtr() = array->getOffsetsPtr();
|
||||
|
||||
const IColumn & src_data = array->getData();
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
IColumn & res_data = res.getData();
|
||||
|
||||
const ColumnNullable * nullable_col = nullptr;
|
||||
ColumnNullable * nullable_res_col = nullptr;
|
||||
|
||||
const IColumn * inner_col;
|
||||
IColumn * inner_res_col;
|
||||
|
||||
if (src_data.isColumnNullable())
|
||||
{
|
||||
nullable_col = static_cast<const ColumnNullable *>(&src_data);
|
||||
inner_col = &nullable_col->getNestedColumn();
|
||||
|
||||
nullable_res_col = static_cast<ColumnNullable *>(&res_data);
|
||||
inner_res_col = &nullable_res_col->getNestedColumn();
|
||||
}
|
||||
else
|
||||
{
|
||||
inner_col = &src_data;
|
||||
inner_res_col = &res_data;
|
||||
}
|
||||
|
||||
if (!( executeNumber<UInt8>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<UInt16>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<UInt32>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<UInt64>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<Int8>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<Int16>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<Int32>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<Int64>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<Float32>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeNumber<Float64>(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeString(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)
|
||||
|| executeFixedString(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col)))
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arguments, size_t result,
|
||||
size_t input_rows_count)
|
||||
{
|
||||
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
Array arr = const_array->getValue<Array>();
|
||||
|
||||
size_t size = arr.size();
|
||||
Array res(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
res[i] = arr[size - i - 1];
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool FunctionArrayReverse::executeNumber(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col)
|
||||
{
|
||||
auto do_reverse = [](const auto & src_data, const auto & src_offsets, auto & res_data)
|
||||
{
|
||||
size_t size = src_offsets.size();
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const auto * src = &src_data[src_prev_offset];
|
||||
const auto * src_end = &src_data[src_offsets[i]];
|
||||
|
||||
if (src == src_end)
|
||||
continue;
|
||||
|
||||
auto dst = &res_data[src_offsets[i] - 1];
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
*dst = *src;
|
||||
++src;
|
||||
--dst;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
};
|
||||
|
||||
if (const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data))
|
||||
{
|
||||
const PaddedPODArray<T> & src_data = src_data_concrete->getData();
|
||||
PaddedPODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
|
||||
res_data.resize(src_data.size());
|
||||
do_reverse(src_data, src_offsets, res_data);
|
||||
|
||||
if ((nullable_col) && (nullable_res_col))
|
||||
{
|
||||
/// Make a reverted null map.
|
||||
const auto & src_null_map = static_cast<const ColumnUInt8 &>(nullable_col->getNullMapColumn()).getData();
|
||||
auto & res_null_map = static_cast<ColumnUInt8 &>(nullable_res_col->getNullMapColumn()).getData();
|
||||
res_null_map.resize(src_data.size());
|
||||
do_reverse(src_null_map, src_offsets, res_null_map);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool FunctionArrayReverse::executeFixedString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col)
|
||||
{
|
||||
if (const ColumnFixedString * src_data_concrete = checkAndGetColumn<ColumnFixedString>(&src_data))
|
||||
{
|
||||
const size_t n = src_data_concrete->getN();
|
||||
const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
ColumnFixedString::Chars_t & res_data = typeid_cast<ColumnFixedString &>(res_data_col).getChars();
|
||||
size_t size = src_offsets.size();
|
||||
res_data.resize(src_data.size());
|
||||
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 * src = &src_data[src_prev_offset * n];
|
||||
const UInt8 * src_end = &src_data[src_offsets[i] * n];
|
||||
|
||||
if (src == src_end)
|
||||
continue;
|
||||
|
||||
UInt8 * dst = &res_data[src_offsets[i] * n - n];
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
/// NOTE: memcpySmallAllowReadWriteOverflow15 doesn't work correctly here.
|
||||
memcpy(dst, src, n);
|
||||
src += n;
|
||||
dst -= n;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
if ((nullable_col) && (nullable_res_col))
|
||||
{
|
||||
/// Make a reverted null map.
|
||||
const auto & src_null_map = static_cast<const ColumnUInt8 &>(nullable_col->getNullMapColumn()).getData();
|
||||
auto & res_null_map = static_cast<ColumnUInt8 &>(nullable_res_col->getNullMapColumn()).getData();
|
||||
res_null_map.resize(src_null_map.size());
|
||||
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 * src = &src_null_map[src_prev_offset];
|
||||
const UInt8 * src_end = &src_null_map[src_offsets[i]];
|
||||
|
||||
if (src == src_end)
|
||||
continue;
|
||||
|
||||
UInt8 * dst = &res_null_map[src_offsets[i] - 1];
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
*dst = *src;
|
||||
++src;
|
||||
--dst;
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool FunctionArrayReverse::executeString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets,
|
||||
IColumn & res_data_col,
|
||||
const ColumnNullable * nullable_col,
|
||||
ColumnNullable * nullable_res_col)
|
||||
{
|
||||
if (const ColumnString * src_data_concrete = checkAndGetColumn<ColumnString>(&src_data))
|
||||
{
|
||||
const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets();
|
||||
ColumnString::Offsets & res_string_offsets = typeid_cast<ColumnString &>(res_data_col).getOffsets();
|
||||
|
||||
const ColumnString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
ColumnString::Chars_t & res_data = typeid_cast<ColumnString &>(res_data_col).getChars();
|
||||
|
||||
size_t size = src_array_offsets.size();
|
||||
res_string_offsets.resize(src_string_offsets.size());
|
||||
res_data.resize(src_data.size());
|
||||
|
||||
ColumnArray::Offset src_array_prev_offset = 0;
|
||||
ColumnString::Offset res_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_array_offsets[i] != src_array_prev_offset)
|
||||
{
|
||||
size_t array_size = src_array_offsets[i] - src_array_prev_offset;
|
||||
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
size_t j_reversed = array_size - j - 1;
|
||||
|
||||
auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1];
|
||||
size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos;
|
||||
|
||||
memcpySmallAllowReadWriteOverflow15(&res_data[res_string_prev_offset], &src_data[src_pos], string_size);
|
||||
|
||||
res_string_prev_offset += string_size;
|
||||
res_string_offsets[src_array_prev_offset + j] = res_string_prev_offset;
|
||||
}
|
||||
}
|
||||
|
||||
src_array_prev_offset = src_array_offsets[i];
|
||||
}
|
||||
|
||||
if ((nullable_col) && (nullable_res_col))
|
||||
{
|
||||
/// Make a reverted null map.
|
||||
const auto & src_null_map = static_cast<const ColumnUInt8 &>(nullable_col->getNullMapColumn()).getData();
|
||||
auto & res_null_map = static_cast<ColumnUInt8 &>(nullable_res_col->getNullMapColumn()).getData();
|
||||
res_null_map.resize(src_string_offsets.size());
|
||||
|
||||
size_t size = src_string_offsets.size();
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const auto * src = &src_null_map[src_prev_offset];
|
||||
const auto * src_end = &src_null_map[src_array_offsets[i]];
|
||||
|
||||
if (src == src_end)
|
||||
continue;
|
||||
|
||||
auto dst = &res_null_map[src_array_offsets[i] - 1];
|
||||
|
||||
while (src < src_end)
|
||||
{
|
||||
*dst = *src;
|
||||
++src;
|
||||
--dst;
|
||||
}
|
||||
|
||||
src_prev_offset = src_array_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionArrayReverse(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayReverse>();
|
||||
}
|
||||
|
||||
}
|
165
dbms/src/Functions/arraySlice.cpp
Normal file
165
dbms/src/Functions/arraySlice.cpp
Normal file
@ -0,0 +1,165 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
/** arraySlice(arr, offset, length) - make slice of array. Offsets and length may be < 0 or Null
|
||||
* - if offset < 0, indexation from right element
|
||||
* - if length < 0, length = len(array) - (positive_index(offset) - 1) + length
|
||||
* indexation:
|
||||
* [ 1, 2, 3, 4, 5, 6]
|
||||
* [-6, -5, -4, -3, -2, -1]
|
||||
* examples:
|
||||
* arraySlice([1, 2, 3, 4, 5, 6], -4, 2) -> [3, 4]
|
||||
* arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5] (6 - (2 - 1) + (-1) = 4)
|
||||
* arraySlice([1, 2, 3, 4, 5, 6], -5, -1) = arraySlice([1, 2, 3, 4, 5, 6], 2, -1) -> [2, 3, 4, 5]
|
||||
*/
|
||||
class FunctionArraySlice : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arraySlice";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArraySlice>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const size_t number_of_arguments = arguments.size();
|
||||
|
||||
if (number_of_arguments < 2 || number_of_arguments > 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(number_of_arguments) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (arguments[0]->onlyNull())
|
||||
return arguments[0];
|
||||
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
for (size_t i = 1; i < number_of_arguments; ++i)
|
||||
{
|
||||
if (!isInteger(removeNullable(arguments[i])) && !arguments[i]->onlyNull())
|
||||
throw Exception(
|
||||
"Argument " + toString(i) + " for function " + getName() + " must be integer but it has type "
|
||||
+ arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const auto & return_type = block.getByPosition(result).type;
|
||||
|
||||
if (return_type->onlyNull())
|
||||
{
|
||||
block.getByPosition(result).column = return_type->createColumnConstWithDefaultValue(input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
auto result_column = return_type->createColumn();
|
||||
|
||||
auto & array_column = block.getByPosition(arguments[0]).column;
|
||||
const auto & offset_column = block.getByPosition(arguments[1]).column;
|
||||
const auto & length_column = arguments.size() > 2 ? block.getByPosition(arguments[2]).column : nullptr;
|
||||
|
||||
std::unique_ptr<GatherUtils::IArraySource> source;
|
||||
|
||||
size_t size = array_column->size();
|
||||
bool is_const = false;
|
||||
|
||||
if (auto const_array_column = typeid_cast<const ColumnConst *>(array_column.get()))
|
||||
{
|
||||
is_const = true;
|
||||
array_column = const_array_column->getDataColumnPtr();
|
||||
}
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(array_column.get()))
|
||||
source = GatherUtils::createArraySource(*argument_column_array, is_const, size);
|
||||
else
|
||||
throw Exception{"First arguments for function " + getName() + " must be array.", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
auto sink = GatherUtils::createArraySink(typeid_cast<ColumnArray &>(*result_column), size);
|
||||
|
||||
if (offset_column->onlyNull())
|
||||
{
|
||||
if (!length_column || length_column->onlyNull())
|
||||
{
|
||||
block.getByPosition(result).column = array_column;
|
||||
return;
|
||||
}
|
||||
else if (length_column->isColumnConst())
|
||||
GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, 0, length_column->getInt(0));
|
||||
else
|
||||
{
|
||||
auto const_offset_column = ColumnConst::create(ColumnInt8::create(1, 1), size);
|
||||
GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *const_offset_column, *length_column);
|
||||
}
|
||||
}
|
||||
else if (offset_column->isColumnConst())
|
||||
{
|
||||
ssize_t offset = offset_column->getUInt(0);
|
||||
|
||||
if (!length_column || length_column->onlyNull())
|
||||
{
|
||||
if (offset > 0)
|
||||
GatherUtils::sliceFromLeftConstantOffsetUnbounded(*source, *sink, static_cast<size_t>(offset - 1));
|
||||
else
|
||||
GatherUtils::sliceFromRightConstantOffsetUnbounded(*source, *sink, static_cast<size_t>(-offset));
|
||||
}
|
||||
else if (length_column->isColumnConst())
|
||||
{
|
||||
ssize_t length = length_column->getInt(0);
|
||||
if (offset > 0)
|
||||
GatherUtils::sliceFromLeftConstantOffsetBounded(*source, *sink, static_cast<size_t>(offset - 1), length);
|
||||
else
|
||||
GatherUtils::sliceFromRightConstantOffsetBounded(*source, *sink, static_cast<size_t>(-offset), length);
|
||||
}
|
||||
else
|
||||
GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *offset_column, *length_column);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!length_column || length_column->onlyNull())
|
||||
GatherUtils::sliceDynamicOffsetUnbounded(*source, *sink, *offset_column);
|
||||
else
|
||||
GatherUtils::sliceDynamicOffsetBounded(*source, *sink, *offset_column, *length_column);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionArraySlice(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArraySlice>();
|
||||
}
|
||||
|
||||
|
||||
}
|
367
dbms/src/Functions/arrayUniq.cpp
Normal file
367
dbms/src/Functions/arrayUniq.cpp
Normal file
@ -0,0 +1,367 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/HashTable/ClearableHashSet.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
/// Counts the number of different elements in the array, or the number of different tuples from the elements at the corresponding positions in several arrays.
|
||||
/// NOTE The implementation partially matches arrayEnumerateUniq.
|
||||
class FunctionArrayUniq : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "arrayUniq";
|
||||
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionArrayUniq>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() == 0)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be at least 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("All arguments for function " + getName() + " must be arrays but argument " +
|
||||
toString(i + 1) + " has type " + arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeUInt32>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
|
||||
private:
|
||||
/// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess.
|
||||
static constexpr size_t INITIAL_SIZE_DEGREE = 9;
|
||||
|
||||
template <typename T>
|
||||
bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values);
|
||||
|
||||
bool execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns);
|
||||
|
||||
void executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values);
|
||||
};
|
||||
|
||||
|
||||
void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
||||
{
|
||||
Columns array_columns(arguments.size());
|
||||
const ColumnArray::Offsets * offsets = nullptr;
|
||||
ColumnRawPtrs data_columns(arguments.size());
|
||||
ColumnRawPtrs original_data_columns(arguments.size());
|
||||
ColumnRawPtrs null_maps(arguments.size());
|
||||
|
||||
bool has_nullable_columns = false;
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
ColumnPtr array_ptr = block.getByPosition(arguments[i]).column;
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(array_ptr.get());
|
||||
if (!array)
|
||||
{
|
||||
const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(
|
||||
block.getByPosition(arguments[i]).column.get());
|
||||
if (!const_array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName()
|
||||
+ " of " + toString(i + 1) + getOrdinalSuffix(i + 1) + " argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
array_ptr = const_array->convertToFullColumn();
|
||||
array = static_cast<const ColumnArray *>(array_ptr.get());
|
||||
}
|
||||
|
||||
array_columns[i] = array_ptr;
|
||||
|
||||
const ColumnArray::Offsets & offsets_i = array->getOffsets();
|
||||
if (i == 0)
|
||||
offsets = &offsets_i;
|
||||
else if (offsets_i != *offsets)
|
||||
throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.",
|
||||
ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
|
||||
data_columns[i] = &array->getData();
|
||||
original_data_columns[i] = data_columns[i];
|
||||
|
||||
if (data_columns[i]->isColumnNullable())
|
||||
{
|
||||
has_nullable_columns = true;
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(*data_columns[i]);
|
||||
data_columns[i] = &nullable_col.getNestedColumn();
|
||||
null_maps[i] = &nullable_col.getNullMapColumn();
|
||||
}
|
||||
else
|
||||
null_maps[i] = nullptr;
|
||||
}
|
||||
|
||||
const ColumnArray * first_array = static_cast<const ColumnArray *>(array_columns[0].get());
|
||||
const IColumn * first_null_map = null_maps[0];
|
||||
auto res = ColumnUInt32::create();
|
||||
|
||||
ColumnUInt32::Container & res_values = res->getData();
|
||||
res_values.resize(offsets->size());
|
||||
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
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)))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns))
|
||||
executeHashed(*offsets, original_data_columns, res_values);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(res);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
{
|
||||
const IColumn * inner_col;
|
||||
|
||||
const auto & array_data = array->getData();
|
||||
if (array_data.isColumnNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnVector<T> * nested = checkAndGetColumn<ColumnVector<T>>(inner_col);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
const typename ColumnVector<T>::Container & values = nested->getData();
|
||||
|
||||
using Set = ClearableHashSet<T, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
Set set;
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
bool found_null = false;
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
found_null = true;
|
||||
else
|
||||
set.insert(values[j]);
|
||||
}
|
||||
|
||||
res_values[i] = set.size() + found_null;
|
||||
prev_off = off;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values)
|
||||
{
|
||||
const IColumn * inner_col;
|
||||
|
||||
const auto & array_data = array->getData();
|
||||
if (array_data.isColumnNullable())
|
||||
{
|
||||
const auto & nullable_col = static_cast<const ColumnNullable &>(array_data);
|
||||
inner_col = &nullable_col.getNestedColumn();
|
||||
}
|
||||
else
|
||||
inner_col = &array_data;
|
||||
|
||||
const ColumnString * nested = checkAndGetColumn<ColumnString>(inner_col);
|
||||
if (!nested)
|
||||
return false;
|
||||
const ColumnArray::Offsets & offsets = array->getOffsets();
|
||||
|
||||
using Set = ClearableHashSet<StringRef, StringRefHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>;
|
||||
|
||||
const PaddedPODArray<UInt8> * null_map_data = nullptr;
|
||||
if (null_map)
|
||||
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
|
||||
|
||||
Set set;
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
bool found_null = false;
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (null_map_data && ((*null_map_data)[j] == 1))
|
||||
found_null = true;
|
||||
else
|
||||
set.insert(nested->getDataAt(j));
|
||||
}
|
||||
|
||||
res_values[i] = set.size() + found_null;
|
||||
prev_off = off;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool FunctionArrayUniq::execute128bit(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
const ColumnRawPtrs & null_maps,
|
||||
ColumnUInt32::Container & res_values,
|
||||
bool has_nullable_columns)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
size_t keys_bytes = 0;
|
||||
Sizes key_sizes(count);
|
||||
|
||||
for (size_t j = 0; j < count; ++j)
|
||||
{
|
||||
if (!columns[j]->isFixedAndContiguous())
|
||||
return false;
|
||||
key_sizes[j] = columns[j]->sizeOfValueIfFixed();
|
||||
keys_bytes += key_sizes[j];
|
||||
}
|
||||
if (has_nullable_columns)
|
||||
keys_bytes += std::tuple_size<KeysNullMap<UInt128>>::value;
|
||||
|
||||
if (keys_bytes > 16)
|
||||
return false;
|
||||
|
||||
using Set = ClearableHashSet<UInt128, UInt128HashCRC32, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
/// Suppose that, for a given row, each of the N columns has an array whose length is M.
|
||||
/// Denote arr_i each of these arrays (1 <= i <= N). Then the following is performed:
|
||||
///
|
||||
/// col1 ... colN
|
||||
///
|
||||
/// arr_1[1], ..., arr_N[1] -> pack into a binary blob b1
|
||||
/// .
|
||||
/// .
|
||||
/// .
|
||||
/// arr_1[M], ..., arr_N[M] -> pack into a binary blob bM
|
||||
///
|
||||
/// Each binary blob is inserted into a hash table.
|
||||
///
|
||||
Set set;
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
{
|
||||
if (has_nullable_columns)
|
||||
{
|
||||
KeysNullMap<UInt128> bitmap{};
|
||||
|
||||
for (ColumnArray::Offset i = 0; i < columns.size(); ++i)
|
||||
{
|
||||
if (null_maps[i])
|
||||
{
|
||||
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[i]).getData();
|
||||
if (null_map[j] == 1)
|
||||
{
|
||||
ColumnArray::Offset bucket = i / 8;
|
||||
ColumnArray::Offset offset = i % 8;
|
||||
bitmap[bucket] |= UInt8(1) << offset;
|
||||
}
|
||||
}
|
||||
}
|
||||
set.insert(packFixed<UInt128>(j, count, columns, key_sizes, bitmap));
|
||||
}
|
||||
else
|
||||
set.insert(packFixed<UInt128>(j, count, columns, key_sizes));
|
||||
}
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void FunctionArrayUniq::executeHashed(
|
||||
const ColumnArray::Offsets & offsets,
|
||||
const ColumnRawPtrs & columns,
|
||||
ColumnUInt32::Container & res_values)
|
||||
{
|
||||
size_t count = columns.size();
|
||||
|
||||
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
|
||||
Set set;
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
set.clear();
|
||||
ColumnArray::Offset off = offsets[i];
|
||||
for (ColumnArray::Offset j = prev_off; j < off; ++j)
|
||||
set.insert(hash128(j, count, columns));
|
||||
|
||||
res_values[i] = set.size();
|
||||
prev_off = off;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionArrayUniq(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayUniq>();
|
||||
}
|
||||
|
||||
}
|
193
dbms/src/Functions/concat.cpp
Normal file
193
dbms/src/Functions/concat.cpp
Normal file
@ -0,0 +1,193 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <Functions/GatherUtils/Sinks.h>
|
||||
#include <Functions/GatherUtils/Slices.h>
|
||||
#include <Functions/GatherUtils/Algorithms.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <ext/range.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
using namespace GatherUtils;
|
||||
|
||||
|
||||
template <typename Name, bool is_injective>
|
||||
class ConcatImpl : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
ConcatImpl(const Context & context) : context(context) {}
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<ConcatImpl>(context);
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return is_injective;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() < 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be at least 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (const auto arg_idx : ext::range(0, arguments.size()))
|
||||
{
|
||||
const auto arg = arguments[arg_idx].get();
|
||||
if (!isStringOrFixedString(arg))
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
if (arguments.size() == 2)
|
||||
executeBinary(block, arguments, result, input_rows_count);
|
||||
else
|
||||
executeNAry(block, arguments, result, input_rows_count);
|
||||
}
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
|
||||
void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
const IColumn * c0 = block.getByPosition(arguments[0]).column.get();
|
||||
const IColumn * c1 = block.getByPosition(arguments[1]).column.get();
|
||||
|
||||
const ColumnString * c0_string = checkAndGetColumn<ColumnString>(c0);
|
||||
const ColumnString * c1_string = checkAndGetColumn<ColumnString>(c1);
|
||||
const ColumnConst * c0_const_string = checkAndGetColumnConst<ColumnString>(c0);
|
||||
const ColumnConst * c1_const_string = checkAndGetColumnConst<ColumnString>(c1);
|
||||
|
||||
auto c_res = ColumnString::create();
|
||||
|
||||
if (c0_string && c1_string)
|
||||
concat(StringSource(*c0_string), StringSource(*c1_string), StringSink(*c_res, c0->size()));
|
||||
else if (c0_string && c1_const_string)
|
||||
concat(StringSource(*c0_string), ConstSource<StringSource>(*c1_const_string), StringSink(*c_res, c0->size()));
|
||||
else if (c0_const_string && c1_string)
|
||||
concat(ConstSource<StringSource>(*c0_const_string), StringSource(*c1_string), StringSink(*c_res, c0->size()));
|
||||
else
|
||||
{
|
||||
/// Fallback: use generic implementation for not very important cases.
|
||||
executeNAry(block, arguments, result, input_rows_count);
|
||||
return;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(c_res);
|
||||
}
|
||||
|
||||
void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
size_t num_sources = arguments.size();
|
||||
StringSources sources(num_sources);
|
||||
|
||||
for (size_t i = 0; i < num_sources; ++i)
|
||||
sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column);
|
||||
|
||||
auto c_res = ColumnString::create();
|
||||
concat(sources, StringSink(*c_res, input_rows_count));
|
||||
block.getByPosition(result).column = std::move(c_res);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameConcat
|
||||
{
|
||||
static constexpr auto name = "concat";
|
||||
};
|
||||
struct NameConcatAssumeInjective
|
||||
{
|
||||
static constexpr auto name = "concatAssumeInjective";
|
||||
};
|
||||
|
||||
using FunctionConcat = ConcatImpl<NameConcat, false>;
|
||||
using FunctionConcatAssumeInjective = ConcatImpl<NameConcatAssumeInjective, true>;
|
||||
|
||||
|
||||
/// Also works with arrays.
|
||||
class FunctionBuilderConcat : public FunctionBuilderImpl
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "concat";
|
||||
static FunctionBuilderPtr create(const Context & context) { return std::make_shared<FunctionBuilderConcat>(context); }
|
||||
|
||||
FunctionBuilderConcat(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool isVariadic() const override { return true; }
|
||||
|
||||
protected:
|
||||
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||
{
|
||||
if (isArray(arguments.at(0).type))
|
||||
return FunctionFactory::instance().get("arrayConcat", context)->build(arguments);
|
||||
else
|
||||
return std::make_shared<DefaultFunction>(
|
||||
FunctionConcat::create(context),
|
||||
ext::map<DataTypes>(arguments, [](const auto & elem) { return elem.type; }),
|
||||
return_type);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() < 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size())
|
||||
+ ", should be at least 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return getLeastSupertype(arguments);
|
||||
}
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionsConcat(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBuilderConcat>();
|
||||
factory.registerFunction<FunctionConcatAssumeInjective>();
|
||||
}
|
||||
|
||||
}
|
18
dbms/src/Functions/countEqual.cpp
Normal file
18
dbms/src/Functions/countEqual.cpp
Normal file
@ -0,0 +1,18 @@
|
||||
#include <Functions/arrayIndex.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameCountEqual { static constexpr auto name = "countEqual"; };
|
||||
|
||||
using FunctionCountEqual = FunctionArrayIndex<IndexCount, NameCountEqual>;
|
||||
|
||||
void registerFunctionCountEqual(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionCountEqual>();
|
||||
}
|
||||
|
||||
|
||||
}
|
22
dbms/src/Functions/empty.cpp
Normal file
22
dbms/src/Functions/empty.cpp
Normal file
@ -0,0 +1,22 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringOrArrayToT.h>
|
||||
#include <Functions/EmptyImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameEmpty
|
||||
{
|
||||
static constexpr auto name = "empty";
|
||||
};
|
||||
using FunctionEmpty = FunctionStringOrArrayToT<EmptyImpl<false>, NameEmpty, UInt8>;
|
||||
|
||||
void registerFunctionEmpty(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionEmpty>();
|
||||
}
|
||||
|
||||
}
|
||||
|
92
dbms/src/Functions/emptyArray.cpp
Normal file
92
dbms/src/Functions/emptyArray.cpp
Normal file
@ -0,0 +1,92 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// TODO Make it simple.
|
||||
|
||||
template <typename Type> struct TypeToColumnType { using ColumnType = ColumnVector<Type>; };
|
||||
template <> struct TypeToColumnType<String> { using ColumnType = ColumnString; };
|
||||
|
||||
template <typename DataType> struct DataTypeToName : TypeName<typename DataType::FieldType> { };
|
||||
template <> struct DataTypeToName<DataTypeDate> { static std::string get() { return "Date"; } };
|
||||
template <> struct DataTypeToName<DataTypeDateTime> { static std::string get() { return "DateTime"; } };
|
||||
|
||||
|
||||
template <typename DataType>
|
||||
struct FunctionEmptyArray : public IFunction
|
||||
{
|
||||
static constexpr auto base_name = "emptyArray";
|
||||
static const String name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionEmptyArray>(); }
|
||||
|
||||
private:
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||
{
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataType>());
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
using UnderlyingColumnType = typename TypeToColumnType<typename DataType::FieldType>::ColumnType;
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(
|
||||
UnderlyingColumnType::create(),
|
||||
ColumnArray::ColumnOffsets::create(input_rows_count, 0));
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename DataType>
|
||||
const String FunctionEmptyArray<DataType>::name = FunctionEmptyArray::base_name + String(DataTypeToName<DataType>::get());
|
||||
|
||||
using FunctionEmptyArrayUInt8 = FunctionEmptyArray<DataTypeUInt8>;
|
||||
using FunctionEmptyArrayUInt16 = FunctionEmptyArray<DataTypeUInt16>;
|
||||
using FunctionEmptyArrayUInt32 = FunctionEmptyArray<DataTypeUInt32>;
|
||||
using FunctionEmptyArrayUInt64 = FunctionEmptyArray<DataTypeUInt64>;
|
||||
using FunctionEmptyArrayInt8 = FunctionEmptyArray<DataTypeInt8>;
|
||||
using FunctionEmptyArrayInt16 = FunctionEmptyArray<DataTypeInt16>;
|
||||
using FunctionEmptyArrayInt32 = FunctionEmptyArray<DataTypeInt32>;
|
||||
using FunctionEmptyArrayInt64 = FunctionEmptyArray<DataTypeInt64>;
|
||||
using FunctionEmptyArrayFloat32 = FunctionEmptyArray<DataTypeFloat32>;
|
||||
using FunctionEmptyArrayFloat64 = FunctionEmptyArray<DataTypeFloat64>;
|
||||
using FunctionEmptyArrayDate = FunctionEmptyArray<DataTypeDate>;
|
||||
using FunctionEmptyArrayDateTime = FunctionEmptyArray<DataTypeDateTime>;
|
||||
using FunctionEmptyArrayString = FunctionEmptyArray<DataTypeString>;
|
||||
|
||||
|
||||
void registerFunctionsEmptyArray(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionEmptyArrayUInt8>();
|
||||
factory.registerFunction<FunctionEmptyArrayUInt16>();
|
||||
factory.registerFunction<FunctionEmptyArrayUInt32>();
|
||||
factory.registerFunction<FunctionEmptyArrayUInt64>();
|
||||
factory.registerFunction<FunctionEmptyArrayInt8>();
|
||||
factory.registerFunction<FunctionEmptyArrayInt16>();
|
||||
factory.registerFunction<FunctionEmptyArrayInt32>();
|
||||
factory.registerFunction<FunctionEmptyArrayInt64>();
|
||||
factory.registerFunction<FunctionEmptyArrayFloat32>();
|
||||
factory.registerFunction<FunctionEmptyArrayFloat64>();
|
||||
factory.registerFunction<FunctionEmptyArrayDate>();
|
||||
factory.registerFunction<FunctionEmptyArrayDateTime>();
|
||||
factory.registerFunction<FunctionEmptyArrayString>();
|
||||
}
|
||||
|
||||
}
|
425
dbms/src/Functions/emptyArrayToSingle.cpp
Normal file
425
dbms/src/Functions/emptyArrayToSingle.cpp
Normal file
@ -0,0 +1,425 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/** emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value.
|
||||
*/
|
||||
class FunctionEmptyArrayToSingle : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "emptyArrayToSingle";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionEmptyArrayToSingle>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument for function " + getName() + " must be array.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
|
||||
};
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
namespace FunctionEmptyArrayToSingleImpl
|
||||
{
|
||||
bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
if (const ColumnConst * const_array = checkAndGetColumnConst<ColumnArray>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
if (const_array->getValue<Array>().empty())
|
||||
{
|
||||
auto nested_type = typeid_cast<const DataTypeArray &>(*block.getByPosition(arguments[0]).type).getNestedType();
|
||||
|
||||
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
|
||||
input_rows_count,
|
||||
Array{nested_type->getDefault()});
|
||||
}
|
||||
else
|
||||
block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T, bool nullable>
|
||||
bool executeNumber(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets & res_offsets,
|
||||
const NullMap * src_null_map,
|
||||
NullMap * res_null_map)
|
||||
{
|
||||
if (const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data))
|
||||
{
|
||||
const PaddedPODArray<T> & src_data = src_data_concrete->getData();
|
||||
PaddedPODArray<T> & res_data = static_cast<ColumnVector<T> &>(res_data_col).getData();
|
||||
|
||||
size_t size = src_offsets.size();
|
||||
res_offsets.resize(size);
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
if (nullable)
|
||||
res_null_map->reserve(src_null_map->size());
|
||||
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
ColumnArray::Offset res_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_offsets[i] != src_prev_offset)
|
||||
{
|
||||
size_t size_to_write = src_offsets[i] - src_prev_offset;
|
||||
res_data.resize(res_prev_offset + size_to_write);
|
||||
memcpy(&res_data[res_prev_offset], &src_data[src_prev_offset], size_to_write * sizeof(T));
|
||||
|
||||
if (nullable)
|
||||
{
|
||||
res_null_map->resize(res_prev_offset + size_to_write);
|
||||
memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write);
|
||||
}
|
||||
|
||||
res_prev_offset += size_to_write;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data.push_back(T());
|
||||
++res_prev_offset;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
|
||||
if (nullable)
|
||||
res_null_map->push_back(1); /// Push NULL.
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
template <bool nullable>
|
||||
bool executeFixedString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets & res_offsets,
|
||||
const NullMap * src_null_map,
|
||||
NullMap * res_null_map)
|
||||
{
|
||||
if (const ColumnFixedString * src_data_concrete = checkAndGetColumn<ColumnFixedString>(&src_data))
|
||||
{
|
||||
const size_t n = src_data_concrete->getN();
|
||||
const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
|
||||
auto concrete_res_data = typeid_cast<ColumnFixedString *>(&res_data_col);
|
||||
if (!concrete_res_data)
|
||||
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
||||
|
||||
ColumnFixedString::Chars_t & res_data = concrete_res_data->getChars();
|
||||
size_t size = src_offsets.size();
|
||||
res_offsets.resize(size);
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
if (nullable)
|
||||
res_null_map->reserve(src_null_map->size());
|
||||
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
ColumnArray::Offset res_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_offsets[i] != src_prev_offset)
|
||||
{
|
||||
size_t size_to_write = src_offsets[i] - src_prev_offset;
|
||||
size_t prev_res_data_size = res_data.size();
|
||||
res_data.resize(prev_res_data_size + size_to_write * n);
|
||||
memcpy(&res_data[prev_res_data_size], &src_data[src_prev_offset * n], size_to_write * n);
|
||||
|
||||
if (nullable)
|
||||
{
|
||||
res_null_map->resize(res_prev_offset + size_to_write);
|
||||
memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write);
|
||||
}
|
||||
|
||||
res_prev_offset += size_to_write;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t prev_res_data_size = res_data.size();
|
||||
res_data.resize(prev_res_data_size + n);
|
||||
memset(&res_data[prev_res_data_size], 0, n);
|
||||
++res_prev_offset;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
|
||||
if (nullable)
|
||||
res_null_map->push_back(1);
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
template <bool nullable>
|
||||
bool executeString(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets & res_array_offsets,
|
||||
const NullMap * src_null_map,
|
||||
NullMap * res_null_map)
|
||||
{
|
||||
if (const ColumnString * src_data_concrete = checkAndGetColumn<ColumnString>(&src_data))
|
||||
{
|
||||
const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets();
|
||||
|
||||
auto concrete_res_string_offsets = typeid_cast<ColumnString *>(&res_data_col);
|
||||
if (!concrete_res_string_offsets)
|
||||
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
||||
ColumnString::Offsets & res_string_offsets = concrete_res_string_offsets->getOffsets();
|
||||
|
||||
const ColumnString::Chars_t & src_data = src_data_concrete->getChars();
|
||||
|
||||
auto concrete_res_data = typeid_cast<ColumnString *>(&res_data_col);
|
||||
if (!concrete_res_data)
|
||||
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
||||
ColumnString::Chars_t & res_data = concrete_res_data->getChars();
|
||||
|
||||
size_t size = src_array_offsets.size();
|
||||
res_array_offsets.resize(size);
|
||||
res_string_offsets.reserve(src_string_offsets.size());
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
if (nullable)
|
||||
res_null_map->reserve(src_null_map->size());
|
||||
|
||||
ColumnArray::Offset src_array_prev_offset = 0;
|
||||
ColumnArray::Offset res_array_prev_offset = 0;
|
||||
|
||||
ColumnString::Offset src_string_prev_offset = 0;
|
||||
ColumnString::Offset res_string_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_array_offsets[i] != src_array_prev_offset)
|
||||
{
|
||||
size_t array_size = src_array_offsets[i] - src_array_prev_offset;
|
||||
|
||||
size_t bytes_to_copy = 0;
|
||||
size_t from_string_prev_offset_local = src_string_prev_offset;
|
||||
for (size_t j = 0; j < array_size; ++j)
|
||||
{
|
||||
size_t string_size = src_string_offsets[src_array_prev_offset + j] - from_string_prev_offset_local;
|
||||
|
||||
res_string_prev_offset += string_size;
|
||||
res_string_offsets.push_back(res_string_prev_offset);
|
||||
|
||||
from_string_prev_offset_local += string_size;
|
||||
bytes_to_copy += string_size;
|
||||
}
|
||||
|
||||
size_t res_data_old_size = res_data.size();
|
||||
res_data.resize(res_data_old_size + bytes_to_copy);
|
||||
memcpy(&res_data[res_data_old_size], &src_data[src_string_prev_offset], bytes_to_copy);
|
||||
|
||||
if (nullable)
|
||||
{
|
||||
res_null_map->resize(res_array_prev_offset + array_size);
|
||||
memcpy(&(*res_null_map)[res_array_prev_offset], &(*src_null_map)[src_array_prev_offset], array_size);
|
||||
}
|
||||
|
||||
res_array_prev_offset += array_size;
|
||||
res_array_offsets[i] = res_array_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data.push_back(0); /// An empty string, including zero at the end.
|
||||
|
||||
if (nullable)
|
||||
res_null_map->push_back(1);
|
||||
|
||||
++res_string_prev_offset;
|
||||
res_string_offsets.push_back(res_string_prev_offset);
|
||||
|
||||
++res_array_prev_offset;
|
||||
res_array_offsets[i] = res_array_prev_offset;
|
||||
}
|
||||
|
||||
src_array_prev_offset = src_array_offsets[i];
|
||||
|
||||
if (src_array_prev_offset)
|
||||
src_string_prev_offset = src_string_offsets[src_array_prev_offset - 1];
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
template <bool nullable>
|
||||
void executeGeneric(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_offsets,
|
||||
IColumn & res_data, ColumnArray::Offsets & res_offsets,
|
||||
const NullMap * src_null_map,
|
||||
NullMap * res_null_map)
|
||||
{
|
||||
size_t size = src_offsets.size();
|
||||
res_offsets.resize(size);
|
||||
res_data.reserve(src_data.size());
|
||||
|
||||
if (nullable)
|
||||
res_null_map->reserve(src_null_map->size());
|
||||
|
||||
ColumnArray::Offset src_prev_offset = 0;
|
||||
ColumnArray::Offset res_prev_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (src_offsets[i] != src_prev_offset)
|
||||
{
|
||||
size_t size_to_write = src_offsets[i] - src_prev_offset;
|
||||
res_data.insertRangeFrom(src_data, src_prev_offset, size_to_write);
|
||||
|
||||
if (nullable)
|
||||
{
|
||||
res_null_map->resize(res_prev_offset + size_to_write);
|
||||
memcpy(&(*res_null_map)[res_prev_offset], &(*src_null_map)[src_prev_offset], size_to_write);
|
||||
}
|
||||
|
||||
res_prev_offset += size_to_write;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data.insertDefault();
|
||||
++res_prev_offset;
|
||||
res_offsets[i] = res_prev_offset;
|
||||
|
||||
if (nullable)
|
||||
res_null_map->push_back(1);
|
||||
}
|
||||
|
||||
src_prev_offset = src_offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <bool nullable>
|
||||
void executeDispatch(
|
||||
const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets,
|
||||
IColumn & res_data_col, ColumnArray::Offsets & res_array_offsets,
|
||||
const NullMap * src_null_map,
|
||||
NullMap * res_null_map)
|
||||
{
|
||||
if (!( executeNumber<UInt8, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<UInt16, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<UInt32, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<UInt64, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<Int8, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<Int16, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<Int32, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<Int64, nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<Float32, nullable>(src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeNumber<Float64, nullable>(src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeString<nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)
|
||||
|| executeFixedString<nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map)))
|
||||
executeGeneric<nullable> (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void FunctionEmptyArrayToSingle::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
{
|
||||
if (FunctionEmptyArrayToSingleImpl::executeConst(block, arguments, result, input_rows_count))
|
||||
return;
|
||||
|
||||
const ColumnArray * array = checkAndGetColumn<ColumnArray>(block.getByPosition(arguments[0]).column.get());
|
||||
if (!array)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
MutableColumnPtr res_ptr = array->cloneEmpty();
|
||||
ColumnArray & res = static_cast<ColumnArray &>(*res_ptr);
|
||||
|
||||
const IColumn & src_data = array->getData();
|
||||
const ColumnArray::Offsets & src_offsets = array->getOffsets();
|
||||
IColumn & res_data = res.getData();
|
||||
ColumnArray::Offsets & res_offsets = res.getOffsets();
|
||||
|
||||
const NullMap * src_null_map = nullptr;
|
||||
NullMap * res_null_map = nullptr;
|
||||
|
||||
const IColumn * inner_col;
|
||||
IColumn * inner_res_col;
|
||||
|
||||
bool nullable = src_data.isColumnNullable();
|
||||
if (nullable)
|
||||
{
|
||||
auto nullable_col = static_cast<const ColumnNullable *>(&src_data);
|
||||
inner_col = &nullable_col->getNestedColumn();
|
||||
src_null_map = &nullable_col->getNullMapData();
|
||||
|
||||
auto nullable_res_col = static_cast<ColumnNullable *>(&res_data);
|
||||
inner_res_col = &nullable_res_col->getNestedColumn();
|
||||
res_null_map = &nullable_res_col->getNullMapData();
|
||||
}
|
||||
else
|
||||
{
|
||||
inner_col = &src_data;
|
||||
inner_res_col = &res_data;
|
||||
}
|
||||
|
||||
if (nullable)
|
||||
FunctionEmptyArrayToSingleImpl::executeDispatch<true>(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map);
|
||||
else
|
||||
FunctionEmptyArrayToSingleImpl::executeDispatch<false>(*inner_col, src_offsets, *inner_res_col, res_offsets, src_null_map, res_null_map);
|
||||
|
||||
block.getByPosition(result).column = std::move(res_ptr);
|
||||
}
|
||||
|
||||
|
||||
void registerFunctionEmptyArrayToSingle(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionEmptyArrayToSingle>();
|
||||
}
|
||||
|
||||
}
|
16
dbms/src/Functions/endsWith.cpp
Normal file
16
dbms/src/Functions/endsWith.cpp
Normal file
@ -0,0 +1,16 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStartsEndsWith.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using FunctionEndsWith = FunctionStartsEndsWith<NameEndsWith>;
|
||||
|
||||
void registerFunctionEndsWith(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionEndsWith>();
|
||||
}
|
||||
}
|
||||
|
18
dbms/src/Functions/has.cpp
Normal file
18
dbms/src/Functions/has.cpp
Normal file
@ -0,0 +1,18 @@
|
||||
#include <Functions/arrayIndex.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameHas { static constexpr auto name = "has"; };
|
||||
|
||||
/// has(arr, x) - whether there is an element x in the array.
|
||||
using FunctionHas = FunctionArrayIndex<IndexToOne, NameHas>;
|
||||
|
||||
void registerFunctionHas(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionHas>();
|
||||
}
|
||||
|
||||
}
|
21
dbms/src/Functions/hasAll.cpp
Normal file
21
dbms/src/Functions/hasAll.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Functions/hasAllAny.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionArrayHasAll : public FunctionArrayHasAllAny
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hasAll";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayHasAll>(context); }
|
||||
FunctionArrayHasAll(const Context & context) : FunctionArrayHasAllAny(context, true, name) {}
|
||||
};
|
||||
|
||||
void registerFunctionHasAll(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayHasAll>();
|
||||
}
|
||||
|
||||
}
|
123
dbms/src/Functions/hasAllAny.h
Normal file
123
dbms/src/Functions/hasAllAny.h
Normal file
@ -0,0 +1,123 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
class FunctionArrayHasAllAny : public IFunction
|
||||
{
|
||||
public:
|
||||
FunctionArrayHasAllAny(const Context & context, bool all, const char * name)
|
||||
: context(context), all(all), name(name) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
bool isVariadic() const override { return false; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
for (auto i : ext::range(0, arguments.size()))
|
||||
{
|
||||
auto array_type = typeid_cast<const DataTypeArray *>(arguments[i].get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
size_t rows = input_rows_count;
|
||||
size_t num_args = arguments.size();
|
||||
|
||||
auto result_column = ColumnUInt8::create(rows);
|
||||
|
||||
DataTypePtr common_type = nullptr;
|
||||
auto commonType = [& common_type, & block, & arguments]()
|
||||
{
|
||||
if (common_type == nullptr)
|
||||
{
|
||||
DataTypes data_types;
|
||||
data_types.reserve(arguments.size());
|
||||
for (const auto & argument : arguments)
|
||||
data_types.push_back(block.getByPosition(argument).type);
|
||||
|
||||
common_type = getLeastSupertype(data_types);
|
||||
}
|
||||
|
||||
return common_type;
|
||||
};
|
||||
|
||||
Columns preprocessed_columns(num_args);
|
||||
|
||||
for (size_t i = 0; i < num_args; ++i)
|
||||
{
|
||||
const auto & argument = block.getByPosition(arguments[i]);
|
||||
ColumnPtr preprocessed_column = argument.column;
|
||||
|
||||
const auto argument_type = typeid_cast<const DataTypeArray *>(argument.type.get());
|
||||
const auto & nested_type = argument_type->getNestedType();
|
||||
|
||||
/// Converts Array(Nothing) or Array(Nullable(Nothing) to common type. Example: hasAll([Null, 1], [Null]) -> 1
|
||||
if (typeid_cast<const DataTypeNothing *>(removeNullable(nested_type).get()))
|
||||
preprocessed_column = castColumn(argument, commonType(), context);
|
||||
|
||||
preprocessed_columns[i] = std::move(preprocessed_column);
|
||||
}
|
||||
|
||||
std::vector<std::unique_ptr<GatherUtils::IArraySource>> sources;
|
||||
|
||||
for (auto & argument_column : preprocessed_columns)
|
||||
{
|
||||
bool is_const = false;
|
||||
|
||||
if (auto argument_column_const = typeid_cast<const ColumnConst *>(argument_column.get()))
|
||||
{
|
||||
is_const = true;
|
||||
argument_column = argument_column_const->getDataColumnPtr();
|
||||
}
|
||||
|
||||
if (auto argument_column_array = typeid_cast<const ColumnArray *>(argument_column.get()))
|
||||
sources.emplace_back(GatherUtils::createArraySource(*argument_column_array, is_const, rows));
|
||||
else
|
||||
throw Exception{"Arguments for function " + getName() + " must be arrays.", ErrorCodes::LOGICAL_ERROR};
|
||||
}
|
||||
|
||||
auto result_column_ptr = typeid_cast<ColumnUInt8 *>(result_column.get());
|
||||
GatherUtils::sliceHas(*sources[0], *sources[1], all, *result_column_ptr);
|
||||
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
bool all;
|
||||
const char * name;
|
||||
};
|
||||
|
||||
}
|
21
dbms/src/Functions/hasAny.cpp
Normal file
21
dbms/src/Functions/hasAny.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <Functions/hasAllAny.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionArrayHasAny : public FunctionArrayHasAllAny
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "hasAny";
|
||||
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionArrayHasAny>(context); }
|
||||
FunctionArrayHasAny(const Context & context) : FunctionArrayHasAllAny(context, false, name) {}
|
||||
};
|
||||
|
||||
void registerFunctionHasAny(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayHasAny>();
|
||||
}
|
||||
|
||||
}
|
19
dbms/src/Functions/indexOf.cpp
Normal file
19
dbms/src/Functions/indexOf.cpp
Normal file
@ -0,0 +1,19 @@
|
||||
#include <Functions/arrayIndex.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameIndexOf { static constexpr auto name = "indexOf"; };
|
||||
|
||||
/// indexOf(arr, x) - returns the index of the element x (starting with 1), if it exists in the array, or 0 if it is not.
|
||||
using FunctionIndexOf = FunctionArrayIndex<IndexIdentity, NameIndexOf>;
|
||||
|
||||
void registerFunctionIndexOf(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionIndexOf>();
|
||||
}
|
||||
|
||||
|
||||
}
|
53
dbms/src/Functions/length.cpp
Normal file
53
dbms/src/Functions/length.cpp
Normal file
@ -0,0 +1,53 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringOrArrayToT.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Calculates the length of a string in bytes.
|
||||
*/
|
||||
struct LengthImpl
|
||||
{
|
||||
static constexpr auto is_fixed_to_constant = true;
|
||||
|
||||
static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]);
|
||||
}
|
||||
|
||||
static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t n, UInt64 & res)
|
||||
{
|
||||
res = n;
|
||||
}
|
||||
|
||||
static void vector_fixed_to_vector(const ColumnString::Chars_t & /*data*/, size_t /*n*/, PaddedPODArray<UInt64> & /*res*/)
|
||||
{
|
||||
}
|
||||
|
||||
static void array(const ColumnString::Offsets & offsets, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NameLength
|
||||
{
|
||||
static constexpr auto name = "length";
|
||||
};
|
||||
|
||||
using FunctionLength = FunctionStringOrArrayToT<LengthImpl, NameLength, UInt64>;
|
||||
|
||||
void registerFunctionLength(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionLength>();
|
||||
}
|
||||
|
||||
}
|
68
dbms/src/Functions/lengthUTF8.cpp
Normal file
68
dbms/src/Functions/lengthUTF8.cpp
Normal file
@ -0,0 +1,68 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringOrArrayToT.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/** If the string is UTF-8 encoded text, it returns the length of the text in code points.
|
||||
* (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization)
|
||||
* (not in characters: the length of the text "" can be either 1 or 2, depending on the normalization)
|
||||
* Otherwise, the behavior is undefined.
|
||||
*/
|
||||
struct LengthUTF8Impl
|
||||
{
|
||||
static constexpr auto is_fixed_to_constant = false;
|
||||
|
||||
static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = UTF8::countCodePoints(&data[prev_offset], offsets[i] - prev_offset - 1);
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt64 & /*res*/)
|
||||
{
|
||||
}
|
||||
|
||||
static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray<UInt64> & res)
|
||||
{
|
||||
size_t size = data.size() / n;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = UTF8::countCodePoints(&data[i * n], n);
|
||||
}
|
||||
}
|
||||
|
||||
static void array(const ColumnString::Offsets &, PaddedPODArray<UInt64> &)
|
||||
{
|
||||
throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
};
|
||||
|
||||
struct NameLengthUTF8
|
||||
{
|
||||
static constexpr auto name = "lengthUTF8";
|
||||
};
|
||||
using FunctionLengthUTF8 = FunctionStringOrArrayToT<LengthUTF8Impl, NameLengthUTF8, UInt64>;
|
||||
|
||||
void registerFunctionLengthUTF8(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionLengthUTF8>();
|
||||
}
|
||||
|
||||
}
|
21
dbms/src/Functions/lower.cpp
Normal file
21
dbms/src/Functions/lower.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include <Functions/LowerUpperImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameLower
|
||||
{
|
||||
static constexpr auto name = "lower";
|
||||
};
|
||||
using FunctionLower = FunctionStringToString<LowerUpperImpl<'A', 'Z'>, NameLower>;
|
||||
|
||||
void registerFunctionLower(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionLower>();
|
||||
}
|
||||
|
||||
}
|
23
dbms/src/Functions/lowerUTF8.cpp
Normal file
23
dbms/src/Functions/lowerUTF8.cpp
Normal file
@ -0,0 +1,23 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include <Functions/LowerUpperUTF8Impl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Poco/Unicode.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameLowerUTF8
|
||||
{
|
||||
static constexpr auto name = "lowerUTF8";
|
||||
};
|
||||
|
||||
using FunctionLowerUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>;
|
||||
|
||||
void registerFunctionLowerUTF8(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionLowerUTF8>();
|
||||
}
|
||||
|
||||
}
|
21
dbms/src/Functions/notEmpty.cpp
Normal file
21
dbms/src/Functions/notEmpty.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringOrArrayToT.h>
|
||||
#include <Functions/EmptyImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameNotEmpty
|
||||
{
|
||||
static constexpr auto name = "notEmpty";
|
||||
};
|
||||
using FunctionNotEmpty = FunctionStringOrArrayToT<EmptyImpl<true>, NameNotEmpty, UInt8>;
|
||||
|
||||
void registerFunctionNotEmpty(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionNotEmpty>();
|
||||
}
|
||||
|
||||
}
|
111
dbms/src/Functions/range.cpp
Normal file
111
dbms/src/Functions/range.cpp
Normal file
@ -0,0 +1,111 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <numeric>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
class FunctionRange : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "range";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionRange>(); }
|
||||
|
||||
private:
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const DataTypePtr & arg = arguments.front();
|
||||
|
||||
if (!isUnsignedInteger(arg))
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
return std::make_shared<DataTypeArray>(arg);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeInternal(Block & block, const IColumn * arg, const size_t result)
|
||||
{
|
||||
static constexpr size_t max_elements = 100'000'000;
|
||||
|
||||
if (const auto in = checkAndGetColumn<ColumnVector<T>>(arg))
|
||||
{
|
||||
const auto & in_data = in->getData();
|
||||
const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{},
|
||||
[this] (const size_t lhs, const size_t rhs)
|
||||
{
|
||||
const auto sum = lhs + rhs;
|
||||
if (sum < lhs)
|
||||
throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing",
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
||||
|
||||
return sum;
|
||||
});
|
||||
|
||||
if (total_values > max_elements)
|
||||
throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) +
|
||||
" array elements, which is greater than the allowed maximum of " + std::to_string(max_elements),
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
|
||||
|
||||
auto data_col = ColumnVector<T>::create(total_values);
|
||||
auto offsets_col = ColumnArray::ColumnOffsets::create(in->size());
|
||||
|
||||
auto & out_data = data_col->getData();
|
||||
auto & out_offsets = offsets_col->getData();
|
||||
|
||||
IColumn::Offset offset{};
|
||||
for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx)
|
||||
{
|
||||
for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx)
|
||||
out_data[offset + elem_idx] = elem_idx;
|
||||
|
||||
offset += in_data[row_idx];
|
||||
out_offsets[row_idx] = offset;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col));
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
|
||||
{
|
||||
const auto col = block.getByPosition(arguments[0]).column.get();
|
||||
|
||||
if (!executeInternal<UInt8>(block, col, result) &&
|
||||
!executeInternal<UInt16>(block, col, result) &&
|
||||
!executeInternal<UInt32>(block, col, result) &&
|
||||
!executeInternal<UInt64>(block, col, result))
|
||||
{
|
||||
throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionRange(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionRange>();
|
||||
}
|
||||
|
||||
}
|
62
dbms/src/Functions/registerFunctionsArray.cpp
Normal file
62
dbms/src/Functions/registerFunctionsArray.cpp
Normal file
@ -0,0 +1,62 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionArray(FunctionFactory &);
|
||||
void registerFunctionArrayElement(FunctionFactory &);
|
||||
void registerFunctionArrayResize(FunctionFactory &);
|
||||
void registerFunctionHas(FunctionFactory &);
|
||||
void registerFunctionHasAll(FunctionFactory &);
|
||||
void registerFunctionHasAny(FunctionFactory &);
|
||||
void registerFunctionIndexOf(FunctionFactory &);
|
||||
void registerFunctionCountEqual(FunctionFactory &);
|
||||
void registerFunctionArrayIntersect(FunctionFactory &);
|
||||
void registerFunctionArrayPushFront(FunctionFactory &);
|
||||
void registerFunctionArrayPushBack(FunctionFactory &);
|
||||
void registerFunctionArrayPopFront(FunctionFactory &);
|
||||
void registerFunctionArrayPopBack(FunctionFactory &);
|
||||
void registerFunctionArrayConcat(FunctionFactory &);
|
||||
void registerFunctionArraySlice(FunctionFactory &);
|
||||
void registerFunctionArrayReverse(FunctionFactory &);
|
||||
void registerFunctionArrayReduce(FunctionFactory &);
|
||||
void registerFunctionRange(FunctionFactory &);
|
||||
void registerFunctionsEmptyArray(FunctionFactory &);
|
||||
void registerFunctionEmptyArrayToSingle(FunctionFactory &);
|
||||
void registerFunctionArrayEnumerate(FunctionFactory &);
|
||||
void registerFunctionArrayEnumerateUniq(FunctionFactory &);
|
||||
void registerFunctionArrayEnumerateDense(FunctionFactory &);
|
||||
void registerFunctionArrayUniq(FunctionFactory &);
|
||||
void registerFunctionArrayDistinct(FunctionFactory &);
|
||||
|
||||
void registerFunctionsArray(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionArray(factory);
|
||||
registerFunctionArrayElement(factory);
|
||||
registerFunctionArrayResize(factory);
|
||||
registerFunctionHas(factory);
|
||||
registerFunctionHasAll(factory);
|
||||
registerFunctionHasAny(factory);
|
||||
registerFunctionIndexOf(factory);
|
||||
registerFunctionCountEqual(factory);
|
||||
registerFunctionArrayIntersect(factory);
|
||||
registerFunctionArrayPushFront(factory);
|
||||
registerFunctionArrayPushBack(factory);
|
||||
registerFunctionArrayPopFront(factory);
|
||||
registerFunctionArrayPopBack(factory);
|
||||
registerFunctionArrayConcat(factory);
|
||||
registerFunctionArraySlice(factory);
|
||||
registerFunctionArrayReverse(factory);
|
||||
registerFunctionArrayReduce(factory);
|
||||
registerFunctionRange(factory);
|
||||
registerFunctionsEmptyArray(factory);
|
||||
registerFunctionEmptyArrayToSingle(factory);
|
||||
registerFunctionArrayEnumerate(factory);
|
||||
registerFunctionArrayEnumerateUniq(factory);
|
||||
registerFunctionArrayEnumerateDense(factory);
|
||||
registerFunctionArrayUniq(factory);
|
||||
registerFunctionArrayDistinct(factory);
|
||||
}
|
||||
|
||||
}
|
||||
|
44
dbms/src/Functions/registerFunctionsString.cpp
Normal file
44
dbms/src/Functions/registerFunctionsString.cpp
Normal file
@ -0,0 +1,44 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionEmpty(FunctionFactory &);
|
||||
void registerFunctionNotEmpty(FunctionFactory &);
|
||||
void registerFunctionLength(FunctionFactory &);
|
||||
void registerFunctionLengthUTF8(FunctionFactory &);
|
||||
void registerFunctionLower(FunctionFactory &);
|
||||
void registerFunctionUpper(FunctionFactory &);
|
||||
void registerFunctionLowerUTF8(FunctionFactory &);
|
||||
void registerFunctionUpperUTF8(FunctionFactory &);
|
||||
void registerFunctionReverse(FunctionFactory &);
|
||||
void registerFunctionReverseUTF8(FunctionFactory &);
|
||||
void registerFunctionsConcat(FunctionFactory &);
|
||||
void registerFunctionSubstring(FunctionFactory &);
|
||||
void registerFunctionSubstringUTF8(FunctionFactory &);
|
||||
void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &);
|
||||
void registerFunctionStartsWith(FunctionFactory &);
|
||||
void registerFunctionEndsWith(FunctionFactory &);
|
||||
|
||||
void registerFunctionsString(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionEmpty(factory);
|
||||
registerFunctionNotEmpty(factory);
|
||||
registerFunctionLength(factory);
|
||||
registerFunctionLengthUTF8(factory);
|
||||
registerFunctionLower(factory);
|
||||
registerFunctionUpper(factory);
|
||||
registerFunctionLowerUTF8(factory);
|
||||
registerFunctionUpperUTF8(factory);
|
||||
registerFunctionReverse(factory);
|
||||
registerFunctionReverseUTF8(factory);
|
||||
registerFunctionsConcat(factory);
|
||||
registerFunctionSubstring(factory);
|
||||
registerFunctionSubstringUTF8(factory);
|
||||
registerFunctionAppendTrailingCharIfAbsent(factory);
|
||||
registerFunctionStartsWith(factory);
|
||||
registerFunctionEndsWith(factory);
|
||||
}
|
||||
|
||||
}
|
||||
|
15
dbms/src/Functions/registerFunctionsTuple.cpp
Normal file
15
dbms/src/Functions/registerFunctionsTuple.cpp
Normal file
@ -0,0 +1,15 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionTuple(FunctionFactory &);
|
||||
void registerFunctionTupleElement(FunctionFactory &);
|
||||
|
||||
void registerFunctionsTuple(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionTuple(factory);
|
||||
registerFunctionTupleElement(factory);
|
||||
}
|
||||
|
||||
}
|
153
dbms/src/Functions/reverse.cpp
Normal file
153
dbms/src/Functions/reverse.cpp
Normal file
@ -0,0 +1,153 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
/** Reverse the string as a sequence of bytes.
|
||||
*/
|
||||
struct ReverseImpl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars_t & res_data,
|
||||
ColumnString::Offsets & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
size_t size = offsets.size();
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
for (size_t j = prev_offset; j < offsets[i] - 1; ++j)
|
||||
res_data[j] = data[offsets[i] + prev_offset - 2 - j];
|
||||
res_data[offsets[i] - 1] = 0;
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
size_t size = data.size() / n;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
for (size_t j = i * n; j < (i + 1) * n; ++j)
|
||||
res_data[j] = data[(i * 2 + 1) * n - j - 1];
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionReverse : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "reverse";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionReverse>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[0])
|
||||
&& !isArray(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arguments[0];
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||
{
|
||||
auto col_res = ColumnFixedString::create(col->getN());
|
||||
ReverseImpl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Also works with arrays.
|
||||
class FunctionBuilderReverse : public FunctionBuilderImpl
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "reverse";
|
||||
static FunctionBuilderPtr create(const Context & context) { return std::make_shared<FunctionBuilderReverse>(context); }
|
||||
|
||||
FunctionBuilderReverse(const Context & context) : context(context) {}
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 1; }
|
||||
|
||||
protected:
|
||||
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
|
||||
{
|
||||
if (isArray(arguments.at(0).type))
|
||||
return FunctionFactory::instance().get("arrayReverse", context)->build(arguments);
|
||||
else
|
||||
return std::make_shared<DefaultFunction>(
|
||||
FunctionReverse::create(context),
|
||||
ext::map<DataTypes>(arguments, [](const auto & elem) { return elem.type; }),
|
||||
return_type);
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
return arguments.at(0);
|
||||
}
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionReverse(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionBuilderReverse>();
|
||||
}
|
||||
|
||||
}
|
81
dbms/src/Functions/reverseUTF8.cpp
Normal file
81
dbms/src/Functions/reverseUTF8.cpp
Normal file
@ -0,0 +1,81 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
|
||||
/** Reverse the sequence of code points in a UTF-8 encoded string.
|
||||
* The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols.
|
||||
* If the string is not encoded in UTF-8, then the behavior is undefined.
|
||||
*/
|
||||
struct ReverseUTF8Impl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars_t & res_data,
|
||||
ColumnString::Offsets & res_offsets)
|
||||
{
|
||||
res_data.resize(data.size());
|
||||
res_offsets.assign(offsets);
|
||||
size_t size = offsets.size();
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
ColumnString::Offset j = prev_offset;
|
||||
while (j < offsets[i] - 1)
|
||||
{
|
||||
if (data[j] < 0xBF)
|
||||
{
|
||||
res_data[offsets[i] + prev_offset - 2 - j] = data[j];
|
||||
j += 1;
|
||||
}
|
||||
else if (data[j] < 0xE0)
|
||||
{
|
||||
memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 1], &data[j], 2);
|
||||
j += 2;
|
||||
}
|
||||
else if (data[j] < 0xF0)
|
||||
{
|
||||
memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 2], &data[j], 3);
|
||||
j += 3;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data[offsets[i] + prev_offset - 2 - j] = data[j];
|
||||
j += 1;
|
||||
}
|
||||
}
|
||||
|
||||
res_data[offsets[i] - 1] = 0;
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &)
|
||||
{
|
||||
throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
struct NameReverseUTF8
|
||||
{
|
||||
static constexpr auto name = "reverseUTF8";
|
||||
};
|
||||
using FunctionReverseUTF8 = FunctionStringToString<ReverseUTF8Impl, NameReverseUTF8, true>;
|
||||
|
||||
void registerFunctionReverseUTF8(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionReverseUTF8>();
|
||||
}
|
||||
|
||||
}
|
16
dbms/src/Functions/startsWith.cpp
Normal file
16
dbms/src/Functions/startsWith.cpp
Normal file
@ -0,0 +1,16 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStartsEndsWith.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using FunctionStartsWith = FunctionStartsEndsWith<NameStartsWith>;
|
||||
|
||||
void registerFunctionStartsWith(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionStartsWith>();
|
||||
}
|
||||
|
||||
}
|
172
dbms/src/Functions/substring.cpp
Normal file
172
dbms/src/Functions/substring.cpp
Normal file
@ -0,0 +1,172 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/GatherUtils/GatherUtils.h>
|
||||
#include <Functions/GatherUtils/Sources.h>
|
||||
#include <Functions/GatherUtils/Sinks.h>
|
||||
#include <Functions/GatherUtils/Slices.h>
|
||||
#include <Functions/GatherUtils/Algorithms.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using namespace GatherUtils;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int ZERO_ARRAY_OR_TUPLE_INDEX;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
class FunctionSubstring : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "substring";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionSubstring>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
size_t number_of_arguments = arguments.size();
|
||||
|
||||
if (number_of_arguments < 2 || number_of_arguments > 3)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(number_of_arguments) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isNumber(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName()
|
||||
+ " of second argument of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (number_of_arguments == 3 && !isNumber(arguments[2]))
|
||||
throw Exception("Illegal type " + arguments[2]->getName()
|
||||
+ " of second argument of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
template <typename Source>
|
||||
void executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length,
|
||||
const ColumnConst * column_start_const, const ColumnConst * column_length_const,
|
||||
Int64 start_value, Int64 length_value, Block & block, size_t result, Source && source,
|
||||
size_t input_rows_count)
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
|
||||
if (!column_length)
|
||||
{
|
||||
if (column_start_const)
|
||||
{
|
||||
if (start_value > 0)
|
||||
sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1);
|
||||
else if (start_value < 0)
|
||||
sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value);
|
||||
else
|
||||
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
}
|
||||
else
|
||||
sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (column_start_const && column_length_const)
|
||||
{
|
||||
if (start_value > 0)
|
||||
sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value);
|
||||
else if (start_value < 0)
|
||||
sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value);
|
||||
else
|
||||
throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
|
||||
}
|
||||
else
|
||||
sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
size_t number_of_arguments = arguments.size();
|
||||
|
||||
ColumnPtr column_string = block.getByPosition(arguments[0]).column;
|
||||
ColumnPtr column_start = block.getByPosition(arguments[1]).column;
|
||||
ColumnPtr column_length;
|
||||
|
||||
if (number_of_arguments == 3)
|
||||
column_length = block.getByPosition(arguments[2]).column;
|
||||
|
||||
const ColumnConst * column_start_const = checkAndGetColumn<ColumnConst>(column_start.get());
|
||||
const ColumnConst * column_length_const = nullptr;
|
||||
|
||||
if (number_of_arguments == 3)
|
||||
column_length_const = checkAndGetColumn<ColumnConst>(column_length.get());
|
||||
|
||||
Int64 start_value = 0;
|
||||
Int64 length_value = 0;
|
||||
|
||||
if (column_start_const)
|
||||
{
|
||||
start_value = column_start_const->getInt(0);
|
||||
}
|
||||
if (column_length_const)
|
||||
{
|
||||
length_value = column_length_const->getInt(0);
|
||||
if (length_value < 0)
|
||||
throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
|
||||
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
|
||||
length_value, block, result, StringSource(*col), input_rows_count);
|
||||
else if (const ColumnFixedString * col = checkAndGetColumn<ColumnFixedString>(column_string.get()))
|
||||
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
|
||||
length_value, block, result, FixedStringSource(*col), input_rows_count);
|
||||
else if (const ColumnConst * col = checkAndGetColumnConst<ColumnString>(column_string.get()))
|
||||
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
|
||||
length_value, block, result, ConstSource<StringSource>(*col), input_rows_count);
|
||||
else if (const ColumnConst * col = checkAndGetColumnConst<ColumnFixedString>(column_string.get()))
|
||||
executeForSource(column_start, column_length, column_start_const, column_length_const, start_value,
|
||||
length_value, block, result, ConstSource<FixedStringSource>(*col), input_rows_count);
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionSubstring(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionSubstring>();
|
||||
}
|
||||
|
||||
}
|
166
dbms/src/Functions/substringUTF8.cpp
Normal file
166
dbms/src/Functions/substringUTF8.cpp
Normal file
@ -0,0 +1,166 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Core/ColumnNumbers.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
|
||||
/** If the string is encoded in UTF-8, then it selects a substring of code points in it.
|
||||
* Otherwise, the behavior is undefined.
|
||||
*/
|
||||
struct SubstringUTF8Impl
|
||||
{
|
||||
static void vector(const ColumnString::Chars_t & data,
|
||||
const ColumnString::Offsets & offsets,
|
||||
size_t start,
|
||||
size_t length,
|
||||
ColumnString::Chars_t & res_data,
|
||||
ColumnString::Offsets & res_offsets)
|
||||
{
|
||||
res_data.reserve(data.size());
|
||||
size_t size = offsets.size();
|
||||
res_offsets.resize(size);
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
ColumnString::Offset res_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
ColumnString::Offset j = prev_offset;
|
||||
ColumnString::Offset pos = 1;
|
||||
ColumnString::Offset bytes_start = 0;
|
||||
ColumnString::Offset bytes_length = 0;
|
||||
while (j < offsets[i] - 1)
|
||||
{
|
||||
if (pos == start)
|
||||
bytes_start = j - prev_offset + 1;
|
||||
|
||||
if (data[j] < 0xBF)
|
||||
j += 1;
|
||||
else if (data[j] < 0xE0)
|
||||
j += 2;
|
||||
else if (data[j] < 0xF0)
|
||||
j += 3;
|
||||
else
|
||||
j += 1;
|
||||
|
||||
if (pos >= start && pos < start + length)
|
||||
bytes_length = j - prev_offset + 1 - bytes_start;
|
||||
else if (pos >= start + length)
|
||||
break;
|
||||
|
||||
++pos;
|
||||
}
|
||||
|
||||
if (bytes_start == 0)
|
||||
{
|
||||
res_data.resize(res_data.size() + 1);
|
||||
res_data[res_offset] = 0;
|
||||
++res_offset;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t bytes_to_copy = std::min(offsets[i] - prev_offset - bytes_start, bytes_length);
|
||||
res_data.resize(res_data.size() + bytes_to_copy + 1);
|
||||
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + bytes_start - 1], bytes_to_copy);
|
||||
res_offset += bytes_to_copy + 1;
|
||||
res_data[res_offset - 1] = 0;
|
||||
}
|
||||
res_offsets[i] = res_offset;
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionSubstringUTF8 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "substringUTF8";
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionSubstringUTF8>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 3;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!isNumber(arguments[1]) || !isNumber(arguments[2]))
|
||||
throw Exception("Illegal type " + (isNumber(arguments[1]) ? arguments[2]->getName() : arguments[1]->getName())
|
||||
+ " of argument of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const ColumnPtr column_string = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr column_start = block.getByPosition(arguments[1]).column;
|
||||
const ColumnPtr column_length = block.getByPosition(arguments[2]).column;
|
||||
|
||||
if (!column_start->isColumnConst() || !column_length->isColumnConst())
|
||||
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.");
|
||||
|
||||
Field start_field = (*block.getByPosition(arguments[1]).column)[0];
|
||||
Field length_field = (*block.getByPosition(arguments[2]).column)[0];
|
||||
|
||||
if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64)
|
||||
throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type.");
|
||||
|
||||
UInt64 start = start_field.get<UInt64>();
|
||||
UInt64 length = length_field.get<UInt64>();
|
||||
|
||||
if (start == 0)
|
||||
throw Exception("Second argument of function substring must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
/// Otherwise may lead to overflow and pass bounds check inside inner loop.
|
||||
if (start >= 0x8000000000000000ULL || length >= 0x8000000000000000ULL)
|
||||
throw Exception("Too large values of 2nd or 3rd argument provided for function substring.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
SubstringUTF8Impl::vector(col->getChars(), col->getOffsets(), start, length, col_res->getChars(), col_res->getOffsets());
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionSubstringUTF8(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionSubstringUTF8>();
|
||||
}
|
||||
|
||||
}
|
86
dbms/src/Functions/tuple.cpp
Normal file
86
dbms/src/Functions/tuple.cpp
Normal file
@ -0,0 +1,86 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
/** tuple(x, y, ...) is a function that allows you to group several columns
|
||||
* tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple.
|
||||
*/
|
||||
|
||||
class FunctionTuple : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "tuple";
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionTuple>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() < 1)
|
||||
throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return std::make_shared<DataTypeTuple>(arguments);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
size_t tuple_size = arguments.size();
|
||||
Columns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
tuple_columns[i] = block.getByPosition(arguments[i]).column;
|
||||
|
||||
/** If tuple is mixed of constant and not constant columns,
|
||||
* convert all to non-constant columns,
|
||||
* because many places in code expect all non-constant columns in non-constant tuple.
|
||||
*/
|
||||
if (ColumnPtr converted = tuple_columns[i]->convertToFullColumnIfConst())
|
||||
tuple_columns[i] = converted;
|
||||
}
|
||||
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionTuple(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionTuple>();
|
||||
}
|
||||
|
||||
}
|
@ -8,7 +8,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
@ -17,76 +16,11 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_INDEX;
|
||||
}
|
||||
|
||||
|
||||
/** tuple(x, y, ...) is a function that allows you to group several columns
|
||||
* tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple.
|
||||
*/
|
||||
|
||||
class FunctionTuple : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "tuple";
|
||||
|
||||
static FunctionPtr create(const Context &)
|
||||
{
|
||||
return std::make_shared<FunctionTuple>();
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
bool isInjective(const Block &) override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() < 1)
|
||||
throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return std::make_shared<DataTypeTuple>(arguments);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
size_t tuple_size = arguments.size();
|
||||
Columns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
{
|
||||
tuple_columns[i] = block.getByPosition(arguments[i]).column;
|
||||
|
||||
/** If tuple is mixed of constant and not constant columns,
|
||||
* convert all to non-constant columns,
|
||||
* because many places in code expect all non-constant columns in non-constant tuple.
|
||||
*/
|
||||
if (ColumnPtr converted = tuple_columns[i]->convertToFullColumnIfConst())
|
||||
tuple_columns[i] = converted;
|
||||
}
|
||||
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Extract element of tuple by constant index or name. The operation is essentially free.
|
||||
* Also the function looks through Arrays: you can get Array of tuple elements from Array of Tuples.
|
||||
*/
|
||||
@ -200,9 +134,8 @@ private:
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionsTuple(FunctionFactory & factory)
|
||||
void registerFunctionTupleElement(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionTuple>();
|
||||
factory.registerFunction<FunctionTupleElement>();
|
||||
}
|
||||
|
21
dbms/src/Functions/upper.cpp
Normal file
21
dbms/src/Functions/upper.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include <Functions/LowerUpperImpl.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameUpper
|
||||
{
|
||||
static constexpr auto name = "upper";
|
||||
};
|
||||
using FunctionUpper = FunctionStringToString<LowerUpperImpl<'a', 'z'>, NameUpper>;
|
||||
|
||||
void registerFunctionUpper(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionUpper>();
|
||||
}
|
||||
|
||||
}
|
23
dbms/src/Functions/upperUTF8.cpp
Normal file
23
dbms/src/Functions/upperUTF8.cpp
Normal file
@ -0,0 +1,23 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include <Functions/LowerUpperUTF8Impl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Poco/Unicode.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameUpperUTF8
|
||||
{
|
||||
static constexpr auto name = "upperUTF8";
|
||||
};
|
||||
|
||||
using FunctionUpperUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>;
|
||||
|
||||
void registerFunctionUpperUTF8(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionUpperUTF8>();
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user