Merge pull request #70 from yandex/METR-19266

Metr 19266
This commit is contained in:
alexey-milovidov 2016-12-10 10:38:17 +04:00 committed by GitHub
commit f60400f184
136 changed files with 10819 additions and 3014 deletions

View File

@ -46,6 +46,7 @@ add_library (dbms
src/Server/ReplicasStatusHandler.h
src/Client/InterruptListener.h
include/DB/Functions/FunctionsLogical.h
include/DB/Functions/FunctionsNull.h
include/DB/Functions/FunctionsReinterpret.h
include/DB/Functions/FunctionFactory.h
include/DB/Functions/FunctionsArray.h
@ -58,6 +59,7 @@ add_library (dbms
include/DB/Functions/FunctionsTransform.h
include/DB/Functions/Conditional/CondException.h
include/DB/Functions/Conditional/common.h
include/DB/Functions/Conditional/NullMapBuilder.h
include/DB/Functions/Conditional/getArrayType.h
include/DB/Functions/Conditional/ArgsInfo.h
include/DB/Functions/Conditional/CondSource.h
@ -149,6 +151,7 @@ add_library (dbms
include/DB/Parsers/ParserCase.h
include/DB/Parsers/ASTTablesInSelectQuery.h
include/DB/Parsers/ParserTablesInSelectQuery.h
include/DB/AggregateFunctions/AggregateFunctionNull.h
include/DB/AggregateFunctions/AggregateFunctionMerge.h
include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h
include/DB/AggregateFunctions/AggregateFunctionIf.h
@ -286,6 +289,8 @@ add_library (dbms
include/DB/DataStreams/SquashingTransform.h
include/DB/DataStreams/SquashingBlockInputStream.h
include/DB/DataStreams/SquashingBlockOutputStream.h
include/DB/DataStreams/NullableAdapterBlockInputStream.h
include/DB/DataTypes/NullSymbol.h
include/DB/DataStreams/ColumnGathererStream.h
include/DB/DataTypes/IDataType.h
include/DB/DataTypes/IDataTypeDummy.h
@ -305,6 +310,8 @@ add_library (dbms
include/DB/DataTypes/DataTypeExpression.h
include/DB/DataTypes/DataTypeDateTime.h
include/DB/DataTypes/DataTypesNumberFixed.h
include/DB/DataTypes/DataTypeNull.h
include/DB/DataTypes/DataTypeNullable.h
include/DB/Dictionaries/FileDictionarySource.h
include/DB/Dictionaries/DictionarySourceFactory.h
include/DB/Dictionaries/DictionaryStructure.h
@ -511,6 +518,7 @@ add_library (dbms
include/DB/Columns/ColumnSet.h
include/DB/Columns/ColumnArray.h
include/DB/Columns/ColumnsNumber.h
include/DB/Columns/ColumnNullable.h
include/DB/Client/MultiplexedConnections.h
include/DB/Client/ConnectionPoolWithFailover.h
include/DB/Client/Connection.h
@ -666,6 +674,7 @@ add_library (dbms
src/Columns/ColumnArray.cpp
src/Columns/ColumnAggregateFunction.cpp
src/Columns/ColumnExpression.cpp
src/Columns/ColumnNullable.cpp
src/Columns/ColumnsCommon.cpp
src/Columns/ColumnString.cpp
@ -712,8 +721,10 @@ add_library (dbms
src/Storages/MergeTree/MergeTreeData.cpp
src/Storages/MergeTree/MergeTreeDataPart.cpp
src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp
src/Storages/MergeTree/MergeTreeReader.cpp
src/Storages/MergeTree/MergeTreeDataWriter.cpp
src/Storages/MergeTree/MergeTreeDataMerger.cpp
src/Storages/MergeTree/MergedBlockOutputStream.cpp
src/Storages/MergeTree/MergeTreePartChecker.cpp
src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp
src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp
@ -798,6 +809,8 @@ add_library (dbms
src/DataStreams/SquashingTransform.cpp
src/DataStreams/SquashingBlockInputStream.cpp
src/DataStreams/SquashingBlockOutputStream.cpp
src/DataStreams/MaterializingBlockInputStream.cpp
src/DataStreams/NullableAdapterBlockInputStream.cpp
src/DataStreams/ColumnGathererStream.cpp
src/DataTypes/DataTypeString.cpp
@ -808,6 +821,9 @@ add_library (dbms
src/DataTypes/DataTypeAggregateFunction.cpp
src/DataTypes/DataTypeEnum.cpp
src/DataTypes/DataTypeTuple.cpp
src/DataTypes/DataTypeNull.cpp
src/DataTypes/DataTypeNullable.cpp
src/DataTypes/NullSymbol.cpp
src/DataTypes/FieldToDataType.cpp
src/Dictionaries/DictionarySourceFactory.cpp
@ -909,6 +925,8 @@ add_library (dbms
src/Interpreters/ClusterProxy/Query.cpp
src/Interpreters/AsynchronousMetrics.cpp
src/Functions/DataTypeTraits.cpp
src/Functions/IFunction.cpp
src/Functions/FunctionFactory.cpp
src/Functions/FunctionsArithmetic.cpp
src/Functions/FunctionsArray.cpp
@ -922,6 +940,7 @@ add_library (dbms
src/Functions/FunctionsHashing.cpp
src/Functions/FunctionsHigherOrder.cpp
src/Functions/FunctionsLogical.cpp
src/Functions/FunctionsNull.cpp
src/Functions/FunctionsRandom.cpp
src/Functions/FunctionsReinterpret.cpp
src/Functions/FunctionsRound.cpp
@ -938,11 +957,13 @@ add_library (dbms
src/Functions/Conditional/getArrayType.cpp
src/Functions/Conditional/ArgsInfo.cpp
src/Functions/Conditional/CondSource.cpp
src/Functions/Conditional/NullMapBuilder.cpp
src/Functions/Conditional/NumericPerformer.cpp
src/Functions/Conditional/StringEvaluator.cpp
src/Functions/Conditional/StringArrayEvaluator.cpp
src/AggregateFunctions/UniqCombinedBiasData.cpp
src/AggregateFunctions/AggregateFunctionNull.cpp
src/AggregateFunctions/AggregateFunctionFactory.cpp
src/AggregateFunctions/AggregateFunctionArray.cpp
src/AggregateFunctions/AggregateFunctionAvg.cpp

View File

@ -7,14 +7,14 @@
namespace DB
{
/** Creates aggregate function by name.
/** Creates an aggregate function by name.
*/
class AggregateFunctionFactory final
{
friend class StorageSystemFunctions;
private:
/// Not std::function, for lower object size and less indirection.
/// No std::function, for smaller object size and less indirection.
using Creator = AggregateFunctionPtr(*)(const String & name, const DataTypes & argument_types);
using AggregateFunctions = std::unordered_map<String, Creator>;
@ -31,12 +31,15 @@ public:
CaseInsensitive
};
/// Register aggregate function with its name.
/// Register an aggregate function by its name.
void registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
AggregateFunctionFactory(const AggregateFunctionFactory &) = delete;
AggregateFunctionFactory & operator=(const AggregateFunctionFactory &) = delete;
private:
AggregateFunctionPtr getImpl(const String & name, const DataTypes & argument_types, int recursion_level) const;
private:
AggregateFunctions aggregate_functions;

View File

@ -0,0 +1,160 @@
#pragma once
#include <DB/AggregateFunctions/IAggregateFunction.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/DataTypes/DataTypeNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// This class implements a wrapper around an aggregate function. Despite its name,
/// this is an adapter. It is used to handle aggregate functions that are called with
/// at least one nullable argument. It implements the logic according to which any
/// row that contains at least one NULL is skipped.
class AggregateFunctionNull : public IAggregateFunction
{
public:
AggregateFunctionNull(AggregateFunctionPtr nested_function_)
: nested_function{nested_function_}
{
}
String getName() const override
{
return nested_function->getName();
}
void setArguments(const DataTypes & arguments) override
{
argument_count = arguments.size();
is_nullable.reserve(arguments.size());
for (const auto & arg : arguments)
{
bool res = arg->isNullable();
is_nullable.push_back(res);
}
DataTypes new_args;
new_args.reserve(arguments.size());
for (const auto & arg : arguments)
{
if (arg->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*arg);
const DataTypePtr & nested_type = nullable_type.getNestedType();
new_args.push_back(nested_type);
}
else
new_args.push_back(arg);
}
nested_function->setArguments(new_args);
}
void setParameters(const Array & params) override
{
nested_function->setParameters(params);
}
DataTypePtr getReturnType() const override
{
return nested_function->getReturnType();
}
void create(AggregateDataPtr place) const override
{
nested_function->create(place);
}
void destroy(AggregateDataPtr place) const noexcept override
{
nested_function->destroy(place);
}
bool hasTrivialDestructor() const override
{
return nested_function->hasTrivialDestructor();
}
size_t sizeOfData() const override
{
return nested_function->sizeOfData();
}
size_t alignOfData() const override
{
return nested_function->alignOfData();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
/// This container stores the columns we really pass to the nested function.
const IColumn * passed_columns[argument_count];
for (size_t i = 0; i < argument_count; ++i)
{
if (is_nullable[i])
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*columns[i]);
if (nullable_col.isNullAt(row_num))
{
/// If at least one column has a null value in the current row,
/// we don't process this row.
return;
}
passed_columns[i] = nullable_col.getNestedColumn().get();
}
else
passed_columns[i] = columns[i];
}
nested_function->add(place, passed_columns, row_num, arena);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
nested_function->merge(place, rhs, arena);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
nested_function->serialize(place, buf);
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
{
nested_function->deserialize(place, buf, arena);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
nested_function->insertResultInto(place, to);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place,
const IColumn ** columns, size_t row_num, Arena * arena)
{
return static_cast<const AggregateFunctionNull &>(*that).add(place, columns, row_num, arena);
}
AddFunc getAddressOfAddFunction() const override
{
return &addFree;
}
private:
AggregateFunctionPtr nested_function;
std::vector<bool> is_nullable;
size_t argument_count = 0;
};
}

View File

@ -313,11 +313,6 @@ public:
throw Exception("Method replicate is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
}
void getExtremes(Field & min, Field & max) const override
{
throw Exception("Method getExtremes is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
}
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override
{
return 0;
@ -341,6 +336,11 @@ public:
{
return data;
}
void getExtremes(Field & min, Field & max) const override
{
throw Exception("Method getExtremes is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED);
}
};

View File

@ -48,9 +48,32 @@ public:
std::string getName() const override { return "ColumnArray(" + getData().getName() + ")"; }
ColumnPtr cloneEmpty() const override
ColumnPtr cloneResized(size_t size) const override
{
return std::make_shared<ColumnArray>(getData().cloneEmpty());
ColumnPtr new_col_holder = std::make_shared<ColumnArray>(getData().cloneEmpty());
if (size > 0)
{
auto & new_col = static_cast<ColumnArray &>(*new_col_holder);
size_t count = std::min(this->size(), size);
/// First create the offsets.
const auto & from_offsets = getOffsets();
auto & new_offsets = new_col.getOffsets();
new_offsets.resize(size);
new_offsets.assign(from_offsets.begin(), from_offsets.begin() + count);
if (size > count)
{
for (size_t i = count; i < size; ++i)
new_offsets[i] = new_offsets[i - 1];
}
/// Then store the data.
new_col.getData().insertRangeFrom(getData(), 0, count);
}
return new_col_holder;
}
size_t size() const override
@ -247,12 +270,6 @@ public:
return getData().byteSize() + getOffsets().size() * sizeof(getOffsets()[0]);
}
void getExtremes(Field & min, Field & max) const override
{
min = Array();
max = Array();
}
bool hasEqualOffsets(const ColumnArray & other) const
{
if (offsets == other.offsets)
@ -305,6 +322,12 @@ public:
return std::make_shared<ColumnArray>(new_data, new_offsets);
}
void getExtremes(Field & min, Field & max) const override
{
min = Array();
max = Array();
}
private:
ColumnPtr data;
ColumnPtr offsets; /// Смещения могут быть разделяемыми для нескольких столбцов - для реализации вложенных структур данных.

View File

@ -213,6 +213,7 @@ public:
ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr())
: ColumnConstBase<T, T, ColumnConst<T>>(s_, data_, data_type_) {}
bool isNull() const override { return false; };
StringRef getDataAt(size_t n) const override;
StringRef getDataAtWithTerminatingZero(size_t n) const override;
UInt64 get64(size_t n) const override;
@ -301,10 +302,22 @@ public:
};
using ColumnNull = ColumnConst<Null>;
using ColumnConstString = ColumnConst<String>;
using ColumnConstArray = ColumnConst<Array>;
using ColumnConstTuple = ColumnConst<Tuple>;
template <>
inline bool ColumnConst<Null>::isNull() const
{
return true;
}
template <>
inline StringRef ColumnConst<Null>::getDataAt(size_t n) const
{
return {};
}
template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const
{
@ -313,6 +326,7 @@ template <typename T> ColumnPtr ColumnConst<T>::convertToFullColumn() const
return res;
}
template <> ColumnPtr ColumnConst<Null>::convertToFullColumn() const;
template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const;

View File

@ -45,9 +45,23 @@ public:
std::string getName() const override { return "ColumnFixedString"; }
ColumnPtr cloneEmpty() const override
ColumnPtr cloneResized(size_t size) const override
{
return std::make_shared<ColumnFixedString>(n);
ColumnPtr new_col_holder = std::make_shared<ColumnFixedString>(n);
if (size > 0)
{
auto & new_col = static_cast<ColumnFixedString &>(*new_col_holder);
new_col.chars.resize(size * n);
size_t count = std::min(this->size(), size);
memcpy(&(new_col.chars[0]), &chars[0], count * n * sizeof(chars[0]));
if (size > count)
memset(&(new_col.chars[count * n]), '\0', (size - count) * n);
}
return new_col_holder;
}
size_t size() const override
@ -290,12 +304,6 @@ public:
return res;
}
void getExtremes(Field & min, Field & max) const override
{
min = String();
max = String();
}
void reserve(size_t size) override
{
chars.reserve(n * size);
@ -306,6 +314,12 @@ public:
const Chars_t & getChars() const { return chars; }
size_t getN() const { return n; }
void getExtremes(Field & min, Field & max) const override
{
min = String();
max = String();
}
};

View File

@ -0,0 +1,80 @@
#pragma once
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsNumber.h>
namespace DB
{
using NullValuesByteMap = PaddedPODArray<UInt8>;
/// Class that specifies nullable columns. A nullable column represents
/// a column, which may have any type, provided with the possibility of
/// storing NULL values. For this purpose, a ColumNullable object stores
/// an ordinary column along with a special column, namely a byte map,
/// whose type is ColumnUInt8. The latter column indicates whether the
/// value of a given row is a NULL or not. Such a design is preferred
/// over a bitmap because columns are usually stored on disk as compressed
/// files. In this regard, using a bitmap instead of a byte map would
/// greatly complicate the implementation with little to no benefits.
class ColumnNullable final : public IColumn
{
public:
ColumnNullable(ColumnPtr nested_column_, ColumnPtr null_map_);
std::string getName() const override { return "ColumnNullable(" + nested_column->getName() + ")"; }
bool isNumeric() const override { return nested_column->isNumeric(); }
bool isNumericNotNullable() const override { return false; }
bool isConst() const override { return nested_column->isConst(); }
bool isFixed() const override { return nested_column->isFixed(); }
bool isNullable() const override { return true; }
ColumnPtr cloneResized(size_t size) const override;
size_t size() const override { return nested_column->size(); }
bool isNullAt(size_t n) const { return static_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
Field operator[](size_t n) const override;
void get(size_t n, Field & res) const override;
UInt64 get64(size_t n) const override { return nested_column->get64(n); }
StringRef getDataAt(size_t n) const override;
void insertData(const char * pos, size_t length) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;;
void insert(const Field & x) override;
void insertDefault() override;
void popBack(size_t n) override;
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override;
void getPermutation(bool reverse, size_t limit, Permutation & res) const override;
void reserve(size_t n) override;
size_t byteSize() const override;
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
ColumnPtr convertToFullColumnIfConst() const override;
void updateHashWithValue(size_t n, SipHash & hash) const override;
void getExtremes(Field & min, Field & max) const override;
/// Return the column that represents values.
ColumnPtr & getNestedColumn() { return nested_column; }
const ColumnPtr & getNestedColumn() const { return nested_column; }
/// Return the column that represents the byte map.
ColumnPtr & getNullValuesByteMap() { return null_map; }
const ColumnPtr & getNullValuesByteMap() const { return null_map; }
/// Apply the null byte map of a specified nullable column onto the
/// null byte map of the current column by performing an element-wise OR
/// between both byte maps. This method is used to determine the null byte
/// map of the result column of a function taking one or more nullable
/// columns.
void applyNullValuesByteMap(const ColumnNullable & other);
private:
/// Convenience methods which make the implementation easier to read.
ColumnUInt8 & getNullMapContent() { return static_cast<ColumnUInt8 &>(*null_map); }
const ColumnUInt8 & getNullMapContent() const { return static_cast<const ColumnUInt8 &>(*null_map); }
private:
ColumnPtr nested_column;
ColumnPtr null_map;
};
}

View File

@ -63,9 +63,42 @@ public:
return chars.size() + offsets.size() * sizeof(offsets[0]);
}
ColumnPtr cloneEmpty() const override
ColumnPtr cloneResized(size_t size) const override
{
return std::make_shared<ColumnString>();
ColumnPtr new_col_holder = std::make_shared<ColumnString>();
if (size > 0)
{
auto & new_col = static_cast<ColumnString &>(*new_col_holder);
size_t count = std::min(this->size(), size);
/// First create the offsets.
new_col.offsets.resize(size);
new_col.offsets.assign(offsets.begin(), offsets.begin() + count);
size_t byte_count = new_col.offsets.back();
if (size > count)
{
/// Create offsets for the (size - count) new empty strings.
for (size_t i = count; i < size; ++i)
new_col.offsets[i] = new_col.offsets[i - 1] + 1;
}
/// Then store the strings.
new_col.chars.resize(new_col.offsets.back());
new_col.chars.assign(chars.begin(), chars.begin() + byte_count);
if (size > count)
{
/// Create (size - count) empty strings.
size_t from = new_col.offsets[count];
size_t n = new_col.offsets.back() - from;
memset(&new_col.chars[from], '\0', n);
}
}
return new_col_holder;
}
Field operator[](size_t n) const override
@ -413,18 +446,17 @@ public:
chars.reserve(n * DBMS_APPROX_STRING_SIZE);
}
void getExtremes(Field & min, Field & max) const override
{
min = String();
max = String();
}
Chars_t & getChars() { return chars; }
const Chars_t & getChars() const { return chars; }
Offsets_t & getOffsets() { return offsets; }
const Offsets_t & getOffsets() const { return offsets; }
void getExtremes(Field & min, Field & max) const override
{
min = String();
max = String();
}
};

View File

@ -247,20 +247,6 @@ public:
return res;
}
void getExtremes(Field & min, Field & max) const override
{
const size_t tuple_size = columns.size();
min = Tuple(TupleBackend(tuple_size));
max = Tuple(TupleBackend(tuple_size));
auto & min_backend = min.get<Tuple &>().t;
auto & max_backend = max.get<Tuple &>().t;
for (const auto i : ext::range(0, tuple_size))
columns[i]->getExtremes(min_backend[i], max_backend[i]);
}
ColumnPtr convertToFullColumnIfConst() const override
{
Block materialized = data;
@ -277,6 +263,20 @@ public:
const Columns & getColumns() const { return columns; }
Columns & getColumns() { return columns; }
void getExtremes(Field & min, Field & max) const override
{
const size_t tuple_size = columns.size();
min = Tuple(TupleBackend(tuple_size));
max = Tuple(TupleBackend(tuple_size));
auto & min_backend = min.get<Tuple &>().t;
auto & max_backend = max.get<Tuple &>().t;
for (const auto i : ext::range(0, tuple_size))
columns[i]->getExtremes(min_backend[i], max_backend[i]);
}
};

View File

@ -250,9 +250,23 @@ public:
std::string getName() const override { return "ColumnVector<" + TypeName<T>::get() + ">"; }
ColumnPtr cloneEmpty() const override
ColumnPtr cloneResized(size_t size) const override
{
return std::make_shared<ColumnVector<T>>();
ColumnPtr new_col_holder = std::make_shared<Self>();
if (size > 0)
{
auto & new_col = static_cast<Self &>(*new_col_holder);
new_col.data.resize(size);
size_t count = std::min(this->size(), size);
memcpy(&new_col.data[0], &data[0], count * sizeof(data[0]));
if (size > count)
memset(&new_col.data[count], value_type(), size - count);
}
return new_col_holder;
}
Field operator[](size_t n) const override
@ -437,6 +451,7 @@ public:
max = typename NearestFieldType<T>::Type(cur_max);
}
/** Более эффективные методы манипуляции */
Container_t & getData()
{

View File

@ -45,10 +45,19 @@ public:
*/
virtual bool isNumeric() const { return false; }
/// Is this column numeric and not nullable?
virtual bool isNumericNotNullable() const { return isNumeric(); }
/** Столбец представляет собой константу.
*/
virtual bool isConst() const { return false; }
/// Is this column a container for nullable values?
virtual bool isNullable() const { return false; }
/// Is this a null column?
virtual bool isNull() const { return false; }
/** Если столбец не константа - возвращает nullptr (либо может вернуть самого себя).
* Если столбец константа, то превращает его в полноценный столбец (если тип столбца предполагает такую возможность) и возвращает его.
* Отдельный случай:
@ -244,6 +253,7 @@ public:
*/
virtual void getExtremes(Field & min, Field & max) const = 0;
/** Если возможно - зарезервировать место для указанного количества элементов. Если невозможно или не поддерживается - ничего не делать.
* Функция влияет только на производительность.
*/

View File

@ -53,11 +53,6 @@ public:
throw Exception("Method updateHashWithValue is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void getExtremes(Field & min, Field & max) const override
{
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
s += length;
@ -91,6 +86,11 @@ public:
return cloneDummy(s == 0 ? 0 : offsets.back());
}
void getExtremes(Field & min, Field & max) const override
{
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
private:
size_t s;
};

View File

@ -1,8 +1,14 @@
#pragma once
#include <DB/Common/Exception.h>
#include <string>
#include <cstring>
namespace DB { namespace ErrorCodes {
extern const int LOGICAL_ERROR;
}}
namespace detail
{
@ -22,7 +28,8 @@ inline bool endsWith(const std::string & s, const std::string & suffix)
}
/// strlen evaluated compile-time.
/// With GCC, strlen is evaluated compile time if we pass it a constant
/// string that is known at compile time.
inline bool startsWith(const std::string & s, const char * prefix)
{
return detail::startsWith(s, prefix, strlen(prefix));
@ -33,6 +40,36 @@ inline bool endsWith(const std::string & s, const char * suffix)
return detail::endsWith(s, suffix, strlen(suffix));
}
/// Given an integer, return the adequate suffix for
/// printing an ordinal number.
template <typename T>
std::string getOrdinalSuffix(T n)
{
static_assert(std::is_integral<T>::value && std::is_unsigned<T>::value,
"Unsigned integer value required");
const auto val = n % 10;
bool is_th;
if ((val >= 1) && (val <= 3))
is_th = (n > 10) && (((n / 10) % 10) == 1);
else
is_th = true;
if (is_th)
return "th";
else
{
switch (val)
{
case 1: return "st";
case 2: return "nd";
case 3: return "rd";
default: throw DB::Exception{"getOrdinalSuffix: internal error",
DB::ErrorCodes::LOGICAL_ERROR};
};
}
}
/// More efficient than libc, because doesn't respect locale.

View File

@ -4,13 +4,16 @@
#include <map>
#include <initializer_list>
#include <DB/Common/Exception.h>
#include <DB/Core/BlockInfo.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Core/ColumnWithTypeAndName.h>
#include <DB/Core/ColumnsWithTypeAndName.h>
#include <DB/Core/ColumnNumbers.h>
#include <DB/Common/Exception.h>
namespace DB
{

View File

@ -409,6 +409,7 @@ template <> struct NearestFieldType<String> { using Type = String ; };
template <> struct NearestFieldType<Array> { using Type = Array ; };
template <> struct NearestFieldType<Tuple> { using Type = Tuple ; };
template <> struct NearestFieldType<bool> { using Type = UInt64 ; };
template <> struct NearestFieldType<Null> { using Type = Null; };
template <typename T>

View File

@ -30,24 +30,38 @@ using Float64 = double;
using String = std::string;
using Strings = std::vector<String>;
/// Ordinary types with nullability.
template <typename T> struct Nullable { using Type = T; };
template <typename T> struct IsNumber { static const bool value = false; };
/// Get a non-nullable type.
template <typename T> struct RemoveNullable { using Type = T; };
template <typename T> struct RemoveNullable<Nullable<T>> { using Type = T; };
template <> struct IsNumber<UInt8> { static const bool value = true; };
template <> struct IsNumber<UInt16> { static const bool value = true; };
template <> struct IsNumber<UInt32> { static const bool value = true; };
template <> struct IsNumber<UInt64> { static const bool value = true; };
template <> struct IsNumber<Int8> { static const bool value = true; };
template <> struct IsNumber<Int16> { static const bool value = true; };
template <> struct IsNumber<Int32> { static const bool value = true; };
template <> struct IsNumber<Int64> { static const bool value = true; };
template <> struct IsNumber<Float32> { static const bool value = true; };
template <> struct IsNumber<Float64> { static const bool value = true; };
/// Check if a type is nullable.
template <typename T> struct IsNullable { static constexpr bool value = false; };
template <typename T> struct IsNullable<Nullable<T>> { static constexpr bool value = true; };
template <typename T> struct IsNumber { static constexpr bool value = false; };
template <typename T> struct IsNumber<Nullable<T> > { static constexpr bool value = IsNumber<T>::value; };
template <> struct IsNumber<UInt8> { static constexpr bool value = true; };
template <> struct IsNumber<UInt16> { static constexpr bool value = true; };
template <> struct IsNumber<UInt32> { static constexpr bool value = true; };
template <> struct IsNumber<UInt64> { static constexpr bool value = true; };
template <> struct IsNumber<Int8> { static constexpr bool value = true; };
template <> struct IsNumber<Int16> { static constexpr bool value = true; };
template <> struct IsNumber<Int32> { static constexpr bool value = true; };
template <> struct IsNumber<Int64> { static constexpr bool value = true; };
template <> struct IsNumber<Float32> { static constexpr bool value = true; };
template <> struct IsNumber<Float64> { static constexpr bool value = true; };
template <typename T> struct TypeName;
template <typename T> struct TypeName<Nullable<T>> { static std::string get() { return "Nullable(" + TypeName<T>::get() + ")"; } };
template <> struct TypeName<Null> { static std::string get() { return "Null"; } };
template <> struct TypeName<Nullable<void>> : TypeName<Null> {};
template <> struct TypeName<Null> { static std::string get() { return "Null"; } };
template <> struct TypeName<UInt8> { static std::string get() { return "UInt8"; } };
template <> struct TypeName<UInt16> { static std::string get() { return "UInt16"; } };
template <> struct TypeName<UInt32> { static std::string get() { return "UInt32"; } };

View File

@ -1,10 +1,7 @@
#pragma once
#include <DB/Columns/ColumnConst.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
@ -13,39 +10,12 @@ namespace DB
class MaterializingBlockInputStream : public IProfilingBlockInputStream
{
public:
MaterializingBlockInputStream(BlockInputStreamPtr input_)
{
children.push_back(input_);
}
String getName() const override { return "Materializing"; }
String getID() const override
{
std::stringstream res;
res << "Materializing(" << children.back()->getID() << ")";
return res.str();
}
MaterializingBlockInputStream(BlockInputStreamPtr input_);
String getName() const override;
String getID() const override;
protected:
Block readImpl() override
{
Block res = children.back()->read();
if (!res)
return res;
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
{
auto & src = res.getByPosition(i).column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
src = converted;
}
return res;
}
Block readImpl() override;
};
}

View File

@ -2,9 +2,10 @@
#include <DB/Columns/ColumnConst.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <ext/range.hpp>
namespace DB
{
@ -18,7 +19,7 @@ public:
void write(const Block & block) override
{
output->write(materialize(block));
output->write(materialize(block));
}
void flush() override { output->flush(); }
@ -40,10 +41,20 @@ private:
for (const auto i : ext::range(0, block.columns()))
{
auto & src = block.getByPosition(i).column;
auto & element = block.getByPosition(i);
auto & src = element.column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
{
src = converted;
auto & type = element.type;
if (type->isNull())
{
/// A ColumnNull that is converted to a full column
/// has the type DataTypeNullable(DataTypeUInt8).
type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>());
}
}
}
return block;

View File

@ -0,0 +1,61 @@
#pragma once
#include <DB/DataStreams/IProfilingBlockInputStream.h>
namespace DB
{
/// This stream allows perfoming INSERT requests in which the types of
/// the target and source blocks are compatible up to nullability:
///
/// - if a target column is nullable while the corresponding source
/// column is not, we embed the source column into a nullable column;
/// - if a source column is nullable while the corresponding target
/// column is not, we extract the nested column from the source;
/// - otherwise we just perform an identity mapping.
class NullableAdapterBlockInputStream : public IProfilingBlockInputStream
{
public:
NullableAdapterBlockInputStream(BlockInputStreamPtr input_, const Block & in_sample_,
const Block & out_sample_, const NamesAndTypesListPtr & required_columns_);
String getName() const override { return "NullableAdapterBlockInputStream"; }
String getID() const override;
protected:
Block readImpl() override;
private:
/// Given a column of a block we have just read,
/// how must we process it?
enum Action
{
/// Do nothing.
NONE = 0,
/// Convert nullable column to ordinary column.
TO_ORDINARY,
/// Convert non-nullable column to nullable column.
TO_NULLABLE
};
/// Actions to be taken for each column of a block.
using Actions = std::vector<Action>;
private:
/// Return true if we must transform the blocks we read.
bool mustTransform() const;
/// Determine the actions to be taken using the source sample block,
/// which describes the columns from which we fetch data inside an INSERT
/// query, and the target sample block which contains the columns
/// we insert data into.
Actions getActions(const Block & in_sample, const Block & out_sample) const;
private:
NamesAndTypesListPtr required_columns;
const Actions actions;
const bool must_transform;
};
}

View File

@ -86,6 +86,9 @@ public:
const DataTypePtr & getNestedType() const { return nested; }
const DataTypeTraits::EnrichedDataTypePtr & getEnrichedNestedType() const { return enriched_nested; }
const DataTypePtr & getOffsetsType() const { return offsets; }
/// Returns the data type found at the most nested level.
const DataTypePtr & getMostNestedType() const;
};
}

View File

@ -0,0 +1,137 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
#include <DB/DataTypes/NullSymbol.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
/// Data type which represents a single NULL value. It is the type
/// associated to a constant column that contains only NULL values,
/// namely ColumnNull, which arises when a NULL is specified as a
/// column in any query.
class DataTypeNull final : public IDataType
{
public:
using FieldType = Null;
public:
std::string getName() const override
{
return "Null";
}
bool isNull() const override
{
return true;
}
DataTypePtr clone() const override
{
return std::make_shared<DataTypeNull>();
}
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
ColumnPtr createColumn() const override
{
return std::make_shared<ColumnNull>(0, Null());
}
ColumnPtr createConstColumn(size_t size, const Field & field) const override
{
return std::make_shared<ColumnNull>(size, Null());
}
Field getDefault() const override
{
return Field{};
}
size_t getSizeOfField() const override
{
/// NULL has the size of the smallest non-null type.
return sizeof(UInt8);
}
void serializeBinary(const Field & field, WriteBuffer & ostr) const override
{
UInt8 x = 0;
writeBinary(x, ostr);
}
void deserializeBinary(Field & field, ReadBuffer & istr) const override
{
UInt8 x;
readBinary(x, istr);
field = Field{};
}
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
{
UInt8 x = 0;
writeBinary(x, ostr);
}
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override
{
UInt8 x;
readBinary(x, istr);
column.insertDefault();
}
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
{
writeCString(NullSymbol::Escaped::name, ostr);
}
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const override
{
assertString(NullSymbol::Escaped::name, istr);
}
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
{
writeCString(NullSymbol::Quoted::name, ostr);
}
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const override
{
assertString(NullSymbol::Quoted::name, istr);
}
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
{
writeCString(NullSymbol::CSV::name, ostr);
}
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override
{
assertString(NullSymbol::CSV::name, istr);
}
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override
{
writeCString(NullSymbol::Plain::name, ostr);
}
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr,
bool force_quoting_64bit_integers) const override
{
writeCString(NullSymbol::JSON::name, ostr);
}
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override
{
assertString(NullSymbol::JSON::name, istr);
}
};
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
namespace DB
{
/// A nullable data type is an ordinary data type provided with a tag
/// indicating that it also contains the NULL value. The following class
/// embodies this concept.
class DataTypeNullable final : public IDataType
{
public:
DataTypeNullable(DataTypePtr nested_data_type_);
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
bool isNullable() const override { return true; }
bool isNumeric() const override { return nested_data_type->isNumeric(); }
bool isNumericNotNullable() const override { return false; }
bool behavesAsNumber() const override { return nested_data_type->behavesAsNumber(); }
DataTypePtr clone() const override { return std::make_shared<DataTypeNullable>(nested_data_type->clone()); }
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); }
void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); }
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const override;
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const override;
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override;
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr,
bool force_quoting_64bit_integers) const override;
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override;
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
ColumnPtr createColumn() const override;
ColumnPtr createConstColumn(size_t size, const Field & field) const override;
Field getDefault() const override { return nested_data_type->getDefault(); }
size_t getSizeOfField() const override { return nested_data_type->getSizeOfField(); }
DataTypePtr & getNestedType() { return nested_data_type; }
const DataTypePtr & getNestedType() const { return nested_data_type; }
private:
DataTypePtr nested_data_type;
};
}

View File

@ -2,6 +2,7 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/IDataTypeNumberFixed.h>
#include <DB/DataTypes/DataTypeNull.h>
namespace DB
@ -53,4 +54,9 @@ template <> struct DataTypeFromFieldType<void>
using Type = DataTypeVoid;
};
template <> struct DataTypeFromFieldType<Null>
{
using Type = DataTypeNull;
};
}

View File

@ -5,6 +5,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeNull.h>
#include <DB/Common/Exception.h>
@ -12,12 +13,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Для заданного значения Field возвращает минимальный тип данных, позволяющий хранить значение этого типа.
* В случае, если Field - массив, конвертирует все элементы к общему типу.
*/
@ -26,7 +21,7 @@ class FieldToDataType : public StaticVisitor<DataTypePtr>
public:
DataTypePtr operator() (Null & x) const
{
throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
return std::make_shared<DataTypeNull>();
}
DataTypePtr operator() (UInt64 & x) const

View File

@ -27,11 +27,20 @@ public:
/// Основное имя типа (например, UInt64).
virtual std::string getName() const = 0;
/// Является ли тип числовым. Дата и дата-с-временем тоже считаются такими.
/// Is this type the null type?
virtual bool isNull() const { return false; }
/// Is this type nullable?
virtual bool isNullable() const { return false; }
/// Is this type numeric? Date and DateTime types are considered as such.
virtual bool isNumeric() const { return false; }
/// Если тип числовой, уместны ли с ним все арифметические операции и приведение типов.
/// true для чисел, false для даты и даты-с-временем.
/// Is this type numeric and not nullable?
virtual bool isNumericNotNullable() const { return isNumeric(); }
/// If this type is numeric, are all the arithmetic operations and type casting
/// relevant for it? True for numbers. False for Date and DateTime types.
virtual bool behavesAsNumber() const { return false; }
/// Клонировать
@ -66,24 +75,29 @@ public:
/// Если функция кидает исключение при чтении, то столбец будет находиться в таком же состоянии, как до вызова функции.
virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0;
/** Текстовая сериализация с эскейпингом, но без квотирования.
/** Text serialization with escaping but without quoting.
*/
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const = 0;
/** Текстовая сериализация в виде литерала, который может быть вставлен в запрос.
/** Text serialization as a literal that may be inserted into a query.
*/
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const = 0;
/** Текстовая сериализация для формата CSV.
* delimiter - какого разделителя ожидать при чтении, если строковое значение не в кавычках (сам разделитель не съедается).
/** Text serialization for the CSV format.
*/
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
/** delimiter - the delimiter we expect when reading a string value that is not double-quoted
* (the delimiter is not consumed).
*/
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const = 0;
/** Текстовая сериализация - для вывода на экран / сохранения в текстовый файл и т. п.
* Без эскейпинга и квотирования.
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
* Without escaping or quoting.
*/
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
@ -93,7 +107,7 @@ public:
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, bool force_quoting_64bit_integers) const = 0;
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const = 0;
/** Текстовая сериализация для подстановки в формат XML.
/** Text serialization for putting into the XML format.
*/
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{

View File

@ -0,0 +1,43 @@
#pragma once
#include <cstring>
namespace DB
{
namespace NullSymbol
{
struct Plain
{
static constexpr auto name = "\\N";
};
struct Escaped
{
static constexpr auto name = "\\N";
};
struct Quoted
{
static constexpr auto name = "NULL";
};
struct CSV
{
static constexpr auto name = "\\N";
};
struct JSON
{
static constexpr auto name = "null";
};
struct XML
{
static constexpr auto name = "\\N";
};
}
}

View File

@ -11,17 +11,29 @@ namespace Conditional
/// Determine the return type of the function multiIf when all the
/// branches (then, else) have numeric types.
DataTypePtr getReturnTypeForArithmeticArgs(const DataTypes & args);
/// Returns true if all the branches (then, else) have numeric types.
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) have numeric types.
bool hasArithmeticBranches(const DataTypes & args);
/// Returns true if all the branches (then, else) are arrays.
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are arrays.
bool hasArrayBranches(const DataTypes & args);
/// Returns true if all the branches (then, else) have the same type name.
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) have the same type name.
bool hasIdenticalTypes(const DataTypes & args);
/// Returns true if all the branches (then, else) are fixed strings.
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are fixed strings.
bool hasFixedStrings(const DataTypes & args);
/// Returns true if all the branches (then, else) are fixed strings of equal length.
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are fixed strings of equal length.
bool hasFixedStringsOfIdenticalLength(const DataTypes & args);
/// Returns true if all the branches (then, else) are strings.
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are strings.
bool hasStrings(const DataTypes & args);
}

View File

@ -2,6 +2,7 @@
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/NullMapBuilder.h>
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnVector.h>
@ -279,6 +280,35 @@ public:
}
};
const Array null_array{Null()};
/// Case for null sources.
template <typename TResult>
class ArraySourceCreator<TResult, Null> final
{
public:
static bool execute(IArraySources<TResult> & sources, const Block & block,
const ColumnNumbers & args, const Branch & br)
{
auto type_name = br.type->getName();
if (TypeName<Null>::get() == type_name)
{
const IColumn * col = block.getByPosition(args[br.index]).column.get();
const ColumnNull * null_col = typeid_cast<const ColumnNull *>(col);
if (null_col == nullptr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
IArraySourcePtr<TResult> source;
source = std::make_unique<ConstArraySource<TResult, Null> >(null_array);
sources.push_back(std::move(source));
return true;
}
else
return false;
}
};
}
/// Processing of multiIf in the case of numeric array types.
@ -286,13 +316,16 @@ template <typename TResult>
class ArrayEvaluator final
{
public:
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
const CondSources conds = createConds(block, args);
size_t row_count = conds[0].getSize();
IArraySources<TResult> sources = createSources(block, args, branches);
ArraySink<TResult> sink = createSink(block, sources, result, row_count);
if (builder)
builder.init(args);
for (size_t cur_row = 0; cur_row < row_count; ++cur_row)
{
bool has_triggered_cond = false;
@ -303,6 +336,8 @@ public:
if (cond.get(cur_row))
{
sink.store(sources[cur_source]->get());
if (builder)
builder.update(args[branches[cur_source].index], cur_row);
has_triggered_cond = true;
break;
}
@ -310,7 +345,11 @@ public:
}
if (!has_triggered_cond)
{
sink.store(sources.back()->get());
if (builder)
builder.update(args[branches.back().index], cur_row);
}
for (auto & source : sources)
source->next();
@ -347,7 +386,8 @@ private:
|| ArraySourceCreator<TResult, Int32>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Int64>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Float32>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Float64>::execute(sources, block, args, br)))
|| ArraySourceCreator<TResult, Float64>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Null>::execute(sources, block, args, br)))
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
}
@ -396,7 +436,8 @@ template <>
class ArrayEvaluator<NumberTraits::Error>
{
public:
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
/// For the meaning of the builder parameter, see the FunctionMultiIf::perform() declaration.
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
throw CondException{CondErrorCodes::ARRAY_EVALUATOR_INVALID_TYPES};
}

View File

@ -23,9 +23,10 @@ public:
CondSource & operator=(CondSource &&) = default;
/// Get the value of this condition for a given row.
inline UInt8 get(size_t row) const
inline bool get(size_t row) const
{
return data_array[row];
bool is_null = !null_map.empty() ? (null_map[row] != 0) : false;
return is_null ? false : static_cast<bool>(data_array[row]);
}
inline size_t getSize() const
@ -39,10 +40,15 @@ private:
static const PaddedPODArray<UInt8> & initDataArray(const Block & block, const ColumnNumbers & args,
size_t i, const ColumnPtr & materialized_col_);
static const PaddedPODArray<UInt8> & initNullMap(const Block & block, const ColumnNumbers & args, size_t i);
private:
const ColumnPtr materialized_col;
const PaddedPODArray<UInt8> & data_array;
const PaddedPODArray<UInt8> & null_map;
static const ColumnPtr null_materialized_col;
static const PaddedPODArray<UInt8> empty_null_map;
};
using CondSources = std::vector<CondSource>;

View File

@ -0,0 +1,78 @@
#pragma once
#include <DB/Core/Block.h>
namespace DB
{
namespace Conditional
{
/// Here is provided a way to incrementally build the null map of the result column
/// of a multiIf invokation if its type is nullable.
class NullMapBuilder final
{
public:
/// Create a dummy builder when we don't need any builder, i.e. when the result
/// of multiIf is not nullable.
NullMapBuilder()
: block{empty_block}
{
}
/// This constructor takes the block that contains the original data received
/// by multiIf, i.e. they have not been processed.
NullMapBuilder(Block & block_)
: block{block_}, row_count{block.rowsInFirstColumn()}
{
}
/// Check whether the builder is dummy or not.
operator bool() const { return block; }
bool operator!() const { return !block; }
/// Initialize the builder. For the non-trivial execution paths of multiIf.
void init(const ColumnNumbers & args);
/// Update the null map being built at the row that has just been processed
/// by multiIf. The parameter index indicates the index of the column being
/// checked for nullity. For non-trivial execution paths of multiIf.
void update(size_t index, size_t row);
/// Build the null map. The parameter index has the same meaning as above.
/// For the trivial execution path of multiIf.
void build(size_t index);
/// Accessor needed to return the fully built null map.
ColumnPtr getNullMap() const { return null_map; }
private:
/// Property of a column.
enum Property
{
/// Neither nullable nor null.
IS_ORDINARY = 0,
/// Nullable column.
IS_NULLABLE,
/// Null column.
IS_NULL
};
private:
Block & block;
/// Remember for each column representing an argument whether it is
/// nullable, null, or neither of them. This avoids us many costly
/// calls to virtual functions.
PaddedPODArray<Property> cols_properties;
ColumnPtr null_map;
size_t row_count;
private:
static Block empty_block;
};
}
}

View File

@ -2,6 +2,7 @@
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/NullMapBuilder.h>
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
@ -141,13 +142,16 @@ template <typename TResult>
class NumericEvaluator final
{
public:
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
const CondSources conds = createConds(block, args);
const NumericSources<TResult> sources = createNumericSources(block, args, branches);
size_t row_count = conds[0].getSize();
PaddedPODArray<TResult> & res = createSink(block, result, row_count);
if (builder)
builder.init(args);
for (size_t cur_row = 0; cur_row < row_count; ++cur_row)
{
bool has_triggered_cond = false;
@ -158,6 +162,8 @@ public:
if (cond.get(cur_row))
{
res[cur_row] = sources[cur_source]->get(cur_row);
if (builder)
builder.update(args[branches[cur_source].index], cur_row);
has_triggered_cond = true;
break;
}
@ -165,7 +171,11 @@ public:
}
if (!has_triggered_cond)
{
res[cur_row] = sources.back()->get(cur_row);
if (builder)
builder.update(args[branches.back().index], cur_row);
}
}
}
@ -213,7 +223,8 @@ private:
|| NumericSourceCreator<TResult, Int32>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Int64>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Float32>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Float64>::execute(source, block, args, br)))
|| NumericSourceCreator<TResult, Float64>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Null>::execute(source, block, args, br)))
throw CondException{CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT, toString(br.index)};
sources.push_back(std::move(source));
@ -228,7 +239,8 @@ template <>
class NumericEvaluator<NumberTraits::Error>
{
public:
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
/// For the meaning of the builder parameter, see the FunctionMultiIf::perform() declaration.
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}

View File

@ -9,11 +9,14 @@ namespace DB
namespace Conditional
{
class NullMapBuilder;
struct NumericPerformer
{
/// Perform a multiIf function for numeric branch (then, else) arguments
/// that may have either scalar types or array types.
static bool perform(Block & block, const ColumnNumbers & args, size_t result);
/// For the meaning of the builder parameter, see the FunctionMultiIf::perform() declaration.
static bool perform(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder);
};
}

View File

@ -9,10 +9,15 @@ namespace DB
namespace Conditional
{
class NullMapBuilder;
class StringArrayEvaluator
{
public:
static bool perform(Block & block, const ColumnNumbers & args, size_t result);
/// The tracker parameter is an index to a column that tracks the originating column of each value of
/// the result column. Calling this function with result == tracker means that no such tracking is
/// required, which happens if multiIf is called with no nullable parameters.
static bool perform(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder);
};
}

View File

@ -9,9 +9,12 @@ namespace DB
namespace Conditional
{
class NullMapBuilder;
struct StringEvaluator final
{
static bool perform(Block & block, const ColumnNumbers & args, size_t result);
/// For the meaning of the builder parameter, see the FunctionMultiIf::perform() declaration.
static bool perform(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder);
};
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <DB/Core/Types.h>
namespace DB
{
@ -15,6 +17,7 @@ struct Branch
size_t index;
DataTypePtr type;
bool is_const;
UInt8 category;
};
using Branches = std::vector<Branch>;

View File

@ -3,6 +3,7 @@
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/EnrichedDataTypePtr.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeNullable.h>
namespace DB
{
@ -10,6 +11,10 @@ namespace DB
namespace DataTypeTraits
{
/// If the input type is nullable, return its nested type.
/// Otherwise it is an identity mapping.
const DataTypePtr & removeNullable(const DataTypePtr & type);
template <typename T>
struct DataTypeFromFieldTypeOrError
{
@ -19,6 +24,29 @@ struct DataTypeFromFieldTypeOrError
}
};
template <typename T>
struct DataTypeFromFieldTypeOrError<Nullable<T>>
{
static DataTypePtr getDataType()
{
auto nested_type = DataTypeFromFieldTypeOrError<T>::getDataType();
if (nested_type != nullptr)
return std::make_shared<DataTypeNullable>(nested_type);
else
return nullptr;
}
};
/// Special case for the null type.
template <>
struct DataTypeFromFieldTypeOrError<Nullable<void>>
{
static DataTypePtr getDataType()
{
return std::make_shared<DataTypeNull>();
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
@ -35,11 +63,13 @@ struct ToEnrichedNumericType
private:
using Type0 = typename std::tuple_element<0, T>::type;
using Type1 = typename std::tuple_element<1, T>::type;
using Nullability = typename std::tuple_element<2, T>::type;
public:
using Type = std::tuple<
typename Type0::FieldType,
typename Type1::FieldType
typename Type1::FieldType,
Nullability
>;
};
@ -50,11 +80,13 @@ struct ToEnrichedDataType
private:
using Type0 = typename std::tuple_element<0, T>::type;
using Type1 = typename std::tuple_element<1, T>::type;
using Nullability = typename std::tuple_element<2, T>::type;
public:
using Type = std::tuple<
typename DataTypeFromFieldType<Type0>::Type,
typename DataTypeFromFieldType<Type1>::Type
typename DataTypeFromFieldType<Type1>::Type,
Nullability
>;
};
@ -76,15 +108,57 @@ struct ToEnrichedDataTypeObject<TEnrichedType, true>
static EnrichedDataTypePtr execute()
{
using Type0 = typename std::tuple_element<0, TEnrichedType>::type;
using DataType0 = typename DataTypeFromFieldType<Type0>::Type;
using Type1 = typename std::tuple_element<1, TEnrichedType>::type;
using DataType1 = typename DataTypeFromFieldType<Type1>::Type;
using Nullability = typename std::tuple_element<2, TEnrichedType>::type;
return std::make_pair(std::make_shared<DataType0>(), std::make_shared<DataType1>());
auto obj_type0 = DataTypeFromFieldTypeOrError<
typename NumberTraits::AddNullability<Type0, Nullability>::Type
>::getDataType();
auto obj_type1 = DataTypeFromFieldTypeOrError<
typename NumberTraits::AddNullability<Type1, Nullability>::Type
>::getDataType();
return std::make_pair(obj_type0, obj_type1);
}
};
namespace
{
template <typename T, typename Nullability>
struct CreateDataTypeObject;
template <typename T>
struct CreateDataTypeObject<T, NumberTraits::HasNull>
{
static DataTypePtr execute()
{
return std::make_shared<DataTypeNullable>(std::make_shared<T>());
}
};
/// Special case for the null type.
template <>
struct CreateDataTypeObject<DataTypeVoid, NumberTraits::HasNull>
{
static DataTypePtr execute()
{
return std::make_shared<DataTypeNull>();
}
};
template <typename T>
struct CreateDataTypeObject<T, NumberTraits::HasNoNull>
{
static DataTypePtr execute()
{
return std::make_shared<T>();
}
};
}
/// Convert an enriched data type into an enriched data type object.
template <typename TEnrichedType>
struct ToEnrichedDataTypeObject<TEnrichedType, false>
@ -93,8 +167,12 @@ struct ToEnrichedDataTypeObject<TEnrichedType, false>
{
using DataType0 = typename std::tuple_element<0, TEnrichedType>::type;
using DataType1 = typename std::tuple_element<1, TEnrichedType>::type;
using Nullability = typename std::tuple_element<2, TEnrichedType>::type;
return std::make_pair(std::make_shared<DataType0>(), std::make_shared<DataType1>());
auto obj_type0 = CreateDataTypeObject<DataType0, Nullability>::execute();
auto obj_type1 = CreateDataTypeObject<DataType1, Nullability>::execute();
return std::make_pair(obj_type0, obj_type1);
}
};
@ -132,6 +210,17 @@ struct DataTypeProduct
>::Type;
};
template <typename T1, typename T2>
struct DataTypeProduct<T1, Nullable<T2> >
{
using Type = typename ToEnrichedDataType<
typename NumberTraits::TypeProduct<
typename ToEnrichedNumericType<T1>::Type,
typename NumberTraits::EmbedType<Nullable<typename T2::FieldType> >::Type
>::Type
>::Type;
};
}
}

View File

@ -671,7 +671,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -699,7 +699,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeLeftType<DataTypeDate>(block, arguments, result)
|| executeLeftType<DataTypeDateTime>(block, arguments, result)
@ -784,7 +784,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -810,7 +810,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)

File diff suppressed because it is too large Load Diff

View File

@ -175,7 +175,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -192,7 +192,7 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto & col_name_type = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_name_type.column;
@ -261,7 +261,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -288,7 +288,7 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto & col_name_type = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_name_type.column;
@ -427,7 +427,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -592,7 +592,7 @@ public:
memcpy(dst, tmp, sizeof(tmp));
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
@ -645,7 +645,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -694,7 +694,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
@ -750,7 +750,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -788,7 +788,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
@ -836,7 +836,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -888,7 +888,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
@ -939,7 +939,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -953,7 +953,7 @@ public:
return std::make_shared<DataTypeFixedString>(16);
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto & col_name_type = block.getByPosition(arguments[0]);
const ColumnPtr & column = col_name_type.column;
@ -1009,7 +1009,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1224,7 +1224,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const IColumn * column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
@ -1257,7 +1257,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1296,7 +1296,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
@ -1365,7 +1365,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1444,7 +1444,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const IColumn * in_column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & out_column = block.getByPosition(result).column;
@ -1478,7 +1478,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1594,7 +1594,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const IColumn * column = block.getByPosition(arguments[0]).column.get();
ColumnPtr & res_column = block.getByPosition(result).column;
@ -1621,7 +1621,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception{
@ -1658,7 +1658,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto value_col = block.getByPosition(arguments.front()).column.get();
@ -1802,7 +1802,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception{
@ -1837,7 +1837,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto value_col = block.getByPosition(arguments.front()).column.get();

View File

@ -785,7 +785,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -859,7 +859,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const auto & col_with_type_and_name_left = block.getByPosition(arguments[0]);
const auto & col_with_type_and_name_right = block.getByPosition(arguments[1]);
@ -905,56 +905,4 @@ using FunctionGreater = FunctionComparison<GreaterOp, NameGreater> ;
using FunctionLessOrEquals = FunctionComparison<LessOrEqualsOp, NameLessOrEquals> ;
using FunctionGreaterOrEquals = FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>;
template <>
void FunctionComparison<EqualsOp, NameEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size)
{
return executeTupleEqualityImpl<FunctionComparison<EqualsOp, NameEquals>, FunctionAnd>(block, result, x, y, tuple_size);
}
template <>
void FunctionComparison<NotEqualsOp, NameNotEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size)
{
return executeTupleEqualityImpl<FunctionComparison<NotEqualsOp, NameNotEquals>, FunctionOr>(block, result, x, y, tuple_size);
}
template <>
void FunctionComparison<LessOp, NameLess>::executeTupleImpl(
Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size)
{
return executeTupleLessGreaterImpl<
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOp, NameLess>>(block, result, x, y, tuple_size);
}
template <>
void FunctionComparison<GreaterOp, NameGreater>::executeTupleImpl(
Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size)
{
return executeTupleLessGreaterImpl<
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOp, NameGreater>>(block, result, x, y, tuple_size);
}
template <>
void FunctionComparison<LessOrEqualsOp, NameLessOrEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size)
{
return executeTupleLessGreaterImpl<
FunctionComparison<LessOp, NameLess>,
FunctionComparison<LessOrEqualsOp, NameLessOrEquals>>(block, result, x, y, tuple_size);
}
template <>
void FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>::executeTupleImpl(
Block & block, size_t result, const ColumnTuple * x, const ColumnTuple * y, size_t tuple_size)
{
return executeTupleLessGreaterImpl<
FunctionComparison<GreaterOp, NameGreater>,
FunctionComparison<GreaterOrEqualsOp, NameGreaterOrEquals>>(block, result, x, y, tuple_size);
}
}

View File

@ -15,14 +15,6 @@
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/DataTypeTraits.h>
/// The following includes are needed for the function multiIf.
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/ArgsInfo.h>
#include <DB/Functions/Conditional/NumericPerformer.h>
#include <DB/Functions/Conditional/StringEvaluator.h>
#include <DB/Functions/Conditional/StringArrayEvaluator.h>
namespace DB
{
@ -1327,7 +1319,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1412,7 +1404,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnUInt8 * cond_col = typeid_cast<const ColumnUInt8 *>(block.getByPosition(arguments[0]).column.get());
const ColumnConst<UInt8> * cond_const_col = typeid_cast<const ColumnConst<UInt8> *>(block.getByPosition(arguments[0]).column.get());
@ -1463,6 +1455,13 @@ public:
}
};
namespace Conditional
{
class NullMapBuilder;
class CondException;
}
/// Function multiIf, which generalizes the function if.
///
@ -1476,284 +1475,67 @@ public:
/// - dates with time;
/// - strings;
/// - arrays of such types.
///
/// Additionally the arguments, conditions or branches, support nullable types
/// and the NULL value.
class FunctionMultiIf final : public IFunction
{
public:
static constexpr auto name = "multiIf";
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionMultiIf>(); }
static FunctionPtr create(const Context & context);
public:
String getName() const override
{
return is_case_mode ? "CASE" : name;
}
void setCaseMode()
{
is_case_mode = true;
}
DataTypePtr getReturnType(const DataTypes & args) const override
{
DataTypePtr data_type;
try
{
data_type = getReturnTypeImpl(args);
}
catch (const Conditional::CondException & ex)
{
rethrowContextually(ex);
}
return data_type;
}
void execute(Block & block, const ColumnNumbers & args, size_t result) override
{
try
{
if (performTrivialCase(block, args, result))
return;
if (Conditional::NumericPerformer::perform(block, args, result))
return;
if (Conditional::StringEvaluator::perform(block, args, result))
return;
if (Conditional::StringArrayEvaluator::perform(block, args, result))
return;
if (is_case_mode)
throw Exception{"Some THEN/ELSE clauses in CASE construction have "
"illegal or incompatible types", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"One or more branch (then, else) columns of function "
+ getName() + " have illegal or incompatible types",
ErrorCodes::ILLEGAL_COLUMN};
}
catch (const Conditional::CondException & ex)
{
rethrowContextually(ex);
}
}
String getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result) override;
private:
DataTypePtr getReturnTypeImpl(const DataTypes & args) const
{
if (!Conditional::hasValidArgCount(args))
{
if (is_case_mode)
throw Exception{"Some mandatory parameters are missing in the CASE "
"construction", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
else
throw Exception{"Invalid number of arguments for function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
DataTypePtr getReturnTypeInternal(const DataTypes & args) const;
/// Check conditions types.
for (size_t i = Conditional::firstCond(); i < Conditional::elseArg(args); i = Conditional::nextCond(i))
{
if (!typeid_cast<const DataTypeUInt8 *>(&*args[i]))
{
if (is_case_mode)
throw Exception{"In CASE construction, illegal type of WHEN clause "
+ toString(i / 2) + ". Must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Illegal type of argument " + toString(i) + " (condition) "
"of function " + getName() + ". Must be UInt8.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
}
/// Internal version of multiIf.
/// The builder parameter is an object that incrementally builds the null map
/// of the result column if it is nullable. When no builder is necessary,
/// just pass a default parameter.
void perform(Block & block, const ColumnNumbers & args, size_t result, Conditional::NullMapBuilder & builder);
if (Conditional::hasArithmeticBranches(args))
return Conditional::getReturnTypeForArithmeticArgs(args);
else if (Conditional::hasArrayBranches(args))
{
/// NOTE Сообщения об ошибках будут относится к типам элементов массивов, что немного некорректно.
DataTypes new_args;
new_args.reserve(args.size());
/// Perform multiIf in the case where all the non-null branches have the same type and all
/// the conditions are constant. The same remark as above applies with regards to
/// the builder parameter.
bool performTrivialCase(Block & block, const ColumnNumbers & args, size_t result, Conditional::NullMapBuilder & builder);
};
auto push_branch_arg = [&args, &new_args](size_t i)
{
const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(args[i].get());
if (type_arr == nullptr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
new_args.push_back(type_arr->getNestedType());
};
/// Function caseWithExpr which implements the CASE construction when it is
/// provided an expression. Users should not call this function.
class FunctionCaseWithExpr : public IFunction
{
public:
static constexpr auto name = "caseWithExpr";
static FunctionPtr create(const Context & context_);
for (size_t i = 0; i < Conditional::elseArg(args); ++i)
{
if (Conditional::isCond(i))
new_args.push_back(args[i]);
else
push_branch_arg(i);
}
push_branch_arg(Conditional::elseArg(args));
return std::make_shared<DataTypeArray>(getReturnType(new_args));
}
else if (!Conditional::hasIdenticalTypes(args))
{
if (Conditional::hasFixedStrings(args))
{
if (!Conditional::hasFixedStringsOfIdenticalLength(args))
{
if (is_case_mode)
throw Exception{"THEN/ELSE clauses in CASE construction "
"have FixedString type and different sizes",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Branch (then, else) arguments of function " + getName() +
" have FixedString type and different sizes",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const IDataType * data = args[Conditional::firstThen()].get();
const auto * fixed_str = typeid_cast<const DataTypeFixedString *>(data);
if (fixed_str == nullptr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
return std::make_shared<DataTypeFixedString>(fixed_str->getN());
}
else if (Conditional::hasStrings(args))
return std::make_shared<DataTypeString>();
else
{
if (is_case_mode)
throw Exception{"THEN/ELSE clauses in CASE construction "
"have incompatible arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{
"Incompatible branch (then, else) arguments for function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
}
else
return args[Conditional::firstThen()];
}
bool performTrivialCase(Block & block, const ColumnNumbers & args, size_t result)
{
size_t else_arg = Conditional::elseArg(args);
auto first_type_name = block.getByPosition(args[Conditional::firstThen()]).type->getName();
for (size_t i = Conditional::secondThen(); i < else_arg; i = Conditional::nextThen(i))
{
if (block.getByPosition(args[i]).type->getName() != first_type_name)
return false;
}
if (block.getByPosition(args.back()).type->getName() != first_type_name)
return false;
auto & res_col = block.getByPosition(result).column;
for (size_t i = Conditional::firstCond(); i < else_arg; i = Conditional::nextCond(i))
{
auto cond_const_col = typeid_cast<const ColumnConst<UInt8> *>(block.getByPosition(args[i]).column.get());
if (!cond_const_col)
return false;
bool has_triggered_cond = cond_const_col->getData();
if (has_triggered_cond)
{
res_col = block.getByPosition(args[Conditional::thenFromCond(i)]).column;
return true;
}
}
res_col = block.getByPosition(args[else_arg]).column;
return true;
}
/// Translate a context-free error into a contextual error.
void rethrowContextually(const Conditional::CondException & ex) const
{
if (is_case_mode)
{
/// CASE construction context.
if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE)
throw Exception{"Illegal type of column " + ex.getMsg1() +
" in CASE construction", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR)
throw Exception{"THEN/ELSE clause parameters in CASE construction are not upscalable to a "
"common type without loss of precision: " + ex.getMsg1(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN)
{
size_t i = std::stoul(ex.getMsg1());
if ((i % 2) == 1)
throw Exception{"Illegal THEN clause " + toString(1 + (i - 1) / 2)
+ " in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal ELSE clause in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
}
else if (ex.getCode() == Conditional::CondErrorCodes::COND_SOURCE_ILLEGAL_COLUMN)
{
size_t i = std::stoul(ex.getMsg2());
if ((i % 2) == 1)
throw Exception{"Illegal column " + ex.getMsg1() + " of THEN clause "
+ toString(1 + (i - 1) / 2) + " in CASE construction."
"Must be ColumnUInt8 or ColumnConstUInt8", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + ex.getMsg1() + " of ELSE clause "
" in CASE construction. Must be ColumnUInt8 or ColumnConstUInt8",
ErrorCodes::ILLEGAL_COLUMN};
}
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT)
{
size_t i = std::stoul(ex.getMsg1());
if ((i % 2) == 1)
throw Exception{"Illegal type of THEN clause " + toString(1 + (i - 1) / 2)
+ " in CASE construction", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Illegal type of ELSE clause in CASE construction",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
else if (ex.getCode() == Conditional::CondErrorCodes::ARRAY_EVALUATOR_INVALID_TYPES)
throw Exception{"Internal logic error: one or more THEN/ELSE clauses of "
"CASE construction have invalid types", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"An unexpected error has occurred in CASE construction",
ErrorCodes::LOGICAL_ERROR};
}
else
{
/// multiIf function context.
if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE)
throw Exception{"Illegal type of column " + ex.getMsg1() +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR)
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + ex.getMsg1(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN)
throw Exception{"Illegal argument " + ex.getMsg1() + " of function multiIf",
ErrorCodes::ILLEGAL_COLUMN};
else if (ex.getCode() == Conditional::CondErrorCodes::COND_SOURCE_ILLEGAL_COLUMN)
throw Exception{"Illegal column " + ex.getMsg1() + " of argument "
+ ex.getMsg2() + " of function multiIf"
"Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_COLUMN};
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT)
throw Exception{"Illegal type of argument " + ex.getMsg1() + " of function multiIf",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::ARRAY_EVALUATOR_INVALID_TYPES)
throw Exception{"Internal logic error: one or more arguments of function "
"multiIf have invalid types", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"An unexpected error has occurred while performing multiIf",
ErrorCodes::LOGICAL_ERROR};
}
}
public:
FunctionCaseWithExpr(const Context & context_);
String getName() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result) override;
private:
bool is_case_mode = false;
const Context & context;
};
/// Function caseWithoutExpr which implements the CASE construction when it
/// isn't provided any expression. Users should not call this function.
class FunctionCaseWithoutExpr : public IFunction
{
public:
static constexpr auto name = "caseWithoutExpr";
static FunctionPtr create(const Context & context_);
public:
String getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override;
void executeImpl(Block & block, const ColumnNumbers & args, size_t result) override;
};
}

View File

@ -17,10 +17,12 @@
#include <DB/DataTypes/DataTypeEnum.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/Interpreters/ExpressionActions.h>
#include <DB/Functions/IFunction.h>
@ -43,6 +45,7 @@ namespace ErrorCodes
extern const int CANNOT_PARSE_TEXT;
}
/** Type conversion functions.
* toType - conversion in "natural way";
*/
@ -1303,16 +1306,16 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return getReturnTypeImpl(arguments);
return getReturnTypeInternal(arguments);
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
try
{
executeImpl(block, arguments, result);
executeInternal(block, arguments, result);
}
catch (Exception & e)
{
@ -1352,7 +1355,7 @@ public:
}
private:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result)
{
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
@ -1386,7 +1389,7 @@ private:
}
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<!(std::is_same<ToDataType2, DataTypeString>::value ||
std::is_same<Name2, NameToUnixTimestamp>::value ||
std::is_same<Name2, NameToDate>::value)>::type * = nullptr) const
@ -1402,7 +1405,7 @@ private:
/** Conversion of anything to String. For DateTime, it allows second optional argument - time zone.
*/
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<std::is_same<ToDataType2, DataTypeString>::value>::type * = nullptr) const
{
if ((arguments.size() < 1) || (arguments.size() > 2))
@ -1429,7 +1432,7 @@ private:
}
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<std::is_same<Name2, NameToUnixTimestamp>::value, void>::type * = nullptr) const
{
if ((arguments.size() < 1) || (arguments.size() > 2))
@ -1456,7 +1459,7 @@ private:
}
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
DataTypePtr getReturnTypeInternal(const DataTypes & arguments,
typename std::enable_if<std::is_same<Name2, NameToDate>::value>::type * = nullptr) const
{
if ((arguments.size() < 1) || (arguments.size() > 2))
@ -1494,12 +1497,12 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return getReturnTypeImpl(arguments);
return getReturnTypeInternal(arguments);
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
@ -1511,7 +1514,7 @@ public:
}
private:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const
DataTypePtr getReturnTypeInternal(const DataTypes & arguments) const
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1541,7 +1544,7 @@ public:
* Если функция неприменима для данных аргументов - кинуть исключение.
* Для неконстантных столбцов arguments[i].column = nullptr.
*/
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
void getReturnTypeAndPrerequisitesImpl(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites) override
{
@ -1561,7 +1564,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto n = getSize(block.getByPosition(arguments[1]));
return execute(block, arguments, result, n);
@ -1917,7 +1920,7 @@ private:
};
/// Prepare nested type conversion
const auto nested_function = prepare(from_nested_type, to_nested_type.get());
const auto nested_function = prepareImpl(from_nested_type, to_nested_type.get());
return [nested_function, from_nested_type, to_nested_type] (
Block & block, const ColumnNumbers & arguments, const size_t result)
@ -1996,7 +1999,7 @@ private:
/// Create conversion wrapper for each element in tuple
for (const auto & idx_type : ext::enumerate(from_type->getElements()))
element_wrappers.push_back(prepare(idx_type.second, to_element_types[idx_type.first].get()));
element_wrappers.push_back(prepareImpl(idx_type.second, to_element_types[idx_type.first].get()));
auto function_tuple = FunctionTuple::create(context);
return [element_wrappers, function_tuple, from_element_types, to_element_types]
@ -2140,7 +2143,93 @@ private:
};
}
WrapperType prepare(const DataTypePtr & from_type, const IDataType * const to_type)
/// Actions to be taken when performing a conversion.
struct Action
{
/// If neither the input type nor the output type is nullable or null,
/// we perform the conversion without any pre and/or processing.
static constexpr auto NONE = UInt64(0);
/// The input has a nullable type. We must extract its nested type
/// before performing any conversion.
static constexpr auto UNWRAP_NULLABLE_INPUT = UInt64(1) << 0;
/// The output has a nullable type. We must wrap the result from the
/// conversion into a ColumnNullable.
static constexpr auto WRAP_RESULT_INTO_NULLABLE = UInt64(1) << 1;
/// The input is the NULL value. Before performing any conversion,
/// we will turn it into a single UInt8 zero value.
static constexpr auto CONVERT_NULL = UInt64(1) << 2;
};
WrapperType prepare(const DataTypePtr & from_type, const IDataType * const to_type, const uint64_t action)
{
auto wrapper = prepareImpl((action & Action::CONVERT_NULL) ?
std::make_shared<DataTypeUInt8>() :
from_type,
to_type);
if (action & Action::WRAP_RESULT_INTO_NULLABLE)
{
return [wrapper, action] (Block & block, const ColumnNumbers & arguments, const size_t result)
{
/// Create a temporary block on which to perform the operation.
auto & res = block.getByPosition(result);
const auto & ret_type = res.type;
const auto & nullable_type = static_cast<const DataTypeNullable &>(*ret_type);
const auto & nested_type = nullable_type.getNestedType();
Block tmp_block;
if (action & Action::UNWRAP_NULLABLE_INPUT)
tmp_block = createBlockWithNestedColumns(block, arguments);
else if (action & Action::CONVERT_NULL)
{
/// The input is replaced by a trivial UInt8 column
/// which contains only one row whose value is 0.
tmp_block = block;
auto & elem = tmp_block.unsafeGetByPosition(arguments[0]);
elem.column = std::make_shared<ColumnUInt8>(1, 0);
elem.type = std::make_shared<DataTypeUInt8>();
}
else
tmp_block = block;
size_t tmp_res_index = block.columns();
tmp_block.insert({nullptr, nested_type, ""});
/// Perform the requested conversion.
wrapper(tmp_block, arguments, tmp_res_index);
/// Wrap the result into a nullable column.
ColumnPtr null_map;
if (action & Action::UNWRAP_NULLABLE_INPUT)
{
/// This is a conversion from a nullable to a nullable type.
/// So we just keep the null map of the input argument.
const auto & col = block.getByPosition(arguments[0]).column;
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
null_map = nullable_col.getNullValuesByteMap();
}
else if (action & Action::CONVERT_NULL)
{
/// A NULL value has been converted to a nullable type.
null_map = std::make_shared<ColumnUInt8>(block.rowsInFirstColumn(), 1);
}
else
{
/// This is a conversion from an ordinary type to a nullable type.
/// So we create a trivial null map.
null_map = std::make_shared<ColumnUInt8>(block.rowsInFirstColumn(), 0);
}
const auto & tmp_res = tmp_block.getByPosition(tmp_res_index);
res.column = std::make_shared<ColumnNullable>(tmp_res.column, null_map);
};
}
else
return wrapper;
}
WrapperType prepareImpl(const DataTypePtr & from_type, const IDataType * const to_type)
{
if (const auto to_actual_type = typeid_cast<const DataTypeUInt8 *>(to_type))
return createWrapper(from_type, to_actual_type);
@ -2238,7 +2327,9 @@ public:
String getName() const override { return name; }
void getReturnTypeAndPrerequisites(
bool hasSpecialSupportForNulls() const override { return true; }
void getReturnTypeAndPrerequisitesImpl(
const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites) override
{
@ -2254,12 +2345,58 @@ public:
out_return_type = DataTypeFactory::instance().get(type_col->getData());
wrapper_function = prepare(arguments.front().type, out_return_type.get());
/// Determine whether pre-processing and/or post-processing must take
/// place during conversion.
uint64_t action = Action::NONE;
const auto & from_type = arguments.front().type;
prepareMonotonicityInformation(arguments.front().type, out_return_type.get());
if (from_type->isNullable())
action |= Action::UNWRAP_NULLABLE_INPUT;
else if (from_type->isNull())
action |= Action::CONVERT_NULL;
if (out_return_type->isNullable())
action |= Action::WRAP_RESULT_INTO_NULLABLE;
/// Check that the requested conversion is allowed.
if (!(action & Action::WRAP_RESULT_INTO_NULLABLE))
{
if (action & Action::CONVERT_NULL)
throw Exception{"Cannot convert NULL into a non-nullable type",
ErrorCodes::CANNOT_CONVERT_TYPE};
else if (action & Action::UNWRAP_NULLABLE_INPUT)
throw Exception{"Cannot convert data from a nullable type to a non-nullable type",
ErrorCodes::CANNOT_CONVERT_TYPE};
}
DataTypePtr from_inner_type;
const IDataType * to_inner_type;
/// Create the requested conversion.
if (action & Action::WRAP_RESULT_INTO_NULLABLE)
{
if (action & Action::UNWRAP_NULLABLE_INPUT)
{
const auto & nullable_type = static_cast<const DataTypeNullable &>(*from_type);
from_inner_type = nullable_type.getNestedType();
}
else
from_inner_type = from_type;
const auto & nullable_type = static_cast<const DataTypeNullable &>(*out_return_type);
to_inner_type = nullable_type.getNestedType().get();
}
else
{
from_inner_type = from_type;
to_inner_type = out_return_type.get();
}
wrapper_function = prepare(from_inner_type, to_inner_type, action);
prepareMonotonicityInformation(from_inner_type, to_inner_type);
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
/// drop second argument, pass others
ColumnNumbers new_arguments{arguments.front()};

View File

@ -616,7 +616,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() == 1)
{
@ -649,7 +649,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
@ -714,7 +714,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 0)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -725,7 +725,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstUInt32>(
block.rowsInFirstColumn(),
@ -747,7 +747,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 0)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -758,7 +758,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstUInt16>(
block.rowsInFirstColumn(),
@ -780,7 +780,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 0)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -791,7 +791,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstUInt16>(
block.rowsInFirstColumn(),
@ -813,7 +813,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -828,7 +828,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (const ColumnUInt32 * times = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get()))
{
@ -949,7 +949,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -968,7 +968,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnUInt32 * starts = typeid_cast<const ColumnUInt32 *>(block.getByPosition(arguments[0]).column.get());
const ColumnConstUInt32 * const_starts = typeid_cast<const ColumnConstUInt32 *>(block.getByPosition(arguments[0]).column.get());

View File

@ -204,7 +204,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1 && arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -225,7 +225,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
/// Ключ словаря, определяющий "точку зрения".
std::string dict_key;
@ -297,7 +297,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -323,7 +323,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
/// Ключ словаря, определяющий "точку зрения".
std::string dict_key;
@ -430,7 +430,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1 && arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -451,7 +451,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
/// Ключ словаря, определяющий "точку зрения".
std::string dict_key;
@ -703,7 +703,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1 && arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -724,7 +724,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
RegionsNames::Language language = RegionsNames::Language::RU;
@ -784,7 +784,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception{
@ -808,7 +808,7 @@ private:
return std::make_shared<DataTypeUInt8>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -929,7 +929,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3 && arguments.size() != 4)
throw Exception{
@ -973,7 +973,7 @@ private:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -1206,7 +1206,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 4)
throw Exception{
@ -1244,7 +1244,7 @@ private:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -1499,7 +1499,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3 && arguments.size() != 4)
throw Exception{
@ -1543,7 +1543,7 @@ private:
return std::make_shared<DataType>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -1813,7 +1813,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 4)
throw Exception{
@ -1857,7 +1857,7 @@ private:
return std::make_shared<DataType>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -2084,7 +2084,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception{
@ -2111,7 +2111,7 @@ private:
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
@ -2247,7 +2247,7 @@ public:
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3)
throw Exception{
@ -2282,7 +2282,7 @@ private:
return std::make_shared<DataTypeUInt8>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)

View File

@ -31,7 +31,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
virtual DataTypePtr getReturnType(const DataTypes & arguments) const override
virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -54,7 +54,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
@ -144,7 +144,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
virtual DataTypePtr getReturnType(const DataTypes & arguments) const override
virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -160,7 +160,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)

View File

@ -48,7 +48,7 @@ private:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 4)
throw Exception(
@ -115,7 +115,7 @@ private:
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto size = block.rowsInFirstColumn();

View File

@ -180,7 +180,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -195,7 +195,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(block.getByPosition(arguments[0]).column.get()))
{
@ -241,7 +241,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -256,7 +256,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (const ColumnString * col_from = typeid_cast<const ColumnString *>(block.getByPosition(arguments[0]).column.get()))
{
@ -339,7 +339,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -354,7 +354,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
IDataType * from_type = block.getByPosition(arguments[0]).type.get();
@ -595,13 +595,13 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt64>();
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t rows = block.rowsInFirstColumn();
auto col_to = std::make_shared<ColumnUInt64>(rows);
@ -708,7 +708,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto arg_count = arguments.size();
if (arg_count != 1 && arg_count != 2)
@ -745,7 +745,7 @@ public:
return std::make_shared<DataTypeUInt64>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto arg_count = arguments.size();

View File

@ -609,7 +609,7 @@ public:
/// Вызывается, если хоть один агрумент функции - лямбда-выражение.
/// Для аргументов-лямбда-выражений определяет типы аргументов этих выражений.
void getLambdaArgumentTypes(DataTypes & arguments) const override
void getLambdaArgumentTypesImpl(DataTypes & arguments) const override
{
if (arguments.size() < 1)
throw Exception("Function " + getName() + " needs at least one argument; passed "
@ -639,7 +639,7 @@ public:
arguments[0] = std::make_shared<DataTypeExpression>(nested_types);
}
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
void getReturnTypeAndPrerequisitesImpl(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
ExpressionActions::Actions & out_prerequisites) override
{
@ -713,7 +713,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result) override
{
if (arguments.size() == 1)
{

View File

@ -248,7 +248,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -268,7 +268,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
ColumnPlainPtrs in(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
@ -402,7 +402,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -419,7 +419,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)

View File

@ -22,7 +22,7 @@ public:
private:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 0)
throw Exception{
@ -34,7 +34,7 @@ private:
return std::make_shared<DataTypeFloat64>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConst<Float64>>(
block.rowsInFirstColumn(),
@ -53,7 +53,7 @@ public:
private:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception{
@ -134,7 +134,7 @@ private:
return false;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto arg = block.getByPosition(arguments[0]).column.get();
@ -204,7 +204,7 @@ public:
private:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception{
@ -409,7 +409,7 @@ private:
return false;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto left_arg = block.getByPosition(arguments[0]).column.get();

View File

@ -108,7 +108,7 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 0)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -118,7 +118,7 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstString>(
block.rowsInFirstColumn(), db_name);
@ -138,7 +138,8 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 0)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -151,7 +152,7 @@ public:
/** convertToFullColumn needed because in distributed query processing,
* each server returns its own value.
*/
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = ColumnConstString(
block.rowsInFirstColumn(),
@ -166,6 +167,11 @@ public:
static constexpr auto name = "visibleWidth";
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionVisibleWidth>(); }
bool hasSpecialSupportForNulls() const override
{
return true;
}
/// Получить имя функции.
String getName() const override
{
@ -173,7 +179,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -184,7 +190,11 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
private:
/// Internal version of visibleWidth.
void perform(Block & block, const ColumnNumbers & arguments, size_t result);
};
@ -200,7 +210,8 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -210,7 +221,8 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
/// Выполнить функцию над блоком.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstString>(
block.rowsInFirstColumn(), block.getByPosition(arguments[0]).type->getName());
@ -230,7 +242,7 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -240,7 +252,7 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstString>(
block.rowsInFirstColumn(), block.getByPosition(arguments[0]).column->getName());
@ -261,7 +273,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments.empty())
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -272,7 +284,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t size = block.rowsInFirstColumn();
block.getByPosition(result).column = ColumnConstUInt64(size, size).convertToFullColumn();
@ -293,7 +305,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments.empty())
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -304,7 +316,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t size = block.rowsInFirstColumn();
auto column = std::make_shared<ColumnUInt64>();
@ -335,7 +347,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments.empty())
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -346,7 +358,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t current_block_number = block_number++;
block.getByPosition(result).column = ColumnConstUInt64(block.rowsInFirstColumn(), current_block_number).convertToFullColumn();
@ -371,7 +383,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments.empty())
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -382,7 +394,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t rows_in_block = block.rowsInFirstColumn();
size_t current_row_number = rows.fetch_add(rows_in_block);
@ -411,7 +423,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -431,7 +443,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
IColumn * col = block.getByPosition(arguments[0]).column.get();
double seconds;
@ -481,7 +493,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -492,7 +504,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const auto & src = block.getByPosition(arguments[0]).column;
if (auto converted = src->convertToFullColumnIfConst())
@ -520,7 +532,8 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function '" + getName() + "' doesn't match: passed "
@ -530,7 +543,8 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
/// Выполнить функцию над блоком.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
/// Second argument must be ColumnSet.
ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column;
@ -568,7 +582,8 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
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);
@ -576,7 +591,8 @@ public:
return std::make_shared<DataTypeTuple>(arguments);
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
/// Выполнить функцию над блоком.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
Block tuple_block;
@ -634,7 +650,7 @@ public:
return name;
}
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
void getReturnTypeAndPrerequisitesImpl(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
ExpressionActions::Actions & out_prerequisites) override
{
@ -663,7 +679,8 @@ public:
out_return_type = elems[index - 1]->clone();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
/// Выполнить функцию над блоком.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnTuple * tuple_col = typeid_cast<const ColumnTuple *>(block.getByPosition(arguments[0]).column.get());
const ColumnConstTuple * const_tuple_col = typeid_cast<const ColumnConstTuple *>(block.getByPosition(arguments[0]).column.get());
@ -705,10 +722,10 @@ public:
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIgnore>(); }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override { return std::make_shared<DataTypeUInt8>(); }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { return std::make_shared<DataTypeUInt8>(); }
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstUInt8>(block.rowsInFirstColumn(), 0);
}
@ -734,10 +751,10 @@ public:
static FunctionPtr create(const Context & context) { return std::make_shared<FunctionIndexHint>(); }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override { return std::make_shared<DataTypeUInt8>(); }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { return std::make_shared<DataTypeUInt8>(); }
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstUInt8>(block.rowsInFirstColumn(), 1);
}
@ -757,7 +774,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " requires exactly one argument.",
@ -767,7 +784,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
}
@ -788,7 +805,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -801,7 +818,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
}
@ -824,7 +841,7 @@ public:
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -839,7 +856,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
ColumnPtr first_column = block.getByPosition(arguments[0]).column;
@ -875,7 +892,8 @@ public:
return name;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3 && arguments.size() != 4)
throw Exception("Function " + getName() + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed "
@ -889,7 +907,8 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
/// Выполнить функцию над блоком.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// Уровень значения, при котором полоска имеет нулевую длину.
Int64 max = extractConstant<Int64>(block, arguments, 2, "Third"); /// Уровень значения, при котором полоска имеет максимальную длину.
@ -1030,7 +1049,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto args_size = arguments.size();
if (args_size != 1)
@ -1059,7 +1078,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto in = block.getByPosition(arguments.front()).column.get();
@ -1143,14 +1162,14 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments.empty())
throw Exception("Function " + getName() + " must be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
static const std::string version = getVersion();
block.getByPosition(result).column = std::make_shared<ColumnConstString>(block.rowsInFirstColumn(), version);
@ -1178,14 +1197,14 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments.empty())
throw Exception("Function " + getName() + " must be called without arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<DataTypeUInt32>();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.getByPosition(result).column = std::make_shared<ColumnConstUInt32>(block.rowsInFirstColumn(), uptime);
}
@ -1209,7 +1228,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -1222,7 +1241,7 @@ public:
return type->getReturnType()->clone();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnAggregateFunction * column_with_states = typeid_cast<const ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
if (!column_with_states)
@ -1322,7 +1341,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -1336,7 +1355,7 @@ public:
return res;
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
auto & src = block.getByPosition(arguments.at(0));
auto & res = block.getByPosition(result);
@ -1371,7 +1390,7 @@ public:
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -1384,7 +1403,7 @@ public:
return type->getReturnType()->clone();
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
ColumnAggregateFunction * column_with_states = typeid_cast<ColumnAggregateFunction *>(&*block.getByPosition(arguments.at(0)).column);
if (!column_with_states)
@ -1415,10 +1434,11 @@ public:
String getName() const override { return name; }
void getReturnTypeAndPrerequisites(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
ExpressionActions::Actions & out_prerequisites) override;
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override;
void getReturnTypeAndPrerequisitesImpl(const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
ExpressionActions::Actions & out_prerequisites) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
private:
const Context & global_context;

View File

@ -0,0 +1,102 @@
#pragma once
#include <DB/Functions/IFunction.h>
#include <DB/DataTypes/IDataType.h>
#include <DB/Core/ColumnNumbers.h>
namespace DB
{
class Block;
class Context;
/// Implements the function isNull which returns true if a value
/// is null, false otherwise.
class FunctionIsNull : public IFunction
{
public:
static constexpr auto name = "isNull";
static FunctionPtr create(const Context & context);
std::string getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
/// Implements the function isNotNull which returns true if a value
/// is not null, false otherwise.
class FunctionIsNotNull : public IFunction
{
public:
static constexpr auto name = "isNotNull";
static FunctionPtr create(const Context & context);
std::string getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
/// Implements the function coalesce which takes a set of arguments and
/// returns the value of the leftmost non-null argument. If no such value is
/// found, coalesce() returns NULL.
class FunctionCoalesce : public IFunction
{
public:
static constexpr auto name = "coalesce";
static FunctionPtr create(const Context & context);
std::string getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
/// Implements the function ifNull which takes 2 arguments and returns
/// the value of the 1st argument if it is not null. Otherwise it returns
/// the value of the 2nd argument.
class FunctionIfNull : public IFunction
{
public:
static constexpr auto name = "ifNull";
static FunctionPtr create(const Context & context);
std::string getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
/// Implements the function nullIf which takes 2 arguments and returns
/// NULL if both arguments have the same value. Otherwise it returns the
/// value of the first argument.
class FunctionNullIf : public IFunction
{
public:
static constexpr auto name = "nullIf";
static FunctionPtr create(const Context & context);
std::string getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
/// Implements the function assumeNotNull which takes 1 argument and works as follows:
/// - if the argument is a nullable column, return its embedded column;
/// - otherwise return the original argument.
/// NOTE: assumeNotNull may not be called with the NULL value.
class FunctionAssumeNotNull : public IFunction
{
public:
static constexpr auto name = "assumeNotNull";
static FunctionPtr create(const Context & context);
std::string getName() const override;
bool hasSpecialSupportForNulls() const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
}

View File

@ -159,7 +159,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() > 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -170,7 +170,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
auto col_to = std::make_shared<ColumnVector<ToType>>();
block.getByPosition(result).column = col_to;
@ -205,7 +205,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() > 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -216,7 +216,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!is_initialized)
{

View File

@ -33,7 +33,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -97,7 +97,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
@ -131,7 +131,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -147,7 +147,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (ColumnString * col_from = typeid_cast<ColumnString *>(block.getByPosition(arguments[0]).column.get()))
{

View File

@ -1068,7 +1068,7 @@ namespace
}
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if ((arguments.size() < 1) || (arguments.size() > 2))
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1103,7 +1103,7 @@ namespace
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
if (!( executeForType<UInt8>(block, arguments, result)
|| executeForType<UInt16>(block, arguments, result)

View File

@ -780,7 +780,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -796,7 +796,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
@ -876,7 +876,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -891,7 +891,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
@ -933,11 +933,10 @@ public:
/// Получить имя функции.
String getName() const override;
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override;
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
@ -955,7 +954,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -976,7 +975,7 @@ public:
return std::make_shared<DataTypeString>();
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
if (arguments.size() == 2)
executeBinary(block, arguments, result);
@ -1434,7 +1433,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1453,7 +1452,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column_string = block.getByPosition(arguments[0]).column;
const ColumnPtr column_start = block.getByPosition(arguments[1]).column;
@ -1520,9 +1519,8 @@ public:
String getName() const override;
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override;
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override;
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override;
};

View File

@ -323,7 +323,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
Generator::checkArguments(arguments);
@ -331,7 +331,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
Generator generator;
generator.init(block, arguments);
@ -419,7 +419,7 @@ public:
class FunctionArrayStringConcat : public IFunction
{
private:
void executeImpl(
void executeInternal(
const ColumnString::Chars_t & src_chars,
const ColumnString::Offsets_t & src_string_offsets,
const ColumnArray::Offsets_t & src_array_offsets,
@ -488,7 +488,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 1 && arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -507,7 +507,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
String delimiter;
if (arguments.size() == 2)
@ -541,7 +541,7 @@ public:
std::shared_ptr<ColumnString> col_res = std::make_shared<ColumnString>();
block.getByPosition(result).column = col_res;
executeImpl(
executeInternal(
col_string.getChars(), col_string.getOffsets(), col_arr.getOffsets(),
delimiter.data(), delimiter.size(),
col_res->getChars(), col_res->getOffsets());

View File

@ -1130,7 +1130,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1153,7 +1153,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column_src = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
@ -1217,7 +1217,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1236,7 +1236,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
using ResultType = typename Impl::ResultType;
@ -1302,7 +1302,7 @@ public:
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -1321,7 +1321,7 @@ public:
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;

View File

@ -52,79 +52,45 @@ public:
String getName() const override
{
return is_case_mode ? "CASE" : name;
return name;
}
void setCaseMode()
{
is_case_mode = true;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto args_size = arguments.size();
if (args_size != 3 && args_size != 4)
{
if (is_case_mode)
throw Exception{"Some mandatory parameters are missing in CASE construction",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
else
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(args_size) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(args_size) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const IDataType * type_x = arguments[0].get();
if (!type_x->isNumeric() && !typeid_cast<const DataTypeString *>(type_x))
{
if (is_case_mode)
throw Exception{"Unsupported type " + type_x->getName()
+ " of parameter of the CASE clause"
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Unsupported type " + type_x->getName()
+ " of first argument of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Unsupported type " + type_x->getName()
+ " of first argument of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
const DataTypeArray * type_arr_from = typeid_cast<const DataTypeArray *>(arguments[1].get());
if (!type_arr_from)
{
if (is_case_mode)
throw Exception{"An internal error has been encountered while checking WHEN clauses "
"in CASE construction", ErrorCodes::LOGICAL_ERROR};
else
throw Exception{"Second argument of function " + getName()
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Second argument of function " + getName()
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
const auto type_arr_from_nested = type_arr_from->getNestedType();
if ((type_x->isNumeric() != type_arr_from_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_from_nested.get())))
{
if (is_case_mode)
throw Exception{"The CASE clause and WHEN clauses in CASE construction "
"must have compatible types: both numeric or both strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"First argument and elements of array of second argument of function " + getName()
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"First argument and elements of array of second argument of function " + getName()
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const DataTypeArray * type_arr_to = typeid_cast<const DataTypeArray *>(arguments[2].get());
if (!type_arr_to)
{
if (is_case_mode)
throw Exception{"An internal error has been encountered while checking THEN clauses "
"in CASE construction", ErrorCodes::LOGICAL_ERROR};
else
throw Exception{"Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
const auto enriched_type_arr_to_nested = type_arr_to->getEnrichedNestedType();
const auto & type_arr_to_nested = enriched_type_arr_to_nested.first;
@ -133,18 +99,9 @@ public:
{
if ((type_x->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
{
if (is_case_mode)
throw Exception{"CASE constructions must satisfy either of the following two conditions: "
"1. CASE clause and THEN clauses have common type T; "
"ELSE clause and WHEN clauses have common type U; "
"2. The CASE construction has no ELSE clause; "
"All the clauses have common type T", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Function " + getName()
+ " has signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Function " + getName()
+ " has signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return type_x->clone();
}
@ -153,31 +110,15 @@ public:
const IDataType * type_default = arguments[3].get();
if (!type_default->isNumeric() && !typeid_cast<const DataTypeString *>(type_default))
{
if (is_case_mode)
throw Exception{"Unsupported type " + type_default->getName()
+ " of the ELSE clause of the CASE expression,"
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Unsupported type " + type_default->getName()
+ " of fourth argument (default value) of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Unsupported type " + type_default->getName()
+ " of fourth argument (default value) of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if ((type_default->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_default) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
{
if (is_case_mode)
throw Exception{"CASE constructions must satisfy either of the following two conditions: "
"1. CASE clause and THEN clauses have common type T; "
"ELSE clause and WHEN clauses have common type U; "
"2. The CASE construction has no ELSE clause; "
"All the clauses have common type T", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
@ -191,19 +132,13 @@ public:
}
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const ColumnConstArray * array_from = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[1]).column.get());
const ColumnConstArray * array_to = typeid_cast<const ColumnConstArray *>(block.getByPosition(arguments[2]).column.get());
if (!array_from || !array_to)
{
if (is_case_mode)
throw Exception{"WHEN clauses and THEN clauses in CASE construction "
"must provide constant data", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN};
prepare(array_from->getData(), array_to->getData(), block, arguments);
@ -234,13 +169,9 @@ public:
&& !executeNum<Float64>(in, out, default_column)
&& !executeString(in, out, default_column))
{
if (is_case_mode)
throw Exception{"Illegal parameter in the CASE clause in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
block.getByPosition(result).column = column_result;
@ -284,14 +215,10 @@ private:
auto out = typeid_cast<ColumnVector<T> *>(out_untyped);
if (!out)
{
if (is_case_mode)
throw Exception{"Illegal column " + out_untyped->getName() + "provided to THEN clauses"
" in CASE construction. Must be " + in->getName(), ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
+ ", must be " + in->getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
+ ", must be " + in->getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
executeImplNumToNum<T>(in->getData(), out->getData());
@ -310,14 +237,9 @@ private:
&& !executeNumToNumWithConstDefault<T, Float64>(in, out_untyped)
&& !executeNumToStringWithConstDefault<T>(in, out_untyped))
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
else
@ -334,14 +256,9 @@ private:
&& !executeNumToNumWithNonConstDefault<T, Float64>(in, out_untyped, default_untyped)
&& !executeNumToStringWithNonConstDefault<T>(in, out_untyped, default_untyped))
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -359,14 +276,9 @@ private:
{
if (!executeStringToString(in, out_untyped))
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
else if (default_untyped->isConst())
@ -383,14 +295,9 @@ private:
&& !executeStringToNumWithConstDefault<Float64>(in, out_untyped)
&& !executeStringToStringWithConstDefault(in, out_untyped))
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
else
@ -407,14 +314,9 @@ private:
&& !executeStringToNumWithNonConstDefault<Float64>(in, out_untyped, default_untyped)
&& !executeStringToStringWithNonConstDefault(in, out_untyped, default_untyped))
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -453,14 +355,9 @@ private:
&& !executeNumToNumWithNonConstDefault2<T, U, Float32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float64>(in, out, default_untyped))
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
@ -500,12 +397,8 @@ private:
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
executeImplNumToStringWithNonConstDefault<T>(
@ -545,12 +438,8 @@ private:
&& !executeStringToNumWithNonConstDefault2<U, Float32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float64>(in, out, default_untyped))
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -598,12 +487,8 @@ private:
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
executeImplStringToStringWithNonConstDefault(
@ -857,12 +742,7 @@ private:
const size_t size = from.size();
if (0 == size)
{
if (is_case_mode)
throw Exception{"CASE constructions require WHEN and THEN clauses", ErrorCodes::BAD_ARGUMENTS};
else
throw Exception{"Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS};
}
throw Exception{"Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS};
std::lock_guard<std::mutex> lock(mutex);
@ -870,12 +750,7 @@ private:
return;
if (from.size() != to.size())
{
if (is_case_mode)
throw Exception{"Imbalance between WHEN and THEN clauses", ErrorCodes::BAD_ARGUMENTS};
else
throw Exception{"Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS};
}
throw Exception{"Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS};
Array converted_to;
const Array * used_to = &to;
@ -960,9 +835,6 @@ private:
prepared = true;
}
private:
bool is_case_mode = false;
};
}

View File

@ -51,7 +51,9 @@ public:
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Перегрузка для тех, кому не нужны prerequisites и значения константных аргументов. Снаружи не вызывается.
virtual DataTypePtr getReturnType(const DataTypes & arguments) const
DataTypePtr getReturnType(const DataTypes & arguments) const;
virtual DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const
{
throw Exception("getReturnType is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -62,7 +64,12 @@ public:
* Для неконстантных столбцов arguments[i].column = nullptr.
* Осмысленные типы элементов в out_prerequisites: APPLY_FUNCTION, ADD_COLUMN.
*/
virtual void getReturnTypeAndPrerequisites(
void getReturnTypeAndPrerequisites(
const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites);
virtual void getReturnTypeAndPrerequisitesImpl(
const ColumnsWithTypeAndName & arguments,
DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites)
@ -70,30 +77,40 @@ public:
DataTypes types(arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
types[i] = arguments[i].type;
out_return_type = getReturnType(types);
out_return_type = getReturnTypeImpl(types);
}
/// Вызывается, если хоть один агрумент функции - лямбда-выражение.
/// Для аргументов-лямбда-выражений определяет типы аргументов этих выражений и кладет результат в arguments.
virtual void getLambdaArgumentTypes(DataTypes & arguments) const
void getLambdaArgumentTypes(DataTypes & arguments) const;
virtual void getLambdaArgumentTypesImpl(DataTypes & arguments) const
{
throw Exception("Function " + getName() + " can't have lambda-expressions as arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// Выполнить функцию над блоком. Замечание: может вызываться одновременно из нескольких потоков, для одного объекта.
/// Перегрузка для тех, кому не нужны prerequisites. Снаружи не вызывается.
virtual void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
throw Exception("execute is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
void execute(Block & block, const ColumnNumbers & arguments, size_t result);
/// Выполнить функцию над блоком. Замечание: может вызываться одновременно из нескольких потоков, для одного объекта.
/// prerequisites идут в том же порядке, что и out_prerequisites, полученные из getReturnTypeAndPrerequisites.
virtual void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result)
void execute(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result);
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
{
execute(block, arguments, result);
throw Exception("executeImpl is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual void executeImpl(Block & block, const ColumnNumbers & arguments, const ColumnNumbers & prerequisites, size_t result)
{
executeImpl(block, arguments, result);
}
/// Returns true if the function implementation directly handles the arguments
/// that correspond to nullable columns and null columns.
virtual bool hasSpecialSupportForNulls() const { return false; }
/** Позволяет узнать, является ли функция монотонной в некотором диапазоне значений.
* Это используется для работы с индексом в сортированном куске данных.
* И позволяет использовать индекс не только, когда написано, например date >= const, но и, например, toMonth(date) >= 11.
@ -120,6 +137,42 @@ public:
}
virtual ~IFunction() {}
protected:
/// Returns the copy of a given block in which each column specified in
/// the "arguments" parameter is replaced with its respective nested
/// column if it is nullable.
static Block createBlockWithNestedColumns(const Block & block, ColumnNumbers args);
/// Similar function as above. Additionally transform the result type if needed.
static Block createBlockWithNestedColumns(const Block & block, ColumnNumbers args, size_t result);
private:
/// Strategy to apply when executing a function.
enum Strategy
{
/// Merely perform the function on its columns.
DIRECTLY_EXECUTE = 0,
/// If at least one argument is nullable, call the function implementation
/// with a block in which nullable columns that correspond to function arguments
/// have been replaced with their respective nested columns. Subsequently, the
/// result column is wrapped into a nullable column.
PROCESS_NULLABLE_COLUMNS,
/// If at least one argument is NULL, return NULL.
RETURN_NULL
};
private:
/// Choose the strategy for performing the function.
Strategy chooseStrategy(const Block & block, const ColumnNumbers & args);
/// If required by the specified strategy, process the given block, then
/// return the processed block. Otherwise return an empty block.
Block preProcessBlock(Strategy strategy, const Block & block, const ColumnNumbers & args,
size_t result);
/// If required by the specified strategy, post-process the result column.
void postProcessResult(Strategy strategy, Block & block, const Block & processed_block,
const ColumnNumbers & args, size_t result);
};

View File

@ -30,6 +30,9 @@ using Signed = boost::mpl::true_ ;
using Integer = boost::mpl::false_ ;
using Floating = boost::mpl::true_ ;
using HasNull = boost::mpl::true_;
using HasNoNull = boost::mpl::false_;
using Bits0 = boost::mpl::int_<0> ;
using Bits8 = boost::mpl::int_<8> ;
using Bits16 = boost::mpl::int_<16> ;
@ -39,6 +42,21 @@ using BitsTooMany = boost::mpl::int_<1024>;
struct Error {};
template <typename T, typename Nullability>
struct AddNullability;
template <typename T>
struct AddNullability<T, HasNull>
{
using Type = Nullable<T>;
};
template <typename T>
struct AddNullability<T, HasNoNull>
{
using Type = T;
};
template <typename T> struct Next;
template <> struct Next<Bits0> { using Type = Bits0; };
@ -52,41 +70,73 @@ template <> struct ExactNext<Bits64> { using Type = BitsTooMany; };
template <typename T> struct Traits;
template <> struct Traits<void> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits0 Bits; };
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; };
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; };
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; };
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; };
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; };
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; };
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; };
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; };
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; };
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; };
template <typename T>
struct Traits<Nullable<T>>
{
using Sign = typename Traits<T>::Sign;
using Floatness = typename Traits<T>::Floatness;
using Bits = typename Traits<T>::Bits;
using Nullity = HasNull;
};
template <typename Sign, typename Floatness, typename Bits> struct Construct;
template <> struct Traits<void> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits0 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Null> : Traits<Nullable<void>> {};
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; typedef HasNoNull Nullity; };
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; typedef HasNoNull Nullity; };
template <> struct Construct<Unsigned, Integer, Bits0> { using Type = void ; };
template <> struct Construct<Unsigned, Floating, Bits0> { using Type = void ; };
template <> struct Construct<Signed, Integer, Bits0> { using Type = void ; };
template <> struct Construct<Signed, Floating, Bits0> { using Type = void ; };
template <> struct Construct<Unsigned, Integer, Bits8> { using Type = UInt8 ; };
template <> struct Construct<Unsigned, Integer, Bits16> { using Type = UInt16 ; };
template <> struct Construct<Unsigned, Integer, Bits32> { using Type = UInt32 ; };
template <> struct Construct<Unsigned, Integer, Bits64> { using Type = UInt64 ; };
template <> struct Construct<Unsigned, Floating, Bits8> { using Type = Float32 ; };
template <> struct Construct<Unsigned, Floating, Bits16> { using Type = Float32 ; };
template <> struct Construct<Unsigned, Floating, Bits32> { using Type = Float32 ; };
template <> struct Construct<Unsigned, Floating, Bits64> { using Type = Float64 ; };
template <> struct Construct<Signed, Integer, Bits8> { using Type = Int8 ; };
template <> struct Construct<Signed, Integer, Bits16> { using Type = Int16 ; };
template <> struct Construct<Signed, Integer, Bits32> { using Type = Int32 ; };
template <> struct Construct<Signed, Integer, Bits64> { using Type = Int64 ; };
template <> struct Construct<Signed, Floating, Bits8> { using Type = Float32 ; };
template <> struct Construct<Signed, Floating, Bits16> { using Type = Float32 ; };
template <> struct Construct<Signed, Floating, Bits32> { using Type = Float32 ; };
template <> struct Construct<Signed, Floating, Bits64> { using Type = Float64 ; };
template <typename Sign, typename Floatness> struct Construct<Sign, Floatness, BitsTooMany> { using Type = Error; };
template <typename Sign, typename Floatness, typename Bits, typename Nullity> struct Construct;
template <typename Sign, typename Floatness, typename Bits>
struct Construct<Sign, Floatness, Bits, HasNull>
{
using Type = Nullable<typename Construct<Sign, Floatness, Bits, HasNoNull>::Type>;
};
template <> struct Construct<Unsigned, Integer, Bits0, HasNull> { using Type = Null; };
template <> struct Construct<Unsigned, Floating, Bits0, HasNull> { using Type = Null; };
template <> struct Construct<Signed, Integer, Bits0, HasNull> { using Type = Null; };
template <> struct Construct<Signed, Floating, Bits0, HasNull> { using Type = Null; };
template <typename Sign, typename Floatness>
struct Construct<Sign, Floatness, BitsTooMany, HasNull>
{
using Type = Error;
};
template <typename Sign, typename Floatness>
struct Construct<Sign, Floatness, BitsTooMany, HasNoNull>
{
using Type = Error;
};
template <> struct Construct<Unsigned, Integer, Bits0, HasNoNull> { using Type = void; };
template <> struct Construct<Unsigned, Floating, Bits0, HasNoNull> { using Type = void; };
template <> struct Construct<Signed, Integer, Bits0, HasNoNull> { using Type = void; };
template <> struct Construct<Signed, Floating, Bits0, HasNoNull> { using Type = void; };
template <> struct Construct<Unsigned, Integer, Bits8, HasNoNull> { using Type = UInt8 ; };
template <> struct Construct<Unsigned, Integer, Bits16, HasNoNull> { using Type = UInt16 ; };
template <> struct Construct<Unsigned, Integer, Bits32, HasNoNull> { using Type = UInt32 ; };
template <> struct Construct<Unsigned, Integer, Bits64, HasNoNull> { using Type = UInt64 ; };
template <> struct Construct<Unsigned, Floating, Bits8, HasNoNull> { using Type = Float32 ; };
template <> struct Construct<Unsigned, Floating, Bits16, HasNoNull> { using Type = Float32 ; };
template <> struct Construct<Unsigned, Floating, Bits32, HasNoNull> { using Type = Float32 ; };
template <> struct Construct<Unsigned, Floating, Bits64, HasNoNull> { using Type = Float64 ; };
template <> struct Construct<Signed, Integer, Bits8, HasNoNull> { using Type = Int8 ; };
template <> struct Construct<Signed, Integer, Bits16, HasNoNull> { using Type = Int16 ; };
template <> struct Construct<Signed, Integer, Bits32, HasNoNull> { using Type = Int32 ; };
template <> struct Construct<Signed, Integer, Bits64, HasNoNull> { using Type = Int64 ; };
template <> struct Construct<Signed, Floating, Bits8, HasNoNull> { using Type = Float32 ; };
template <> struct Construct<Signed, Floating, Bits16, HasNoNull> { using Type = Float32 ; };
template <> struct Construct<Signed, Floating, Bits32, HasNoNull> { using Type = Float32 ; };
template <> struct Construct<Signed, Floating, Bits64, HasNoNull> { using Type = Float64 ; };
template <typename T>
inline bool isErrorType()
@ -99,6 +149,18 @@ inline bool isErrorType<Error>()
return true;
}
/// Returns the type A augmented with nullity = nullity(A) | nullity(B)
template <typename A, typename B>
struct UpdateNullity
{
using Type = typename Construct<
typename Traits<A>::Sign,
typename Traits<A>::Floatness,
typename Traits<A>::Bits,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type
>::Type;
};
/** Результат сложения или умножения вычисляется по следующим правилам:
* - если один из аргументов с плавающей запятой, то результат - с плавающей запятой, иначе - целый;
* - если одно из аргументов со знаком, то результат - со знаком, иначе - без знака;
@ -110,7 +172,8 @@ template <typename A, typename B> struct ResultOfAdditionMultiplication
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
};
template <typename A, typename B> struct ResultOfSubtraction
@ -118,7 +181,8 @@ template <typename A, typename B> struct ResultOfSubtraction
typedef typename Construct<
Signed,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
};
/** При делении всегда получается число с плавающей запятой.
@ -135,7 +199,8 @@ template <typename A, typename B> struct ResultOfIntegerDivision
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
Integer,
typename Traits<A>::Bits>::Type Type;
typename Traits<A>::Bits,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
};
/** При взятии остатка получается число, битность которого равна делителю.
@ -145,7 +210,8 @@ template <typename A, typename B> struct ResultOfModulo
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
Integer,
typename Traits<B>::Bits>::Type Type;
typename Traits<B>::Bits,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
};
template <typename A> struct ResultOfNegate
@ -156,7 +222,8 @@ template <typename A> struct ResultOfNegate
typename boost::mpl::if_<
typename Traits<A>::Sign,
typename Traits<A>::Bits,
typename Next<typename Traits<A>::Bits>::Type>::type>::Type Type;
typename Next<typename Traits<A>::Bits>::Type>::type,
typename Traits<A>::Nullity>::Type Type;
};
template <typename A> struct ResultOfAbs
@ -164,7 +231,8 @@ template <typename A> struct ResultOfAbs
typedef typename Construct<
Unsigned,
typename Traits<A>::Floatness,
typename Traits <A>::Bits>::Type Type;
typename Traits <A>::Bits,
typename Traits<A>::Nullity>::Type Type;
};
/** При побитовых операциях получается целое число, битность которого равна максимальной из битностей аргументов.
@ -182,7 +250,8 @@ template <typename A, typename B> struct ResultOfBit
typename boost::mpl::if_<
typename Traits<B>::Floatness,
Bits64,
typename Traits<B>::Bits>::type>::type>::Type Type;
typename Traits<B>::Bits>::type>::type,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type Type;
};
template <typename A> struct ResultOfBitNot
@ -190,7 +259,8 @@ template <typename A> struct ResultOfBitNot
typedef typename Construct<
typename Traits<A>::Sign,
Integer,
typename Traits<A>::Bits>::Type Type;
typename Traits<A>::Bits,
typename Traits<A>::Nullity>::Type Type;
};
/** Приведение типов для функции if:
@ -210,11 +280,11 @@ struct ResultOfIf
/// 1)
typename boost::mpl::if_<
typename boost::mpl::equal_to<typename Traits<A>::Bits, Bits0>::type,
B,
typename UpdateNullity<B, A>::Type,
typename boost::mpl::if_<
typename boost::mpl::equal_to<typename Traits<B>::Bits, Bits0>::type,
A,
/// 4) и 6)
typename UpdateNullity<A, B>::Type,
/// 4) and 6)
typename boost::mpl::if_<
typename boost::mpl::or_<
typename Traits<A>::Floatness,
@ -232,8 +302,9 @@ struct ResultOfIf
typename Traits<B>::Floatness,
typename Traits<B>::Bits,
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type,
Bits32>::type>::Type,
/// 2) и 3)
Bits32>::type,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type>::Type,
/// 2) and 3)
typename boost::mpl::if_<
typename boost::mpl::equal_to<
typename Traits<A>::Sign,
@ -242,8 +313,8 @@ struct ResultOfIf
typename boost::mpl::less<
typename Traits<A>::Bits,
typename Traits<B>::Bits>::type,
B,
A>::type,
typename UpdateNullity<B, A>::Type,
typename UpdateNullity<A, B>::Type>::type,
/// 5)
typename Construct<
Signed,
@ -256,7 +327,9 @@ struct ResultOfIf
typename boost::mpl::if_<
typename Traits<B>::Sign,
typename Traits<B>::Bits,
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type>::Type>::type>::type>::type>::type Type;
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type,
typename boost::mpl::or_<typename Traits<A>::Nullity, typename Traits<B>::Nullity>::type
>::Type>::type>::type>::type>::type Type;
};
/** Перед применением оператора % и побитовых операций, операнды приводятся к целым числам. */
@ -268,7 +341,9 @@ template <typename A> struct ToInteger
typename boost::mpl::if_<
typename Traits<A>::Floatness,
Bits64,
typename Traits<A>::Bits>::type>::Type Type;
typename Traits<A>::Bits>::type,
typename Traits<A>::Nullity
>::Type Type;
};
/// Notes on type composition.
@ -373,19 +448,19 @@ namespace Enriched
{
/// Definitions of enriched types.
using Void = std::tuple<void, void>;
using Int8 = std::tuple<DB::Int8, void>;
using Int16 = std::tuple<DB::Int16, void>;
using Int32 = std::tuple<DB::Int32, void>;
using Int64 = std::tuple<DB::Int64, void>;
using UInt8 = std::tuple<DB::UInt8, void>;
using UInt16 = std::tuple<DB::UInt16, void>;
using UInt32 = std::tuple<DB::UInt32, void>;
using UInt64 = std::tuple<DB::UInt64, void>;
using Float32 = std::tuple<DB::Float32, void>;
using Float64 = std::tuple<DB::Float64, void>;
using IntFloat32 = std::tuple<DB::Int32, DB::Float32>;
using IntFloat64 = std::tuple<DB::Int64, DB::Float64>;
template <typename Nullity> using Void = std::tuple<void, void, Nullity>;
template <typename Nullity> using Int8 = std::tuple<DB::Int8, void, Nullity>;
template <typename Nullity> using Int16 = std::tuple<DB::Int16, void, Nullity>;
template <typename Nullity> using Int32 = std::tuple<DB::Int32, void, Nullity>;
template <typename Nullity> using Int64 = std::tuple<DB::Int64, void, Nullity>;
template <typename Nullity> using UInt8 = std::tuple<DB::UInt8, void, Nullity>;
template <typename Nullity> using UInt16 = std::tuple<DB::UInt16, void, Nullity>;
template <typename Nullity> using UInt32 = std::tuple<DB::UInt32, void, Nullity>;
template <typename Nullity> using UInt64 = std::tuple<DB::UInt64, void, Nullity>;
template <typename Nullity> using Float32 = std::tuple<DB::Float32, void, Nullity>;
template <typename Nullity> using Float64 = std::tuple<DB::Float64, void, Nullity>;
template <typename Nullity> using IntFloat32 = std::tuple<DB::Int32, DB::Float32, Nullity>;
template <typename Nullity> using IntFloat64 = std::tuple<DB::Int64, DB::Float64, Nullity>;
}
@ -393,23 +468,41 @@ using IntFloat64 = std::tuple<DB::Int64, DB::Float64>;
template <typename T>
struct EmbedType;
template <> struct EmbedType<void> { using Type = Enriched::Void; };
template <> struct EmbedType<Int8> { using Type = Enriched::Int8; };
template <> struct EmbedType<Int16> { using Type = Enriched::Int16; };
template <> struct EmbedType<Int32> { using Type = Enriched::Int32; };
template <> struct EmbedType<Int64> { using Type = Enriched::Int64; };
template <> struct EmbedType<UInt8> { using Type = Enriched::UInt8; };
template <> struct EmbedType<UInt16> { using Type = Enriched::UInt16; };
template <> struct EmbedType<UInt32> { using Type = Enriched::UInt32; };
template <> struct EmbedType<UInt64> { using Type = Enriched::UInt64; };
template <> struct EmbedType<Float32> { using Type = Enriched::Float32; };
template <> struct EmbedType<Float64> { using Type = Enriched::Float64; };
template <> struct EmbedType<Error> { using Type = Error; };
template <> struct EmbedType<void> { using Type = Enriched::Void<HasNoNull>; };
template <> struct EmbedType<Int8> { using Type = Enriched::Int8<HasNoNull>; };
template <> struct EmbedType<Int16> { using Type = Enriched::Int16<HasNoNull>; };
template <> struct EmbedType<Int32> { using Type = Enriched::Int32<HasNoNull>; };
template <> struct EmbedType<Int64> { using Type = Enriched::Int64<HasNoNull>; };
template <> struct EmbedType<UInt8> { using Type = Enriched::UInt8<HasNoNull>; };
template <> struct EmbedType<UInt16> { using Type = Enriched::UInt16<HasNoNull>; };
template <> struct EmbedType<UInt32> { using Type = Enriched::UInt32<HasNoNull>; };
template <> struct EmbedType<UInt64> { using Type = Enriched::UInt64<HasNoNull>; };
template <> struct EmbedType<Float32> { using Type = Enriched::Float32<HasNoNull>; };
template <> struct EmbedType<Float64> { using Type = Enriched::Float64<HasNoNull>; };
template <> struct EmbedType<Null> { using Type = Enriched::Void<HasNull>; };
template <> struct EmbedType<Nullable<Int8> > { using Type = Enriched::Int8<HasNull>; };
template <> struct EmbedType<Nullable<Int16> > { using Type = Enriched::Int16<HasNull>; };
template <> struct EmbedType<Nullable<Int32> > { using Type = Enriched::Int32<HasNull>; };
template <> struct EmbedType<Nullable<Int64> > { using Type = Enriched::Int64<HasNull>; };
template <> struct EmbedType<Nullable<UInt8> > { using Type = Enriched::UInt8<HasNull>; };
template <> struct EmbedType<Nullable<UInt16> > { using Type = Enriched::UInt16<HasNull>; };
template <> struct EmbedType<Nullable<UInt32> > { using Type = Enriched::UInt32<HasNull>; };
template <> struct EmbedType<Nullable<UInt64> > { using Type = Enriched::UInt64<HasNull>; };
template <> struct EmbedType<Nullable<Float32> > { using Type = Enriched::Float32<HasNull>; };
template <> struct EmbedType<Nullable<Float64> > { using Type = Enriched::Float64<HasNull>; };
/// Get an ordinary type from an enriched type.
template <typename TType>
struct ToOrdinaryType
{
using Type = typename std::tuple_element<0, TType>::type;
using Type = typename std::conditional<
std::is_same<typename std::tuple_element<2, TType>::type, HasNoNull>::value,
typename std::tuple_element<0, TType>::type,
Nullable<typename std::tuple_element<0, TType>::type>
>::type;
};
/// Get an ordinary type from an enriched type.
@ -420,83 +513,142 @@ struct ToOrdinaryType<Error>
using Type = Error;
};
/// Compute the product of two enriched numeric types.
namespace
{
/// The following helper functions and structures are used for the TypeProduct implementation.
/// Check if two types are equal up to nullity.
template <typename T1, typename T2>
constexpr bool areSimilarTypes()
{
return std::is_same<
typename std::tuple_element<0, T1>::type,
typename std::tuple_element<0, T2>::type
>::value &&
std::is_same<
typename std::tuple_element<1, T1>::type,
typename std::tuple_element<1, T2>::type
>::value;
}
/// Check if a pair of types {A,B} equals a pair of types {A1,B1} up to nullity.
template <typename A, typename B, template <typename> class A1, template <typename> class B1>
constexpr bool areSimilarPairs()
{
/// NOTE: the use of HasNoNull here is a trick. It has no meaning.
return (areSimilarTypes<A, A1<HasNoNull>>() && areSimilarTypes<B, B1<HasNoNull>>()) ||
(areSimilarTypes<A, B1<HasNoNull>>() && areSimilarTypes<B, A1<HasNoNull>>());
}
/// Check if a pair of enriched types {A,B} that have straight mappings to ordinary
/// types must be processed in a special way.
template <typename A, typename B>
constexpr bool isExceptionalPair()
{
return areSimilarPairs<A, B, Enriched::Int8, Enriched::UInt16>() ||
areSimilarPairs<A, B, Enriched::Int8, Enriched::UInt32>() ||
areSimilarPairs<A, B, Enriched::Int16, Enriched::UInt16>() ||
areSimilarPairs<A, B, Enriched::Int16, Enriched::UInt32>() ||
areSimilarPairs<A, B, Enriched::Int32, Enriched::UInt32>();
}
/// Check if a pair of enriched types {A,B} is ordinary. Here "ordinary" means
/// that both types map to ordinary types and that they are not exceptional as
/// defined in the function above.
template <typename A, typename B>
constexpr bool isOrdinaryPair()
{
return std::is_same<typename std::tuple_element<1, A>::type, void>::value &&
std::is_same<typename std::tuple_element<1, B>::type, void>::value &&
!isExceptionalPair<A, B>();
}
/// Returns nullity(A) | nullity(B).
template <typename A, typename B>
struct CombinedNullity
{
private:
using NullityA = typename Traits<typename ToOrdinaryType<A>::Type>::Nullity;
using NullityB = typename Traits<typename ToOrdinaryType<B>::Type>::Nullity;
public:
using Type = typename boost::mpl::or_<NullityA, NullityB>::type;
};
}
/// Compute the product of two enriched numeric types.
/// This statement catches all the incorrect combinations.
template <typename T1, typename T2, typename Enable = void>
struct TypeProduct
{
using Type = Error;
};
/// Compute the product of two enriched numeric types.
/// Case when both of the source types and the resulting type map to ordinary types.
/// Case when both these types are ordinary in the meaning defined above.
template <typename A, typename B>
struct TypeProduct<std::tuple<A, void>, std::tuple<B, void> >
struct TypeProduct<A, B, typename std::enable_if<isOrdinaryPair<A, B>()>::type>
{
private:
using Result = typename NumberTraits::ResultOfIf<A, B>::Type;
using Result = typename ResultOfIf<
typename ToOrdinaryType<A>::Type,
typename ToOrdinaryType<B>::Type
>::Type;
public:
using Type = typename std::conditional<
std::is_same<Result, Error>::value,
Error,
std::tuple<Result, void>
>::type;
using Type = typename EmbedType<Result>::Type;
};
/// Compute the product of two enriched numeric types.
/// Case when a source type or the resulting type does not map to any ordinary type.
template <> struct TypeProduct<Enriched::Int8, Enriched::UInt16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::Int8> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int8, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::Int8> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int16, Enriched::UInt16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::Int16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int16, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::Int16> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int32, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::Int32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int8> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int8, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int16, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int32> { using Type = Enriched::Int32; };
template <> struct TypeProduct<Enriched::Int32, Enriched::IntFloat32> { using Type = Enriched::Int32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int64> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::Int64, Enriched::IntFloat32> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Float32> { using Type = Enriched::Float32; };
template <> struct TypeProduct<Enriched::Float32, Enriched::IntFloat32> { using Type = Enriched::Float32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Float64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::Float64, Enriched::IntFloat32> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::UInt8> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt8, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::UInt16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::IntFloat32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::IntFloat32> { using Type = Enriched::IntFloat64; };
#define DEFINE_TYPE_PRODUCT_RULE(T1, T2, T3) \
template <typename A, typename B> \
struct TypeProduct< \
A, \
B, \
typename std::enable_if< \
!isOrdinaryPair<A, B>() && \
areSimilarPairs<A, B, T1, T2>() \
>::type> \
{ \
using Type = typename T3<typename CombinedNullity<A, B>::Type>; \
}
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int8> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int8, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int16> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int16, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int32, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int64> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::Int64, Enriched::IntFloat64> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Float32> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::Float32, Enriched::IntFloat64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Float64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::Float64, Enriched::IntFloat64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::UInt8> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt8, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::UInt16> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int8, Enriched::UInt16, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int8, Enriched::UInt32, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int16, Enriched::UInt16, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int16, Enriched::UInt32, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::Int32, Enriched::UInt32, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Int8, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Int16, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Int32, Enriched::Int32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Int64, Enriched::Int64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Float32, Enriched::Float32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Float64, Enriched::Float64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::UInt8, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::UInt16, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::UInt32, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::IntFloat32, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::IntFloat64, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Int8, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Int16, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Int32, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Int64, Enriched::Int64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Float32, Enriched::Float64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Float64, Enriched::Float64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::UInt8, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::UInt16, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::UInt32, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::IntFloat64, Enriched::IntFloat64);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat32, Enriched::Void, Enriched::IntFloat32);
DEFINE_TYPE_PRODUCT_RULE(Enriched::IntFloat64, Enriched::Void, Enriched::IntFloat64);
#undef DEFINE_TYPE_PRODUCT_RULE
}

View File

@ -813,4 +813,12 @@ inline String toString(const T & x)
return res;
}
/// Write a specified number of times a given value into a write buffer.
template <typename T>
inline void writeBinary(const T & x, size_t count, WriteBuffer & buf)
{
for (size_t i = 0; i < count; ++i)
writeBinary(x, buf);
}
}

View File

@ -11,6 +11,7 @@
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnNullable.h>
template <>
@ -22,8 +23,48 @@ namespace DB
using Sizes = std::vector<size_t>;
/// When packing the values of nullable columns at a given row, we have to
/// store the fact that these values are nullable or not. This is achieved
/// by encoding this information as a bitmap. Let S be the size in bytes of
/// a packed values binary blob and T the number of bytes we may place into
/// this blob, the size that the bitmap shall occupy in the blob is equal to:
/// ceil(T/8). Thus we must have: S = T + ceil(T/8). Below we indicate for
/// each value of S, the corresponding value of T, and the bitmap size:
///
/// 32,28,4
/// 16,14,2
/// 8,7,1
/// 4,3,1
/// 2,1,1
///
/// Записать набор ключей фиксированной длины в T, уложив их подряд (при допущении, что они помещаются).
namespace
{
template <typename T>
constexpr auto getBitmapSize()
{
return
(sizeof(T) == 32) ?
4 :
(sizeof(T) == 16) ?
2 :
((sizeof(T) == 8) ?
1 :
((sizeof(T) == 4) ?
1 :
((sizeof(T) == 2) ?
1 :
0)));
}
}
template <typename T>
using KeysNullMap = std::array<UInt8, getBitmapSize<T>()>;
/// Pack into a binary blob of type T a set of fixed-size keys. Granted that all the keys fit into the
/// binary blob, they are disposed in it consecutively.
template <typename T>
static inline T ALWAYS_INLINE packFixed(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, const Sizes & key_sizes)
@ -31,11 +72,11 @@ static inline T ALWAYS_INLINE packFixed(
union
{
T key;
char bytes[sizeof(key)];
char bytes[sizeof(key)] = {};
};
memset(bytes, 0, sizeof(key));
size_t offset = 0;
for (size_t j = 0; j < keys_size; ++j)
{
switch (key_sizes[j])
@ -65,8 +106,74 @@ static inline T ALWAYS_INLINE packFixed(
return key;
}
/// Similar as above but supports nullable values.
template <typename T>
static inline T ALWAYS_INLINE packFixed(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, const Sizes & key_sizes,
const KeysNullMap<T> & bitmap)
{
union
{
T key;
char bytes[sizeof(key)] = {};
};
/// Хэшировать набор ключей в UInt128.
size_t offset = 0;
static constexpr auto bitmap_size = std::tuple_size<KeysNullMap<T>>::value;
static constexpr bool has_bitmap = bitmap_size > 0;
if (has_bitmap)
{
memcpy(bytes + offset, bitmap.data(), bitmap_size * sizeof(UInt8));
offset += bitmap_size;
}
for (size_t j = 0; j < keys_size; ++j)
{
bool is_null;
if (!has_bitmap)
is_null = false;
else
{
size_t bucket = j / 8;
size_t off = j % 8;
is_null = ((bitmap[bucket] >> off) & 1) == 1;
}
if (is_null)
continue;
switch (key_sizes[j])
{
case 1:
memcpy(bytes + offset, &static_cast<const ColumnUInt8 *>(key_columns[j])->getData()[i], 1);
offset += 1;
break;
case 2:
memcpy(bytes + offset, &static_cast<const ColumnUInt16 *>(key_columns[j])->getData()[i], 2);
offset += 2;
break;
case 4:
memcpy(bytes + offset, &static_cast<const ColumnUInt32 *>(key_columns[j])->getData()[i], 4);
offset += 4;
break;
case 8:
memcpy(bytes + offset, &static_cast<const ColumnUInt64 *>(key_columns[j])->getData()[i], 8);
offset += 8;
break;
default:
memcpy(bytes + offset, &static_cast<const ColumnFixedString *>(key_columns[j])->getChars()[i * key_sizes[j]], key_sizes[j]);
offset += key_sizes[j];
}
}
return key;
}
/// Hash a set of keys into a UInt128 value.
static inline UInt128 ALWAYS_INLINE hash128(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys)
{
@ -86,7 +193,7 @@ static inline UInt128 ALWAYS_INLINE hash128(
}
/// Почти то же самое, но без возврата ссылок на данные ключей.
/// Almost the same as above but it doesn't return any reference to key data.
static inline UInt128 ALWAYS_INLINE hash128(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns)
{
@ -141,17 +248,29 @@ static inline StringRef * ALWAYS_INLINE extractKeysAndPlaceInPool(
}
/** Скопировать ключи в пул в непрерывный кусок памяти.
* Потом разместить в пуле StringRef-ы на них.
*
* [key1][key2]...[keyN][ref1][ref2]...[refN]
* ^---------------------| |
* ^---------------------|
* ^---return-value----^
*
* Вернуть StringRef на кусок памяти с ключами (без учёта StringRef-ов после них).
*/
static inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous(
/// Place the specified keys into a continuous memory chunk. The implementation
/// of this function depends on whether some keys are nullable or not. See comments
/// below for the specialized implementations.
template <bool has_nullable_keys>
static StringRef extractKeysAndPlaceInPoolContiguous(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool);
/// Implementation for the case when there are no nullable keys.
/// Copy the specified keys to a continuous memory chunk of a pool.
/// Subsequently append StringRef objects referring to each key.
///
/// [key1][key2]...[keyN][ref1][ref2]...[refN]
/// ^ ^ : | |
/// +-----|--------:-----+ |
/// : +--------:-----------+
/// : :
/// <-------------->
/// (1)
///
/// Return a StringRef object, referring to the area (1) of the memory
/// chunk that contains the keys. In other words, we ignore their StringRefs.
template <>
inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous<false>(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
{
size_t sum_keys_size = 0;
@ -177,6 +296,93 @@ static inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous(
return {res, sum_keys_size};
}
/// Implementation for the case where there is at least one nullable key.
/// Inside a continuous memory chunk of a pool, put a bitmap that indicates
/// for each specified key whether its value is null or not. Copy the keys
/// whose values are not nulls to the memory chunk. Subsequently append
/// StringRef objects referring to each key, even those who contain a null.
///
/// [bitmap][key1][key2][key4]...[keyN][ref1][ref2][ref3 (null)]...[refN]
/// : ^ ^ : | |
/// : +-----|--------------:-----+ |
/// : +--------------:-----------+
/// : :
/// <---------------------------->
/// (1)
///
/// Return a StringRef object, referring to the area (1) of the memory
/// chunk that contains the bitmap and the keys. In other words, we ignore
/// the keys' StringRefs.
template <>
inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous<true>(
size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
{
size_t bitmap_size = keys_size / 8;
if ((keys_size % 8) != 0) { ++bitmap_size; }
std::vector<UInt8> bitmap(bitmap_size);
/// Prepare the keys to be stored. Create the bitmap.
size_t keys_bytes = 0;
for (size_t j = 0; j < keys_size; ++j)
{
const IColumn * observed_column;
bool is_null;
if (key_columns[j]->isNullable())
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*key_columns[j]);
observed_column = nullable_col.getNestedColumn().get();
const auto & null_map = static_cast<const ColumnUInt8 &>(*nullable_col.getNullValuesByteMap()).getData();
is_null = null_map[i] == 1;
}
else
{
observed_column = key_columns[j];
is_null = false;
}
if (is_null)
{
size_t bucket = j / 8;
size_t offset = j % 8;
bitmap[bucket] |= UInt8(1) << offset;
keys[j] = StringRef{};
}
else
{
keys[j] = observed_column->getDataAtWithTerminatingZero(i);
keys_bytes += keys[j].size;
}
}
/// Allocate space for bitmap + non-null keys + StringRef objects.
char * res = pool.alloc(bitmap_size + keys_bytes + keys_size * sizeof(StringRef));
char * place = res;
/// Store the bitmap.
memcpy(place, bitmap.data(), bitmap.size());
place += bitmap.size();
/// Store the non-null keys data.
for (size_t j = 0; j < keys_size; ++j)
{
size_t bucket = j / 8;
size_t offset = j % 8;
if (((bitmap[bucket] >> offset) & 1) == 0)
{
memcpy(place, keys[j].data, keys[j].size);
keys[j].data = place;
place += keys[j].size;
}
}
/// Store StringRef objects for all the keys, i.e. even for those
/// whose value is null.
memcpy(place, &keys[0], keys_size * sizeof(StringRef));
return {res, bitmap_size + keys_bytes};
}
/** Сериализовать ключи в непрерывный кусок памяти.
*/

View File

@ -25,6 +25,7 @@
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnAggregateFunction.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnNullable.h>
namespace DB
@ -278,9 +279,105 @@ struct AggregationMethodFixedString
}
};
namespace aggregator_impl
{
/// This class is designed to provide the functionality that is required for
/// supporting nullable keys in AggregationMethodKeysFixed. If there are
/// no nullable keys, this class is merely implemented as an empty shell.
template <typename Key, bool has_nullable_keys>
class BaseStateKeysFixed;
/// Case where nullable keys are supported.
template <typename Key>
class BaseStateKeysFixed<Key, true>
{
protected:
void init(const ConstColumnPlainPtrs & key_columns)
{
null_maps.reserve(key_columns.size());
actual_columns.reserve(key_columns.size());
for (const auto & col : key_columns)
{
if (col->isNullable())
{
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
actual_columns.push_back(nullable_col.getNestedColumn().get());
null_maps.push_back(nullable_col.getNullValuesByteMap().get());
}
else
{
actual_columns.push_back(col);
null_maps.push_back(nullptr);
}
}
}
/// Return the columns which actually contain the values of the keys.
/// For a given key column, if it is nullable, we return its nested
/// column. Otherwise we return the key column itself.
inline const ConstColumnPlainPtrs & getActualColumns() const
{
return actual_columns;
}
/// Create a bitmap that indicates whether, for a particular row,
/// a key column bears a null value or not.
KeysNullMap<Key> createBitmap(size_t row) const
{
KeysNullMap<Key> bitmap{};
for (size_t k = 0; k < null_maps.size(); ++k)
{
if (null_maps[k] != nullptr)
{
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[k]).getData();
if (null_map[row] == 1)
{
size_t bucket = k / 8;
size_t offset = k % 8;
bitmap[bucket] |= UInt8(1) << offset;
}
}
}
return bitmap;
}
private:
ConstColumnPlainPtrs actual_columns;
ConstColumnPlainPtrs null_maps;
};
/// Case where nullable keys are not supported.
template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
void init(const ConstColumnPlainPtrs & key_columns)
{
throw Exception{"Internal error: calling init() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
const ConstColumnPlainPtrs & getActualColumns() const
{
throw Exception{"Internal error: calling getActualColumns() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
KeysNullMap<Key> createBitmap(size_t row) const
{
throw Exception{"Internal error: calling createBitmap() for non-nullable keys"
" is forbidden", ErrorCodes::LOGICAL_ERROR};
}
};
}
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
template <typename TData>
template <typename TData, bool has_nullable_keys_ = false>
struct AggregationMethodKeysFixed
{
using Data = TData;
@ -288,6 +385,7 @@ struct AggregationMethodKeysFixed
using Mapped = typename Data::mapped_type;
using iterator = typename Data::iterator;
using const_iterator = typename Data::const_iterator;
static constexpr bool has_nullable_keys = has_nullable_keys_;
Data data;
@ -296,10 +394,15 @@ struct AggregationMethodKeysFixed
template <typename Other>
AggregationMethodKeysFixed(const Other & other) : data(other.data) {}
struct State
class State final : private aggregator_impl::BaseStateKeysFixed<Key, has_nullable_keys>
{
public:
using Base = aggregator_impl::BaseStateKeysFixed<Key, has_nullable_keys>;
void init(ConstColumnPlainPtrs & key_columns)
{
if (has_nullable_keys)
Base::init(key_columns);
}
Key getKey(
@ -310,7 +413,13 @@ struct AggregationMethodKeysFixed
StringRefs & keys,
Arena & pool) const
{
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
if (has_nullable_keys)
{
auto bitmap = Base::createBitmap(i);
return packFixed<Key>(i, keys_size, Base::getActualColumns(), key_sizes, bitmap);
}
else
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
};
@ -327,19 +436,57 @@ struct AggregationMethodKeysFixed
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
size_t offset = 0;
static constexpr auto bitmap_size = has_nullable_keys ? std::tuple_size<KeysNullMap<Key>>::value : 0;
/// In any hash key value, column values to be read start just after the bitmap, if it exists.
size_t pos = bitmap_size;
for (size_t i = 0; i < keys_size; ++i)
{
size_t size = key_sizes[i];
key_columns[i]->insertData(reinterpret_cast<const char *>(&value.first) + offset, size);
offset += size;
IColumn * observed_column;
ColumnUInt8 * null_map;
/// If we have a nullable column, get its nested column and its null map.
if (has_nullable_keys && key_columns[i]->isNullable())
{
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*key_columns[i]);
observed_column = nullable_col.getNestedColumn().get();
null_map = static_cast<ColumnUInt8 *>(nullable_col.getNullValuesByteMap().get());
}
else
{
observed_column = key_columns[i];
null_map = nullptr;
}
bool is_null;
if (has_nullable_keys && key_columns[i]->isNullable())
{
/// The current column is nullable. Check if the value of the
/// corresponding key is nullable. Update the null map accordingly.
size_t bucket = i / 8;
size_t offset = i % 8;
UInt8 val = (reinterpret_cast<const UInt8 *>(&value.first)[bucket] >> offset) & 1;
null_map->insert(val);
is_null = val == 1;
}
else
is_null = false;
if (has_nullable_keys && is_null)
observed_column->insertDefault();
else
{
size_t size = key_sizes[i];
observed_column->insertData(reinterpret_cast<const char *>(&value.first) + pos, size);
pos += size;
}
}
}
};
/// Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData>
template <typename TData, bool has_nullable_keys_ = false>
struct AggregationMethodConcat
{
using Data = TData;
@ -348,6 +495,8 @@ struct AggregationMethodConcat
using iterator = typename Data::iterator;
using const_iterator = typename Data::const_iterator;
static constexpr bool has_nullable_keys = has_nullable_keys_;
Data data;
AggregationMethodConcat() {}
@ -369,7 +518,7 @@ struct AggregationMethodConcat
StringRefs & keys,
Arena & pool) const
{
return extractKeysAndPlaceInPoolContiguous(i, keys_size, key_columns, keys, pool);
return extractKeysAndPlaceInPoolContiguous<has_nullable_keys>(i, keys_size, key_columns, keys, pool);
}
};
@ -389,6 +538,17 @@ struct AggregationMethodConcat
static const bool no_consecutive_keys_optimization = true;
static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
if (has_nullable_keys)
insertKeyIntoNullableColumnsImpl(value, key_columns, keys_size, key_sizes);
else
insertKeyIntoColumnsImpl(value, key_columns, keys_size, key_sizes);
}
private:
/// Insert the values of the specified keys into the corresponding columns.
/// Implementation for the case where there are no nullable keys.
static void insertKeyIntoColumnsImpl(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
/// См. функцию extractKeysAndPlaceInPoolContiguous.
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
@ -396,9 +556,9 @@ struct AggregationMethodConcat
if (unlikely(0 == value.first.size))
{
/** Исправление, если все ключи - пустые массивы. Для них в хэш-таблицу записывается StringRef нулевой длины, но с ненулевым указателем.
* Но при вставке в хэш-таблицу, такой StringRef оказывается равен другому ключу нулевой длины,
* у которого указатель на данные может быть любым мусором и использовать его нельзя.
*/
* Но при вставке в хэш-таблицу, такой StringRef оказывается равен другому ключу нулевой длины,
* у которого указатель на данные может быть любым мусором и использовать его нельзя.
*/
for (size_t i = 0; i < keys_size; ++i)
key_columns[i]->insertDefault();
}
@ -408,6 +568,92 @@ struct AggregationMethodConcat
key_columns[i]->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size);
}
}
/// Insert the value of the specified keys into the corresponding columns.
/// Implementation for the case where there is at least one nullable key.
static void insertKeyIntoNullableColumnsImpl(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes)
{
size_t compact_bitmap_size = keys_size / 8;
if ((keys_size % 8) != 0) { ++compact_bitmap_size; }
if (unlikely(value.first.size < compact_bitmap_size))
{
/// This code path is logically impossible.
/// Only a bug in the code base can trigger it.
throw Exception{"Aggregator: corrupted hash table key", ErrorCodes::LOGICAL_ERROR};
}
else if (unlikely(value.first.size == compact_bitmap_size))
{
/// This case occurs when each of the keys falls into either of the following two
/// categories: (i) it has a null value; (ii) it represents an empty array.
/// The remarks are the same as for the implementation of the non-nullable case above.
const UInt8 * compact_bitmap = reinterpret_cast<const UInt8 *>(value.first.data);
for (size_t i = 0; i < keys_size; ++i)
{
IColumn * observed_column;
if (key_columns[i]->isNullable())
{
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*key_columns[i]);
observed_column = nullable_col.getNestedColumn().get();
ColumnUInt8 & null_map = static_cast<ColumnUInt8 &>(*nullable_col.getNullValuesByteMap());
size_t bucket = i / 8;
size_t offset = i % 8;
UInt8 is_null = (compact_bitmap[bucket] >> offset) & 1;
null_map.insert(is_null);
}
else
observed_column = key_columns[i];
observed_column->insertDefault();
}
}
else
{
const UInt8 * compact_bitmap = reinterpret_cast<const UInt8 *>(value.first.data);
const StringRef * key_refs = reinterpret_cast<const StringRef *>(value.first.data + value.first.size);
for (size_t i = 0; i < keys_size; ++i)
{
IColumn * observed_column;
ColumnUInt8 * null_map;
/// If we have a nullable column, get its nested column and its null map.
if (key_columns[i]->isNullable())
{
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*key_columns[i]);
observed_column = nullable_col.getNestedColumn().get();
null_map = static_cast<ColumnUInt8 *>(nullable_col.getNullValuesByteMap().get());
}
else
{
observed_column = key_columns[i];
null_map = nullptr;
}
bool is_null;
if (key_columns[i]->isNullable())
{
/// The current column is nullable. Check if the value of the
/// corresponding key is nullable. Update the null map accordingly.
size_t bucket = i / 8;
size_t offset = i % 8;
UInt8 val = (compact_bitmap[bucket] >> offset) & 1;
null_map->insert(val);
is_null = val == 1;
}
else
is_null = false;
if (is_null)
observed_column->insertDefault();
else
observed_column->insertDataWithTerminatingZero(key_refs[i].data, key_refs[i].size);
}
}
}
};
@ -595,6 +841,14 @@ struct AggregatedDataVariants : private boost::noncopyable
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyHash64>> concat_hash64;
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyHash64>> serialized_hash64;
/// Support for nullable keys.
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128, true>> nullable_keys128;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256, true>> nullable_keys256;
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKey, true>> nullable_concat;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel, true>> nullable_keys128_two_level;
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel, true>> nullable_keys256_two_level;
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyTwoLevel, true>> nullable_concat_two_level;
/// В этом и подобных макросах, вариант without_key не учитывается.
#define APPLY_FOR_AGGREGATED_VARIANTS(M) \
M(key8, false) \
@ -624,6 +878,12 @@ struct AggregatedDataVariants : private boost::noncopyable
M(keys256_hash64, false) \
M(concat_hash64, false) \
M(serialized_hash64, false) \
M(nullable_keys128, false) \
M(nullable_keys256, false) \
M(nullable_concat, false) \
M(nullable_keys128_two_level, true) \
M(nullable_keys256_two_level, true) \
M(nullable_concat_two_level, true)
enum class Type
{
@ -741,6 +1001,9 @@ struct AggregatedDataVariants : private boost::noncopyable
M(hashed) \
M(concat) \
M(serialized) \
M(nullable_keys128) \
M(nullable_keys256) \
M(nullable_concat) \
#define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \
M(key8) \
@ -783,7 +1046,10 @@ struct AggregatedDataVariants : private boost::noncopyable
M(keys256_two_level) \
M(hashed_two_level) \
M(concat_two_level) \
M(serialized_two_level)
M(serialized_two_level) \
M(nullable_keys128_two_level) \
M(nullable_keys256_two_level) \
M(nullable_concat_two_level)
};
using AggregatedDataVariantsPtr = std::shared_ptr<AggregatedDataVariants>;

View File

@ -145,19 +145,122 @@ struct SetMethodFixedString
}
};
namespace set_impl
{
/// This class is designed to provide the functionality that is required for
/// supporting nullable keys in SetMethodKeysFixed. If there are
/// no nullable keys, this class is merely implemented as an empty shell.
template <typename Key, bool has_nullable_keys>
class BaseStateKeysFixed;
/// Case where nullable keys are supported.
template <typename Key>
class BaseStateKeysFixed<Key, true>
{
protected:
void init(const ConstColumnPlainPtrs & key_columns)
{
null_maps.reserve(key_columns.size());
actual_columns.reserve(key_columns.size());
for (const auto & col : key_columns)
{
if (col->isNullable())
{
const auto & nullable_col = static_cast<const ColumnNullable &>(*col);
actual_columns.push_back(nullable_col.getNestedColumn().get());
null_maps.push_back(nullable_col.getNullValuesByteMap().get());
}
else
{
actual_columns.push_back(col);
null_maps.push_back(nullptr);
}
}
}
/// Return the columns which actually contain the values of the keys.
/// For a given key column, if it is nullable, we return its nested
/// column. Otherwise we return the key column itself.
inline const ConstColumnPlainPtrs & getActualColumns() const
{
return actual_columns;
}
/// Create a bitmap that indicates whether, for a particular row,
/// a key column bears a null value or not.
KeysNullMap<Key> createBitmap(size_t row) const
{
KeysNullMap<Key> bitmap{};
for (size_t k = 0; k < null_maps.size(); ++k)
{
if (null_maps[k] != nullptr)
{
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[k]).getData();
if (null_map[row] == 1)
{
size_t bucket = k / 8;
size_t offset = k % 8;
bitmap[bucket] |= UInt8(1) << offset;
}
}
}
return bitmap;
}
private:
ConstColumnPlainPtrs actual_columns;
ConstColumnPlainPtrs null_maps;
};
/// Case where nullable keys are not supported.
template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
void init(const ConstColumnPlainPtrs & key_columns)
{
throw Exception{"Internal error: calling init() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
const ConstColumnPlainPtrs & getActualColumns() const
{
throw Exception{"Internal error: calling getActualColumns() for non-nullable"
" keys is forbidden", ErrorCodes::LOGICAL_ERROR};
}
KeysNullMap<Key> createBitmap(size_t row) const
{
throw Exception{"Internal error: calling createBitmap() for non-nullable keys"
" is forbidden", ErrorCodes::LOGICAL_ERROR};
}
};
}
/// Для случая, когда все ключи фиксированной длины, и они помещаются в N (например, 128) бит.
template <typename TData>
template <typename TData, bool has_nullable_keys_ = false>
struct SetMethodKeysFixed
{
using Data = TData;
using Key = typename Data::key_type;
static constexpr bool has_nullable_keys = has_nullable_keys_;
Data data;
struct State
class State : private set_impl::BaseStateKeysFixed<Key, has_nullable_keys>
{
public:
using Base = set_impl::BaseStateKeysFixed<Key, has_nullable_keys>;
void init(const ConstColumnPlainPtrs & key_columns)
{
if (has_nullable_keys)
Base::init(key_columns);
}
Key getKey(
@ -166,7 +269,13 @@ struct SetMethodKeysFixed
size_t i,
const Sizes & key_sizes) const
{
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
if (has_nullable_keys)
{
auto bitmap = Base::createBitmap(i);
return packFixed<Key>(i, keys_size, Base::getActualColumns(), key_sizes, bitmap);
}
else
return packFixed<Key>(i, keys_size, key_columns, key_sizes);
}
};
@ -222,6 +331,10 @@ struct SetVariants
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt256, UInt256HashCRC32>>> keys256;
std::unique_ptr<SetMethodHashed<HashSet<UInt128, UInt128TrivialHash>>> hashed;
/// Support for nullable keys.
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt128, UInt128HashCRC32>, true>> nullable_keys128;
std::unique_ptr<SetMethodKeysFixed<HashSet<UInt256, UInt256HashCRC32>, true>> nullable_keys256;
/** В отличие от Aggregator, здесь не используется метод concat.
* Это сделано потому что метод hashed, хоть и медленнее, но в данном случае, использует меньше оперативки.
* так как при его использовании, сами значения ключей не сохраняются.
@ -238,6 +351,8 @@ struct SetVariants
M(key_fixed_string) \
M(keys128) \
M(keys256) \
M(nullable_keys128) \
M(nullable_keys256) \
M(hashed)
enum class Type

View File

@ -15,6 +15,7 @@
#include <DB/AggregateFunctions/AggregateFunctionArray.h>
#include <DB/AggregateFunctions/AggregateFunctionState.h>
#include <DB/AggregateFunctions/AggregateFunctionMerge.h>
#include <DB/AggregateFunctions/AggregateFunctionNull.h>
namespace DB

View File

@ -222,11 +222,21 @@ protected:
};
/** Parser for nullity checking with IS (NOT) NULL.
*/
class ParserNullityChecking : public IParserBase
{
protected:
const char * getName() const override { return "nullity checking"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) override;
};
class ParserLogicalNotExpression : public IParserBase
{
private:
static const char * operators[];
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserComparisonExpression>()};
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserNullityChecking>()};
protected:
const char * getName() const { return "logical-NOT expression"; }

View File

@ -525,7 +525,7 @@ private:
* Если !part, просто проверяет, что все нужные преобразования типов допустимы.
*/
void createConvertExpression(const DataPartPtr & part, const NamesAndTypesList & old_columns, const NamesAndTypesList & new_columns,
ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata);
ExpressionActionsPtr & out_expression, NameToNameMap & out_rename_map, bool & out_force_update_metadata) const;
/// Рассчитывает размеры столбцов в сжатом виде для текущего состояния data_parts. Вызывается под блокировкой.
void calculateColumnSizes();

View File

@ -13,28 +13,30 @@ class IDataType;
class CachedCompressedReadBuffer;
class CompressedReadBufferFromFile;
class IDataType;
/** Умеет читать данные между парой засечек из одного куска. При чтении последовательных отрезков не делает лишних seek-ов.
* При чтении почти последовательных отрезков делает seek-и быстро, не выбрасывая содержимое буфера.
*/
class MergeTreeReader
class MergeTreeReader : private boost::noncopyable
{
using OffsetColumns = std::map<std::string, ColumnPtr>;
public:
using ValueSizeMap = std::map<std::string, double>;
MergeTreeReader(const String & path, /// Путь к куску
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns,
UncompressedCache * uncompressed_cache, MarkCache * mark_cache, bool save_marks_in_cache,
UncompressedCache * uncompressed_cache,
MarkCache * mark_cache,
bool save_marks_in_cache,
MergeTreeData & storage, const MarkRanges & all_mark_ranges,
size_t aio_threshold, size_t max_read_buffer_size, const ValueSizeMap & avg_value_size_hints = ValueSizeMap{},
size_t aio_threshold, size_t max_read_buffer_size,
const ValueSizeMap & avg_value_size_hints = ValueSizeMap{},
const ReadBufferFromFileBase::ProfileCallback & profile_callback = ReadBufferFromFileBase::ProfileCallback{},
clockid_t clock_type = CLOCK_MONOTONIC_COARSE);
~MergeTreeReader();
const ValueSizeMap & getAvgValueSizeHints() const { return avg_value_size_hints; }
const ValueSizeMap & getAvgValueSizeHints() const;
/** Если столбцов нет в блоке, добавляет их, если есть - добавляет прочитанные значения к ним в конец.
* Не добавляет столбцы, для которых нет файлов. Чтобы их добавить, нужно вызвать fillMissingColumns.
@ -54,30 +56,35 @@ public:
void fillMissingColumnsAndReorder(Block & res, const Names & ordered_names);
private:
struct Stream
class Stream
{
MarkCache::MappedPtr marks;
ReadBuffer * data_buffer;
std::unique_ptr<CachedCompressedReadBuffer> cached_buffer;
std::unique_ptr<CompressedReadBufferFromFile> non_cached_buffer;
std::string path_prefix;
size_t max_mark_range;
bool is_empty = false;
public:
Stream(
const String & path_prefix_, UncompressedCache * uncompressed_cache,
const String & path_prefix_, const String & extension_,
UncompressedCache * uncompressed_cache,
MarkCache * mark_cache, bool save_marks_in_cache,
const MarkRanges & all_mark_ranges, size_t aio_threshold, size_t max_read_buffer_size,
const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type);
static std::unique_ptr<Stream> createEmptyPtr();
void loadMarks(MarkCache * cache, bool save_in_cache);
void loadMarks(MarkCache * cache, bool save_in_cache, bool is_null_stream);
void seekToMark(size_t index);
bool isEmpty() const { return is_empty; }
ReadBuffer * data_buffer;
private:
Stream() = default;
MarkCache::MappedPtr marks;
std::unique_ptr<CachedCompressedReadBuffer> cached_buffer;
std::unique_ptr<CompressedReadBufferFromFile> non_cached_buffer;
std::string path_prefix;
std::string extension;
bool is_empty = false;
};
using FileStreams = std::map<std::string, std::unique_ptr<Stream>>;

View File

@ -16,7 +16,8 @@ class MarkCache;
/** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked
* to perform. */
* to perform.
*/
class MergeTreeThreadBlockInputStream : public IProfilingBlockInputStream
{
/// "thread" index (there are N threads and each thread is assigned index in interval [0..N-1])
@ -43,7 +44,6 @@ protected:
private:
/// Requests read task from MergeTreeReadPool and signals whether it got one
bool getNewTask();
Block readFromPart();
void injectVirtualColumns(Block & block);

View File

@ -1,14 +1,9 @@
#pragma once
#include <DB/IO/createWriteBufferFromFileBase.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/CompressedWriteBuffer.h>
#include <DB/IO/HashingWriteBuffer.h>
#include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <DB/Columns/ColumnArray.h>
@ -26,15 +21,7 @@ public:
size_t min_compress_block_size_,
size_t max_compress_block_size_,
CompressionMethod compression_method_,
size_t aio_threshold_)
: storage(storage_),
min_compress_block_size(min_compress_block_size_),
max_compress_block_size(max_compress_block_size_),
aio_threshold(aio_threshold_),
compression_method(compression_method_)
{
}
size_t aio_threshold_);
protected:
using OffsetColumns = std::set<std::string>;
@ -44,17 +31,17 @@ protected:
ColumnStream(
const String & escaped_column_name_,
const String & data_path,
const std::string & data_file_extension_,
const std::string & marks_path,
const std::string & marks_file_extension_,
size_t max_compress_block_size,
CompressionMethod compression_method,
size_t estimated_size,
size_t aio_threshold) :
escaped_column_name(escaped_column_name_),
plain_file(createWriteBufferFromFileBase(data_path, estimated_size, aio_threshold, max_compress_block_size)),
plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_method), compressed(compressed_buf),
marks_file(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) {}
size_t aio_threshold);
String escaped_column_name;
std::string data_file_extension;
std::string marks_file_extension;
/// compressed -> compressed_buf -> plain_hashing -> plain_file
std::unique_ptr<WriteBufferFromFileBase> plain_file;
@ -66,167 +53,21 @@ protected:
WriteBufferFromFile marks_file;
HashingWriteBuffer marks;
void finalize()
{
compressed.next();
plain_file->next();
marks.next();
}
void finalize();
void sync()
{
plain_file->sync();
marks_file.sync();
}
void sync();
void addToChecksums(MergeTreeData::DataPart::Checksums & checksums, String name = "")
{
if (name == "")
name = escaped_column_name;
checksums.files[name + ".bin"].is_compressed = true;
checksums.files[name + ".bin"].uncompressed_size = compressed.count();
checksums.files[name + ".bin"].uncompressed_hash = compressed.getHash();
checksums.files[name + ".bin"].file_size = plain_hashing.count();
checksums.files[name + ".bin"].file_hash = plain_hashing.getHash();
checksums.files[name + ".mrk"].file_size = marks.count();
checksums.files[name + ".mrk"].file_hash = marks.getHash();
}
void addToChecksums(MergeTreeData::DataPart::Checksums & checksums);
};
using ColumnStreams = std::map<String, std::unique_ptr<ColumnStream>>;
void addStream(const String & path, const String & name, const IDataType & type, size_t estimated_size = 0,
size_t level = 0, String filename = "", bool skip_offsets = false)
{
String escaped_column_name;
if (filename.size())
escaped_column_name = escapeForFileName(filename);
else
escaped_column_name = escapeForFileName(name);
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
if (!skip_offsets)
{
String size_name = DataTypeNested::extractNestedTableName(name)
+ ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
String escaped_size_name = escapeForFileName(DataTypeNested::extractNestedTableName(name))
+ ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
column_streams[size_name] = std::make_unique<ColumnStream>(
escaped_size_name,
path + escaped_size_name + ".bin",
path + escaped_size_name + ".mrk",
max_compress_block_size,
compression_method,
estimated_size,
aio_threshold);
}
addStream(path, name, *type_arr->getNestedType(), estimated_size, level + 1);
}
else
{
column_streams[name] = std::make_unique<ColumnStream>(
escaped_column_name,
path + escaped_column_name + ".bin",
path + escaped_column_name + ".mrk",
max_compress_block_size,
compression_method,
estimated_size,
aio_threshold);
}
}
void addStream(const String & path, const String & name, const IDataType & type, size_t estimated_size,
size_t level, const String & filename, bool skip_offsets);
/// Записать данные одного столбца.
void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns,
size_t level = 0, bool skip_offsets = false)
{
size_t size = column.size();
/// Для массивов требуется сначала сериализовать размеры, а потом значения.
const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type);
if (!skip_offsets && type_arr)
{
String size_name = DataTypeNested::extractNestedTableName(name)
+ ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level);
if (offset_columns.count(size_name) == 0)
{
offset_columns.insert(size_name);
ColumnStream & stream = *column_streams[size_name];
size_t prev_mark = 0;
while (prev_mark < size)
{
size_t limit = 0;
/// Если есть index_offset, то первая засечка идёт не сразу, а после этого количества строк.
if (prev_mark == 0 && index_offset != 0)
{
limit = index_offset;
}
else
{
limit = storage.index_granularity;
/// Уже могло накопиться достаточно данных для сжатия в новый блок.
if (stream.compressed.offset() >= min_compress_block_size)
stream.compressed.next();
writeIntBinary(stream.plain_hashing.count(), stream.marks);
writeIntBinary(stream.compressed.offset(), stream.marks);
}
type_arr->serializeOffsets(column, stream.compressed, prev_mark, limit);
/// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего.
stream.compressed.nextIfAtEnd();
prev_mark += limit;
}
}
}
{
ColumnStream & stream = *column_streams[name];
size_t prev_mark = 0;
while (prev_mark < size)
{
size_t limit = 0;
/// Если есть index_offset, то первая засечка идёт не сразу, а после этого количества строк.
if (prev_mark == 0 && index_offset != 0)
{
limit = index_offset;
}
else
{
limit = storage.index_granularity;
/// Уже могло накопиться достаточно данных для сжатия в новый блок.
if (stream.compressed.offset() >= min_compress_block_size)
stream.compressed.next();
writeIntBinary(stream.plain_hashing.count(), stream.marks);
writeIntBinary(stream.compressed.offset(), stream.marks);
}
type.serializeBinary(column, stream.compressed, prev_mark, limit);
/// Чтобы вместо засечек, указывающих на конец сжатого блока, были засечки, указывающие на начало следующего.
stream.compressed.nextIfAtEnd();
prev_mark += limit;
}
}
}
size_t level, bool skip_offsets);
MergeTreeData & storage;
@ -241,6 +82,11 @@ protected:
size_t aio_threshold;
CompressionMethod compression_method;
private:
/// Internal version of writeData.
void writeDataImpl(const String & name, const IDataType & type, const IColumn & column,
OffsetColumns & offset_columns, size_t level, bool write_array_data, bool skip_offsets);
};
@ -254,17 +100,7 @@ public:
MergeTreeData & storage_,
String part_path_,
const NamesAndTypesList & columns_list_,
CompressionMethod compression_method)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method,
storage_.context.getSettings().min_bytes_to_use_direct_io),
columns_list(columns_list_), part_path(part_path_)
{
init();
for (const auto & it : columns_list)
addStream(part_path, it.name, *it.type);
}
CompressionMethod compression_method);
MergedBlockOutputStream(
MergeTreeData & storage_,
@ -272,226 +108,38 @@ public:
const NamesAndTypesList & columns_list_,
CompressionMethod compression_method,
const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_,
size_t aio_threshold_)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method,
aio_threshold_),
columns_list(columns_list_), part_path(part_path_)
{
init();
for (const auto & it : columns_list)
{
size_t estimated_size = 0;
if (aio_threshold > 0)
{
auto it2 = merged_column_to_size_.find(it.name);
if (it2 != merged_column_to_size_.end())
estimated_size = it2->second;
}
addStream(part_path, it.name, *it.type, estimated_size);
}
}
size_t aio_threshold_);
std::string getPartPath() const
{
return part_path;
}
std::string getPartPath() const;
/// Если данные заранее отсортированы.
void write(const Block & block) override
{
writeImpl(block, nullptr);
}
void write(const Block & block) override;
/** Если данные не отсортированы, но мы заранее вычислили перестановку, после которой они станут сортированными.
* Этот метод используется для экономии оперативки, так как не нужно держать одновременно два блока - исходный и отсортированный.
*/
void writeWithPermutation(const Block & block, const IColumn::Permutation * permutation)
{
writeImpl(block, permutation);
}
void writeWithPermutation(const Block & block, const IColumn::Permutation * permutation);
void writeSuffix() override
{
throw Exception("Method writeSuffix is not supported by MergedBlockOutputStream", ErrorCodes::NOT_IMPLEMENTED);
}
void writeSuffix() override;
MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums(
const NamesAndTypesList & total_column_list,
MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr)
{
/// Заканчиваем запись и достаем чексуммы.
MergeTreeData::DataPart::Checksums checksums;
MergeTreeData::DataPart::Checksums * additional_column_checksums = nullptr);
if (additional_column_checksums)
checksums = std::move(*additional_column_checksums);
MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums();
if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted)
{
index_stream->next();
checksums.files["primary.idx"].file_size = index_stream->count();
checksums.files["primary.idx"].file_hash = index_stream->getHash();
index_stream = nullptr;
}
for (auto & column_stream : column_streams)
{
column_stream.second->finalize();
column_stream.second->addToChecksums(checksums);
}
column_streams.clear();
if (marks_count == 0)
{
/// Кусок пустой - все записи удалились.
Poco::File(part_path).remove(true);
checksums.files.clear();
return checksums;
}
{
/// Записываем файл с описанием столбцов.
WriteBufferFromFile out(part_path + "columns.txt", 4096);
total_column_list.writeText(out);
}
{
/// Записываем файл с чексуммами.
WriteBufferFromFile out(part_path + "checksums.txt", 4096);
checksums.write(out);
}
return checksums;
}
MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums()
{
return writeSuffixAndGetChecksums(columns_list, nullptr);
}
MergeTreeData::DataPart::Index & getIndex()
{
return index_columns;
}
MergeTreeData::DataPart::Index & getIndex();
/// Сколько засечек уже записано.
size_t marksCount()
{
return marks_count;
}
size_t marksCount();
private:
void init()
{
Poco::File(part_path).createDirectories();
if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted)
{
index_file_stream = std::make_unique<WriteBufferFromFile>(
part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
index_stream = std::make_unique<HashingWriteBuffer>(*index_file_stream);
}
}
void init();
/** Если задана permutation, то переставляет значения в столбцах при записи.
* Это нужно, чтобы не держать целый блок в оперативке для его сортировки.
*/
void writeImpl(const Block & block, const IColumn::Permutation * permutation)
{
size_t rows = block.rows();
/// Множество записанных столбцов со смещениями, чтобы не писать общие для вложенных структур столбцы несколько раз
OffsetColumns offset_columns;
auto sort_description = storage.getSortDescription();
/// Сюда будем складывать столбцы, относящиеся к Primary Key, чтобы потом записать индекс.
std::vector<ColumnWithTypeAndName> primary_columns(sort_description.size());
std::map<String, size_t> primary_columns_name_to_position;
for (size_t i = 0, size = sort_description.size(); i < size; ++i)
{
const auto & descr = sort_description[i];
String name = !descr.column_name.empty()
? descr.column_name
: block.getByPosition(descr.column_number).name;
if (!primary_columns_name_to_position.emplace(name, i).second)
throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS);
primary_columns[i] = !descr.column_name.empty()
? block.getByName(descr.column_name)
: block.getByPosition(descr.column_number);
/// Столбцы первичного ключа переупорядочиваем заранее и складываем в primary_columns.
if (permutation)
primary_columns[i].column = primary_columns[i].column->permute(*permutation, 0);
}
if (index_columns.empty())
{
index_columns.resize(sort_description.size());
for (size_t i = 0, size = sort_description.size(); i < size; ++i)
index_columns[i] = primary_columns[i].column.get()->cloneEmpty();
}
/// Теперь пишем данные.
for (const auto & it : columns_list)
{
const ColumnWithTypeAndName & column = block.getByName(it.name);
if (permutation)
{
auto primary_column_it = primary_columns_name_to_position.find(it.name);
if (primary_columns_name_to_position.end() != primary_column_it)
{
writeData(column.name, *column.type, *primary_columns[primary_column_it->second].column, offset_columns);
}
else
{
/// Столбцы, не входящие в первичный ключ, переупорядочиваем здесь; затем результат освобождается - для экономии оперативки.
ColumnPtr permutted_column = column.column->permute(*permutation, 0);
writeData(column.name, *column.type, *permutted_column, offset_columns);
}
}
else
{
writeData(column.name, *column.type, *column.column, offset_columns);
}
}
{
/** While filling index (index_columns), disable memory tracker.
* Because memory is allocated here (maybe in context of INSERT query),
* but then freed in completely different place (while merging parts), where query memory_tracker is not available.
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker;
/// Пишем индекс. Индекс содержит значение Primary Key для каждой index_granularity строки.
for (size_t i = index_offset; i < rows; i += storage.index_granularity)
{
if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted)
{
for (size_t j = 0, size = primary_columns.size(); j < size; ++j)
{
const IColumn & primary_column = *primary_columns[j].column.get();
index_columns[j].get()->insertFrom(primary_column, i);
primary_columns[j].type.get()->serializeBinary(primary_column, i, *index_stream);
}
}
++marks_count;
}
}
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity;
}
void writeImpl(const Block & block, const IColumn::Permutation * permutation);
private:
NamesAndTypesList columns_list;
@ -509,64 +157,12 @@ private:
class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream
{
public:
MergedColumnOnlyOutputStream(MergeTreeData & storage_, String part_path_, bool sync_, CompressionMethod compression_method,
bool skip_offsets_ = false)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method,
storage_.context.getSettings().min_bytes_to_use_direct_io),
part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_)
{
}
MergedColumnOnlyOutputStream(
MergeTreeData & storage_, String part_path_, bool sync_, CompressionMethod compression_method, bool skip_offsets_);
void write(const Block & block) override
{
if (!initialized)
{
column_streams.clear();
for (size_t i = 0; i < block.columns(); ++i)
{
addStream(part_path, block.getByPosition(i).name,
*block.getByPosition(i).type, 0, 0, block.getByPosition(i).name, skip_offsets);
}
initialized = true;
}
size_t rows = block.rows();
OffsetColumns offset_columns;
for (size_t i = 0; i < block.columns(); ++i)
{
const ColumnWithTypeAndName & column = block.getByPosition(i);
writeData(column.name, *column.type, *column.column, offset_columns, 0, skip_offsets);
}
size_t written_for_last_mark = (storage.index_granularity - index_offset + rows) % storage.index_granularity;
index_offset = (storage.index_granularity - written_for_last_mark) % storage.index_granularity;
}
void writeSuffix() override
{
throw Exception("Method writeSuffix is not supported by MergedColumnOnlyOutputStream", ErrorCodes::NOT_IMPLEMENTED);
}
MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums()
{
MergeTreeData::DataPart::Checksums checksums;
for (auto & column_stream : column_streams)
{
column_stream.second->finalize();
if (sync)
column_stream.second->sync();
column_stream.second->addToChecksums(checksums);
}
column_streams.clear();
initialized = false;
return checksums;
}
void write(const Block & block) override;
void writeSuffix() override;
MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums();
private:
String part_path;

View File

@ -9,6 +9,7 @@
#include <DB/Storages/IStorage.h>
#include <DB/Common/FileChecker.h>
#include <DB/Common/escapeForFileName.h>
namespace DB
@ -122,6 +123,7 @@ protected:
BlockInputStreams read(
size_t from_mark,
size_t to_mark,
size_t from_null_mark,
const Names & column_names,
ASTPtr query,
const Context & context,
@ -134,15 +136,22 @@ private:
Files_t files; /// name -> data
Names column_names; /// column_index -> name
Names null_map_filenames;
Poco::File marks_file;
Poco::File null_marks_file;
void loadMarksImpl(bool load_null_marks);
/// Порядок добавления файлов не должен меняться: он соответствует порядку столбцов в файле с засечками.
void addFile(const String & column_name, const IDataType & type, size_t level = 0);
bool loaded_marks;
bool has_nullable_columns = false;
size_t max_compress_block_size;
size_t file_count = 0;
size_t null_file_count = 0;
protected:
FileChecker file_checker;
@ -156,6 +165,8 @@ private:
* Вернуть первую попавшуюся группу засечек, в которых указано количество строчек, а не внутренностей массивов.
*/
const Marks & getMarksWithRealRowCount() const;
std::string getFullPath() const { return path + escapeForFileName(name) + '/';}
};
}

View File

@ -3,6 +3,7 @@
#include <DB/IO/WriteHelpers.h>
#include <DB/DataTypes/DataTypeAggregateFunction.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/Common/StringUtils.h>
#include <Poco/String.h>
@ -51,6 +52,7 @@ AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested)
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionNull(AggregateFunctionPtr & nested);
AggregateFunctionFactory::AggregateFunctionFactory()
@ -94,6 +96,45 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
{
bool has_nullable_types = false;
for (const auto & arg_type : argument_types)
{
if (arg_type->isNullable())
{
has_nullable_types = true;
break;
}
}
if (has_nullable_types)
{
DataTypes new_argument_types;
new_argument_types.reserve(argument_types.size());
for (const auto & arg_type : argument_types)
{
if (arg_type->isNullable())
{
const DataTypeNullable & actual_type = static_cast<const DataTypeNullable &>(*arg_type.get());
const DataTypePtr & nested_type = actual_type.getNestedType();
new_argument_types.push_back(nested_type);
}
else
new_argument_types.push_back(arg_type);
}
AggregateFunctionPtr function = getImpl(name, new_argument_types, recursion_level);
return createAggregateFunctionNull(function);
}
else
return getImpl(name, argument_types, recursion_level);
}
AggregateFunctionPtr AggregateFunctionFactory::getImpl(const String & name, const DataTypes & argument_types, int recursion_level) const
{
auto it = aggregate_functions.find(name);
if (it != aggregate_functions.end())
{
auto it = aggregate_functions.find(name);
if (it != aggregate_functions.end())

View File

@ -0,0 +1,11 @@
#include <DB/AggregateFunctions/AggregateFunctionNull.h>
namespace DB
{
AggregateFunctionPtr createAggregateFunctionNull(AggregateFunctionPtr & nested)
{
return std::make_shared<AggregateFunctionNull>(nested);
}
}

View File

@ -7,6 +7,7 @@
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnTuple.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <ext/enumerate.hpp>
@ -14,6 +15,14 @@
namespace DB
{
template <>
ColumnPtr ColumnConst<Null>::convertToFullColumn() const
{
/// We basically create a column whose rows have NULL values.
ColumnPtr full_col = std::make_shared<ColumnUInt8>(size(), 0);
ColumnPtr null_map = std::make_shared<ColumnUInt8>(size(), 1);
return std::make_shared<ColumnNullable>(full_col, null_map);
}
template <> ColumnPtr ColumnConst<String>::convertToFullColumn() const
{
@ -74,7 +83,18 @@ ColumnPtr ColumnConst<Array>::convertToFullColumn() const
const Array & array = getDataFromHolderImpl();
size_t array_size = array.size();
ColumnPtr nested_column = type->getNestedType()->createColumn();
const auto & nested_type = type->getNestedType();
ColumnPtr nested_column;
if (nested_type->isNull())
{
/// Special case: an array of Null is actually an array of Nullable(UInt8).
nested_column = std::make_shared<ColumnNullable>(
std::make_shared<ColumnUInt8>(), std::make_shared<ColumnUInt8>());
}
else
nested_column = type->getNestedType()->createColumn();
auto res = std::make_shared<ColumnArray>(nested_column);
ColumnArray::Offsets_t & offsets = res->getOffsets();

View File

@ -0,0 +1,353 @@
#include <DB/Columns/ColumnNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ColumnNullable::ColumnNullable(ColumnPtr nested_column_, ColumnPtr null_map_)
: nested_column{nested_column_}, null_map{null_map_}
{
if (nested_column->isNullable())
throw Exception{"A nullable column cannot contain another nullable column", ErrorCodes::LOGICAL_ERROR};
}
ColumnPtr ColumnNullable::convertToFullColumnIfConst() const
{
ColumnPtr new_col_holder;
if (auto full_col = nested_column->convertToFullColumnIfConst())
new_col_holder = std::make_shared<ColumnNullable>(full_col, null_map);
return new_col_holder;
}
void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const
{
const auto & arr = getNullMapContent().getData();
hash.update(reinterpret_cast<const char *>(&arr[n]), sizeof(arr[0]));
if (arr[n] == 0)
nested_column->updateHashWithValue(n, hash);
}
ColumnPtr ColumnNullable::cloneResized(size_t size) const
{
ColumnPtr new_nested_col = nested_column->cloneResized(size);
ColumnPtr new_null_map = getNullMapContent().cloneResized(size);
return std::make_shared<ColumnNullable>(new_nested_col, new_null_map);
}
Field ColumnNullable::operator[](size_t n) const
{
if (isNullAt(n))
return Field{};
else
{
const IColumn & col = *nested_column;
return col[n];
}
}
void ColumnNullable::get(size_t n, Field & res) const
{
if (isNullAt(n))
res = Field{};
else
nested_column->get(n, res);
}
StringRef ColumnNullable::getDataAt(size_t n) const
{
throw Exception{"Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED};
}
void ColumnNullable::insertData(const char * pos, size_t length)
{
throw Exception{"Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED};
}
StringRef ColumnNullable::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
{
const auto & arr = getNullMapContent().getData();
static constexpr auto s = sizeof(arr[0]);
auto pos = arena.allocContinue(s, begin);
memcpy(pos, &arr[n], s);
size_t nested_size = 0;
if (arr[n] == 0)
nested_size = nested_column->serializeValueIntoArena(n, arena, begin).size;
return StringRef{begin, s + nested_size};
}
const char * ColumnNullable::deserializeAndInsertFromArena(const char * pos)
{
UInt8 val = *reinterpret_cast<const UInt8 *>(pos);
pos += sizeof(val);
getNullMapContent().insert(val);
if (val == 0)
pos = nested_column->deserializeAndInsertFromArena(pos);
else
nested_column->insertDefault();
return pos;
}
void ColumnNullable::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(src);
getNullMapContent().insertRangeFrom(*nullable_col.null_map, start, length);
nested_column->insertRangeFrom(*nullable_col.nested_column, start, length);
}
void ColumnNullable::insert(const Field & x)
{
if (x.isNull())
{
nested_column->insertDefault();
getNullMapContent().insert(1);
}
else
{
nested_column->insert(x);
getNullMapContent().insert(0);
}
}
void ColumnNullable::insertDefault()
{
nested_column->insertDefault();
getNullMapContent().insert(0);
}
void ColumnNullable::popBack(size_t n)
{
nested_column->popBack(n);
getNullMapContent().popBack(n);
}
ColumnPtr ColumnNullable::filter(const Filter & filt, ssize_t result_size_hint) const
{
ColumnPtr filtered_data = nested_column->filter(filt, result_size_hint);
ColumnPtr filtered_null_map = getNullMapContent().filter(filt, result_size_hint);
return std::make_shared<ColumnNullable>(filtered_data, filtered_null_map);
}
ColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const
{
ColumnPtr permuted_data = nested_column->permute(perm, limit);
ColumnPtr permuted_null_map = getNullMapContent().permute(perm, limit);
return std::make_shared<ColumnNullable>(permuted_data, permuted_null_map);
}
int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const
{
/// NULL values share the properties of NaN values.
/// Here the last parameter of compareAt is called null_direction_hint
/// instead of the usual nan_direction_hint and is used to implement
/// the ordering specified by either NULLS FIRST or NULLS LAST in the
/// ORDER BY construction.
const ColumnNullable & nullable_rhs = static_cast<const ColumnNullable &>(rhs_);
bool lval_is_null = isNullAt(n);
bool rval_is_null = nullable_rhs.isNullAt(m);
if (unlikely(lval_is_null || rval_is_null))
{
if (lval_is_null && rval_is_null)
return 0;
else
return lval_is_null ? null_direction_hint : -null_direction_hint;
}
const IColumn & nested_rhs = *(nullable_rhs.getNestedColumn());
return nested_column->compareAt(n, m, nested_rhs, null_direction_hint);
}
void ColumnNullable::getPermutation(bool reverse, size_t limit, Permutation & res) const
{
nested_column->getPermutation(reverse, limit, res);
size_t s = res.size();
/// Since we have created a permutation "res" that sorts a subset of the column values
/// and some of these values may actually be nulls, there is no guarantee that
/// these null values are well positioned. So we create a permutation "p" which
/// operates on the result of "res" by moving all the null values to the required
/// direction and leaving the order of the remaining elements unchanged.
/// Create the permutation p.
Permutation p;
p.resize(s);
size_t pos_left = 0;
size_t pos_right = s - 1;
if (reverse)
{
/// Move the null elements to the right.
for (size_t i = 0; i < s; ++i)
{
if (isNullAt(res[i]))
{
p[i] = pos_right;
--pos_right;
}
else
{
p[i] = pos_left;
++pos_left;
}
}
}
else
{
/// Move the null elements to the left.
for (size_t i = 0; i < s; ++i)
{
size_t j = s - i - 1;
if (isNullAt(res[j]))
{
p[j] = pos_left;
++pos_left;
}
else
{
p[j] = pos_right;
--pos_right;
}
}
}
/// Combine the permutations res and p.
Permutation res2;
res2.resize(s);
for (size_t i = 0; i < s; ++i)
res2[i] = res[p[i]];
res = std::move(res2);
}
void ColumnNullable::reserve(size_t n)
{
nested_column->reserve(n);
getNullMapContent().reserve(n);
}
size_t ColumnNullable::byteSize() const
{
return nested_column->byteSize() + getNullMapContent().byteSize();
}
namespace
{
/// The following function implements a slightly more general version
/// of getExtremes() than the implementation from ColumnVector.
/// It takes into account the possible presence of nullable values.
template <typename T>
void getExtremesFromNullableContent(const ColumnVector<T> & col, const NullValuesByteMap & null_map, Field & min, Field & max)
{
const auto & data = col.getData();
size_t size = data.size();
if (size == 0)
{
min = typename NearestFieldType<T>::Type(0);
max = typename NearestFieldType<T>::Type(0);
return;
}
size_t min_i = 0;
for (; min_i < size; ++min_i)
{
if (null_map[min_i] == 0)
break;
}
if (min_i == size)
{
min = Field{};
max = Field{};
return;
}
T cur_min = data[min_i];
T cur_max = data[min_i];
for (size_t i = min_i + 1; i < size; ++i)
{
if (null_map[i] != 0)
continue;
if (data[i] < cur_min)
cur_min = data[i];
if (data[i] > cur_max)
cur_max = data[i];
}
min = typename NearestFieldType<T>::Type(cur_min);
max = typename NearestFieldType<T>::Type(cur_max);
}
}
void ColumnNullable::getExtremes(Field & min, Field & max) const
{
if (const auto col = typeid_cast<const ColumnInt8 *>(nested_column.get()))
getExtremesFromNullableContent<Int8>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnInt16 *>(nested_column.get()))
getExtremesFromNullableContent<Int16>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnInt32 *>(nested_column.get()))
getExtremesFromNullableContent<Int32>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnInt64 *>(nested_column.get()))
getExtremesFromNullableContent<Int64>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnUInt8 *>(nested_column.get()))
getExtremesFromNullableContent<UInt8>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnUInt16 *>(nested_column.get()))
getExtremesFromNullableContent<UInt16>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnUInt32 *>(nested_column.get()))
getExtremesFromNullableContent<UInt32>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnUInt64 *>(nested_column.get()))
getExtremesFromNullableContent<UInt64>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnFloat32 *>(nested_column.get()))
getExtremesFromNullableContent<Float32>(*col, getNullMapContent().getData(), min, max);
else if (const auto col = typeid_cast<const ColumnFloat64 *>(nested_column.get()))
getExtremesFromNullableContent<Float64>(*col, getNullMapContent().getData(), min, max);
else
nested_column->getExtremes(min, max);
}
ColumnPtr ColumnNullable::replicate(const Offsets_t & offsets) const
{
ColumnPtr replicated_data = nested_column->replicate(offsets);
ColumnPtr replicated_null_map = getNullMapContent().replicate(offsets);
return std::make_shared<ColumnNullable>(replicated_data, replicated_null_map);
}
void ColumnNullable::applyNullValuesByteMap(const ColumnNullable & other)
{
NullValuesByteMap & arr1 = getNullMapContent().getData();
const NullValuesByteMap & arr2 = other.getNullMapContent().getData();
if (arr1.size() != arr2.size())
throw Exception{"Inconsistent sizes", ErrorCodes::LOGICAL_ERROR};
for (size_t i = 0; i < arr1.size(); ++i)
arr1[i] |= arr2[i];
}
}

View File

@ -7,8 +7,10 @@
#include <DB/Storages/ColumnDefault.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <memory>
@ -372,7 +374,16 @@ void Block::checkNestedArraysOffsets() const
for (const auto & elem : data)
{
if (const ColumnArray * column_array = typeid_cast<const ColumnArray *>(&*elem.column))
const IColumn * observed_col;
if (elem.column->isNullable())
{
const auto & nullable_col = static_cast<const ColumnNullable &>(*elem.column);
observed_col = nullable_col.getNestedColumn().get();
}
else
observed_col = elem.column.get();
if (const ColumnArray * column_array = typeid_cast<const ColumnArray *>(observed_col))
{
String name = DataTypeNested::extractNestedTableName(elem.name);
@ -397,7 +408,16 @@ void Block::optimizeNestedArraysOffsets()
for (auto & elem : data)
{
if (ColumnArray * column_array = typeid_cast<ColumnArray *>(&*elem.column))
IColumn * observed_col;
if (elem.column->isNullable())
{
auto & nullable_col = static_cast<ColumnNullable &>(*elem.column);
observed_col = nullable_col.getNestedColumn().get();
}
else
observed_col = elem.column.get();
if (ColumnArray * column_array = typeid_cast<ColumnArray *>(observed_col))
{
String name = DataTypeNested::extractNestedTableName(elem.name);

View File

@ -350,6 +350,8 @@ namespace ErrorCodes
extern const int TABLE_DIFFERS_TOO_MUCH = 345;
extern const int CANNOT_ICONV = 346;
extern const int CANNOT_LOAD_CONFIG = 347;
extern const int RESHARDING_NULLABLE_SHARDING_KEY = 348;
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN = 349;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -1,4 +1,5 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/Columns/ColumnsCommon.h>
#include <DB/Interpreters/ExpressionActions.h>
@ -77,6 +78,12 @@ Block FilterBlockInputStream::readImpl()
if (column)
{
if (column->isNullable())
{
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*column);
column = nullable_col.getNestedColumn();
}
const ColumnConstUInt8 * column_const = typeid_cast<const ColumnConstUInt8 *>(&*column);
if (column_const)
@ -110,8 +117,22 @@ Block FilterBlockInputStream::readImpl()
size_t columns = res.columns();
ColumnPtr column = res.getByPosition(filter_column).column;
bool is_nullable_column = column->isNullable();
const ColumnUInt8 * column_vec = typeid_cast<const ColumnUInt8 *>(&*column);
auto init_observed_column = [&column, &is_nullable_column]()
{
if (is_nullable_column)
{
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*column.get());
return nullable_col.getNestedColumn().get();
}
else
return column.get();
};
IColumn * observed_column = init_observed_column();
const ColumnUInt8 * column_vec = typeid_cast<const ColumnUInt8 *>(observed_column);
if (!column_vec)
{
/** Бывает, что на этапе анализа выражений (в sample_block) столбцы-константы ещё не вычислены,
@ -119,7 +140,7 @@ Block FilterBlockInputStream::readImpl()
* Это происходит, если функция возвращает константу для неконстантного аргумента.
* Например, функция ignore.
*/
const ColumnConstUInt8 * column_const = typeid_cast<const ColumnConstUInt8 *>(&*column);
const ColumnConstUInt8 * column_const = typeid_cast<const ColumnConstUInt8 *>(observed_column);
if (column_const)
{
@ -139,6 +160,28 @@ Block FilterBlockInputStream::readImpl()
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
}
if (is_nullable_column)
{
/// Exclude the entries of the filter column that actually are NULL values.
/// Access the filter content.
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*column);
auto & nested_col = nullable_col.getNestedColumn();
auto & actual_col = static_cast<ColumnUInt8 &>(*nested_col);
auto & filter_col = actual_col.getData();
/// Access the null values byte map content.
ColumnPtr & null_map = nullable_col.getNullValuesByteMap();
ColumnUInt8 & content = static_cast<ColumnUInt8 &>(*null_map);
auto & data = content.getData();
for (size_t i = 0; i < data.size(); ++i)
{
if (data[i] != 0)
filter_col[i] = 0;
}
}
const IColumn::Filter & filter = column_vec->getData();
/** Выясним, сколько строк будет в результате.

View File

@ -0,0 +1,55 @@
#include <DB/DataStreams/MaterializingBlockInputStream.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
MaterializingBlockInputStream::MaterializingBlockInputStream(BlockInputStreamPtr input_)
{
children.push_back(input_);
}
String MaterializingBlockInputStream::getName() const
{
return "Materializing";
}
String MaterializingBlockInputStream::getID() const
{
std::stringstream res;
res << "Materializing(" << children.back()->getID() << ")";
return res.str();
}
Block MaterializingBlockInputStream::readImpl()
{
Block res = children.back()->read();
if (!res)
return res;
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
{
auto & element = res.getByPosition(i);
auto & src = element.column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
{
src = converted;
auto & type = element.type;
if (type->isNull())
{
/// A ColumnNull that is converted to a full column
/// has the type DataTypeNullable(DataTypeUInt8).
type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>());
}
}
}
return res;
}
}

View File

@ -5,7 +5,11 @@
#include <DB/IO/CompressedReadBufferFromFile.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/DataStreams/NativeBlockInputStream.h>
@ -21,7 +25,6 @@ namespace ErrorCodes
extern const int CANNOT_READ_ALL_DATA;
}
NativeBlockInputStream::NativeBlockInputStream(
ReadBuffer & istr_, UInt64 server_revision_,
bool use_index_,
@ -43,10 +46,25 @@ NativeBlockInputStream::NativeBlockInputStream(
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows)
{
/** Для массивов требуется сначала десериализовать смещения, а потом значения.
*/
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
if (type.isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & nested_type = *nullable_type.getNestedType();
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(column);
IColumn & nested_col = *nullable_col.getNestedColumn();
IColumn & null_map = *nullable_col.getNullValuesByteMap();
DataTypeUInt8{}.deserializeBinary(null_map, istr, rows, 0);
readData(nested_type, nested_col, istr, rows);
return;
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/** Для массивов требуется сначала десериализовать смещения, а потом значения.
*/
IColumn & offsets_column = *typeid_cast<ColumnArray &>(column).getOffsetsColumn();
type_arr->getOffsetsType()->deserializeBinary(offsets_column, istr, rows, 0);

View File

@ -6,8 +6,12 @@
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/MarkInCompressedFile.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
@ -16,7 +20,6 @@
namespace DB
{
NativeBlockOutputStream::NativeBlockOutputStream(
WriteBuffer & ostr_, UInt64 client_revision_,
WriteBuffer * index_ostr_, size_t initial_size_of_file_)
@ -50,10 +53,23 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
else
full_column = column;
/** Для массивов требуется сначала сериализовать смещения, а потом значения.
*/
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
if (type.isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(type);
const IDataType & nested_type = *nullable_type.getNestedType();
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*full_column.get());
const ColumnPtr & nested_col = nullable_col.getNestedColumn();
const IColumn & null_map = *nullable_col.getNullValuesByteMap();
DataTypeUInt8{}.serializeBinary(null_map, ostr, offset, limit);
writeData(nested_type, nested_col, ostr, offset, limit);
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/** Для массивов требуется сначала сериализовать смещения, а потом значения.
*/
const ColumnArray & column_array = typeid_cast<const ColumnArray &>(*full_column);
type_arr->getOffsetsType()->serializeBinary(*column_array.getOffsetsColumn(), ostr, offset, limit);
@ -79,8 +95,19 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
? offsets[end - 1] - nested_offset
: 0;
const DataTypePtr & nested_type = type_arr->getNestedType();
DataTypePtr actual_type;
if (nested_type->isNull())
{
/// Special case: an array of Null is actually an array of Nullable(UInt8).
actual_type = std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>());
}
else
actual_type = nested_type;
if (limit == 0 || nested_limit)
writeData(*type_arr->getNestedType(), typeid_cast<const ColumnArray &>(*full_column).getDataPtr(), ostr, nested_offset, nested_limit);
writeData(*actual_type, typeid_cast<const ColumnArray &>(*full_column).getDataPtr(), ostr, nested_offset, nested_limit);
}
}
else

View File

@ -0,0 +1,155 @@
#include <DB/DataStreams/NullableAdapterBlockInputStream.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/DataTypes/DataTypeNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int LOGICAL_ERROR;
}
NullableAdapterBlockInputStream::NullableAdapterBlockInputStream(
BlockInputStreamPtr input_,
const Block & in_sample_, const Block & out_sample_,
const NamesAndTypesListPtr & required_columns_)
: required_columns{required_columns_},
actions{getActions(in_sample_, out_sample_)},
must_transform{mustTransform()}
{
children.push_back(input_);
}
String NullableAdapterBlockInputStream::getID() const
{
std::stringstream res;
res << "NullableAdapterBlockInputStream(" << children.back()->getID() << ")";
return res.str();
}
Block NullableAdapterBlockInputStream::readImpl()
{
Block block = children.back()->read();
if (!block || !must_transform)
return block;
Block res;
size_t s = block.columns();
for (size_t i = 0; i < s; ++i)
{
const auto & elem = block.unsafeGetByPosition(i);
ColumnWithTypeAndName new_elem;
if (actions[i] == TO_ORDINARY)
{
const auto & nullable_col = static_cast<const ColumnNullable &>(*elem.column);
const auto & nullable_type = static_cast<const DataTypeNullable &>(*elem.type);
const auto & null_map = static_cast<const ColumnUInt8 &>(*nullable_col.getNullValuesByteMap()).getData();
bool has_nulls = std::any_of(null_map.begin(), null_map.end(), [](UInt8 val){ return val == 1; });
if (has_nulls)
throw Exception{"Cannot insert NULL value into non-nullable column",
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
else
res.insert({
nullable_col.getNestedColumn(),
nullable_type.getNestedType(),
elem.name
});
}
else if (actions[i] == TO_NULLABLE)
{
auto null_map = std::make_shared<ColumnUInt8>(elem.column->size(), 0);
res.insert({
std::make_shared<ColumnNullable>(elem.column, null_map),
std::make_shared<DataTypeNullable>(elem.type),
elem.name
});
}
else if (actions[i] == NONE)
res.insert(elem);
else
throw Exception{"NullableAdapterBlockInputStream: internal error", ErrorCodes::LOGICAL_ERROR};
}
return res;
}
bool NullableAdapterBlockInputStream::mustTransform() const
{
return !std::all_of(actions.begin(), actions.end(), [](Action action) { return action == NONE; });
}
NullableAdapterBlockInputStream::Actions NullableAdapterBlockInputStream::getActions(
const Block & in_sample, const Block & out_sample) const
{
size_t in_size = in_sample.columns();
size_t out_size = out_sample.columns();
Actions actions;
actions.reserve(in_size);
size_t j = 0;
for (size_t i = 0; i < in_size; ++i)
{
const auto & in_elem = in_sample.unsafeGetByPosition(i);
while (j < out_size)
{
const auto & out_elem = out_sample.unsafeGetByPosition(j);
if (in_elem.name == out_elem.name)
{
bool is_in_nullable = in_elem.type->isNullable();
bool is_out_nullable = out_elem.type->isNullable();
if (is_in_nullable && !is_out_nullable)
actions.push_back(TO_ORDINARY);
else if (!is_in_nullable && is_out_nullable)
actions.push_back(TO_NULLABLE);
else
actions.push_back(NONE);
++j;
break;
}
else
{
++j;
if (j == out_size)
{
auto print_columns = [](const NamesAndTypesList & columns)
{
bool is_first = true;
std::ostringstream ostr;
for (const auto & it : columns)
{
if (is_first)
is_first = false;
else
ostr << ", ";
ostr << it.name;
}
return ostr.str();
};
throw Exception{"There is no column with name " + in_elem.name
+ ". There are columns: "
+ print_columns(*required_columns),
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE};
}
}
}
}
return actions;
}
}

View File

@ -2,7 +2,6 @@
#include <unistd.h>
#include <DB/Functions/FunctionsMiscellaneous.h>
#include <DB/DataStreams/PrettyBlockOutputStream.h>
@ -45,19 +44,17 @@ void PrettyBlockOutputStream::calculateWidths(Block & block, Widths_t & max_widt
size_t result_number = block.columns();
block.insert(column);
ColumnNumbers arguments;
arguments.push_back(i);
visible_width_func.execute(block, arguments, result_number);
column.column = block.getByPosition(i + columns).column;
visible_width_func.execute(block, {i}, result_number);
column.column = block.getByPosition(result_number).column;
if (const ColumnUInt64 * col = typeid_cast<const ColumnUInt64 *>(&*column.column))
{
const ColumnUInt64::Container_t & res = col->getData();
for (size_t j = 0; j < rows; ++j)
{
if (res[j] > max_widths[i])
max_widths[i] = res[j];
}
}
else if (const ColumnConstUInt64 * col = typeid_cast<const ColumnConstUInt64 *>(&*column.column))
{

View File

@ -7,6 +7,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
void PrettyCompactBlockOutputStream::writeHeader(
const Block & block,
const Widths_t & max_widths,

View File

@ -93,8 +93,11 @@ Block SummingSortedBlockInputStream::readImpl()
else
{
/// Оставляем только числовые типы. При чём, даты и даты-со-временем здесь такими не считаются.
if (!column.type->isNumeric() || column.type->getName() == "Date" ||
column.type->getName() == "DateTime")
if (!column.type->isNumeric() ||
column.type->getName() == "Date" ||
column.type->getName() == "DateTime" ||
column.type->getName() == "Nullable(Date)" ||
column.type->getName() == "Nullable(DateTime)")
continue;
/// Входят ли в PK?

View File

@ -3,6 +3,7 @@
#include <DB/Interpreters/convertFieldToType.h>
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/DataStreams/ValuesRowInputStream.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/Core/Block.h>
@ -111,12 +112,33 @@ bool ValuesRowInputStream::read(Block & block)
Field value = convertFieldToType(evaluateConstantExpression(ast, context), type);
/// TODO После добавления поддержки NULL, добавить сюда проверку на data type is nullable.
if (value.isNull())
throw Exception("Expression returns value " + apply_visitor(FieldVisitorToString(), value)
+ ", that is out of range of type " + type.getName()
+ ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE);
{
/// Check that we are indeed allowed to insert a NULL.
bool is_null_allowed = false;
if (type.isNullable())
is_null_allowed = true;
else
{
/// NOTE: For now we support only one level of null values, i.e.
/// there are not yet such things as Array(Nullable(Array(Nullable(T))).
/// Therefore the code below is valid within the current limitations.
const auto array_type = typeid_cast<const DataTypeArray *>(&type);
if (array_type != nullptr)
{
const auto & nested_type = array_type->getMostNestedType();
if (nested_type->isNullable())
is_null_allowed = true;
}
}
if (!is_null_allowed)
throw Exception{"Expression returns value " + apply_visitor(FieldVisitorToString(), value)
+ ", that is out of range of type " + type.getName()
+ ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
}
col.column->insert(value);

View File

@ -1,5 +1,6 @@
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
@ -188,7 +189,8 @@ void DataTypeArray::deserializeOffsets(IColumn & column, ReadBuffer & istr, size
}
void DataTypeArray::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
template <typename Writer>
static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, Writer && write_nested)
{
const ColumnArray & column_array = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets_t & offsets = column_array.getOffsets();
@ -203,7 +205,7 @@ void DataTypeArray::serializeText(const IColumn & column, size_t row_num, WriteB
{
if (i != offset)
writeChar(',', ostr);
nested->serializeTextQuoted(nested_column, i, ostr);
write_nested(nested_column, i);
}
writeChar(']', ostr);
}
@ -258,9 +260,23 @@ static void deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && r
}
void DataTypeArray::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
serializeTextImpl(column, row_num, ostr,
[&](const IColumn & nested_column, size_t i)
{
nested->serializeTextQuoted(nested_column, i, ostr);
});
}
void DataTypeArray::deserializeText(IColumn & column, ReadBuffer & istr) const
{
deserializeTextImpl(column, istr, [&](IColumn & nested_column) { nested->deserializeTextQuoted(nested_column, istr); });
deserializeTextImpl(column, istr,
[&](IColumn & nested_column)
{
nested->deserializeTextQuoted(nested_column, istr);
});
}
@ -359,7 +375,16 @@ void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons
ColumnPtr DataTypeArray::createColumn() const
{
return std::make_shared<ColumnArray>(nested->createColumn());
if (nested->isNull())
{
ColumnPtr col = std::make_shared<ColumnUInt8>();
ColumnPtr null_map = std::make_shared<ColumnUInt8>();
ColumnPtr nullable_col = std::make_shared<ColumnNullable>(col, null_map);
return std::make_shared<ColumnArray>(nullable_col);
}
else
return std::make_shared<ColumnArray>(nested->createColumn());
}
@ -369,4 +394,25 @@ ColumnPtr DataTypeArray::createConstColumn(size_t size, const Field & field) con
return std::make_shared<ColumnConstArray>(size, get<const Array &>(field), std::make_shared<DataTypeArray>(nested));
}
const DataTypePtr & DataTypeArray::getMostNestedType() const
{
const DataTypeArray * array = this;
const IDataType * array_nested_type = array->getNestedType().get();
while (true)
{
const DataTypeArray * type = typeid_cast<const DataTypeArray *>(array_nested_type);
if (type == nullptr)
break;
else
{
array = type;
array_nested_type = array->getNestedType().get();
}
}
return array->getNestedType();
}
}

View File

@ -9,6 +9,8 @@
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeNull.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
@ -56,6 +58,7 @@ DataTypeFactory::DataTypeFactory()
{"Date", std::make_shared<DataTypeDate>()},
{"DateTime", std::make_shared<DataTypeDateTime>()},
{"String", std::make_shared<DataTypeString>()},
{"Null", std::make_shared<DataTypeNull>()}
}
{
}
@ -106,8 +109,21 @@ DataTypePtr DataTypeFactory::get(const String & name) const
String base_name(name.data() + matches[1].offset, matches[1].length);
String parameters(name.data() + matches[2].offset, matches[2].length);
if (base_name == "Nullable")
return std::make_shared<DataTypeNullable>(get(parameters));
if (base_name == "Array")
return std::make_shared<DataTypeArray>(get(parameters));
{
if (parameters == "Null")
{
/// Special case: Array(Null) is actually Array(Nullable(UInt8)).
return std::make_shared<DataTypeArray>(
std::make_shared<DataTypeNullable>(
std::make_shared<DataTypeUInt8>()));
}
else
return std::make_shared<DataTypeArray>(get(parameters));
}
if (base_name == "AggregateFunction")
{

View File

@ -0,0 +1,25 @@
#include <DB/DataTypes/DataTypeNull.h>
namespace DB
{
void DataTypeNull::serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
{
size_t size = column.size();
if ((limit == 0) || ((offset + limit) > size))
limit = size - offset;
UInt8 x = 0;
writeBinary(x, limit, ostr);
}
void DataTypeNull::deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
{
ColumnNull & null_col = static_cast<ColumnNull &>(column);
istr.ignore(sizeof(UInt8) * limit);
null_col.insertRangeFrom(ColumnNull{0, Null()}, 0, limit);
}
}

View File

@ -0,0 +1,261 @@
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/DataTypes/NullSymbol.h>
#include <DB/Columns/ColumnNullable.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
namespace
{
/// When a column is serialized as a binary data file, its null values are directly
/// represented as null symbols into this file. The methods below are helpers that
/// we use for the binary deserialization. They are used as follows:
///
/// auto action = NullDeserializer<NullSymbol::XXX>::execute(col, istr);
///
/// if (action == Action::NONE) /// eof
/// return;
/// else if (action == Action::ADD_ORDINARY) /// add an ordinary value
/// ... deserialize the nested column ...
///
/// updateNullMap(col, action);
///
/// This two-step process is required because when we perform an INSERT query
/// whose values are expressions, ValuesRowInputStream attempts to deserialize
/// a stream, then raises an exception, and finally evaluates expressions inside
/// the exception handler. If we did something like:
///
/// ... deserialize the null map...
/// ... deserialize the nested column...
///
/// there would be garbage in the null map.
/// Action to be performed while updating the null map of a nullable column.
enum class Action
{
NONE, /// do nothing
ADD_NULL, /// add a value indicating a NULL
ADD_ORDINARY /// add a value indicating an ordinary value
};
/// The template class below provides one method that takes a nullable column being
/// deserialized and looks if there is a pending null symbol in the corresponding
/// binary data file. It returns the appropriate action to be performed on the null
/// map of the column.
template <typename Null>
struct NullDeserializer
{
static Action execute(ColumnNullable & col, ReadBuffer & istr)
{
if (!istr.eof())
{
if (*istr.position() == Null::name[0])
{
++istr.position();
static constexpr auto length = __builtin_strlen(Null::name);
if (length > 1)
assertString(&Null::name[1], istr);
return Action::ADD_NULL;
}
else
return Action::ADD_ORDINARY;
}
else
return Action::NONE;
}
};
/// This function takes the appropiate action when updating the null map of a nullable
/// column.
void updateNullMap(ColumnNullable & col, const Action & action)
{
auto & null_map = static_cast<ColumnUInt8 &>(*col.getNullValuesByteMap()).getData();
if (action == Action::ADD_NULL)
{
null_map.push_back(1);
ColumnPtr & nested_col = col.getNestedColumn();
nested_col->insertDefault();
}
else if (action == Action::ADD_ORDINARY)
null_map.push_back(0);
else
throw Exception{"DataTypeNullable: internal error", ErrorCodes::LOGICAL_ERROR};
}
}
DataTypeNullable::DataTypeNullable(DataTypePtr nested_data_type_)
: nested_data_type{nested_data_type_}
{
}
void DataTypeNullable::serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
nested_data_type->serializeBinary(*col.getNestedColumn(), ostr, offset, limit);
}
void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
{
ColumnNullable & col = static_cast<ColumnNullable &>(column);
nested_data_type->deserializeBinary(*col.getNestedColumn(), istr, limit, avg_value_size_hint);
}
void DataTypeNullable::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
nested_data_type->serializeBinary(*col.getNestedColumn(), row_num, ostr);
}
void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable & col = static_cast<ColumnNullable &>(column);
nested_data_type->deserializeBinary(*col.getNestedColumn(), istr);
}
void DataTypeNullable::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
writeCString(NullSymbol::Escaped::name, ostr);
else
nested_data_type->serializeTextEscaped(*col.getNestedColumn(), row_num, ostr);
}
void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable & col = static_cast<ColumnNullable &>(column);
auto action = NullDeserializer<NullSymbol::Escaped>::execute(col, istr);
if (action == Action::NONE)
return;
else if (action == Action::ADD_ORDINARY)
nested_data_type->deserializeTextEscaped(*col.getNestedColumn(), istr);
updateNullMap(col, action);
}
void DataTypeNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
{
/// This is not a typo. We really mean "Escaped" and not "Quoted".
/// The reason is that, when displaying an array of nullable strings,
/// we want to see \N instead of NULL.
writeCString(NullSymbol::Escaped::name, ostr);
}
else
nested_data_type->serializeTextQuoted(*col.getNestedColumn(), row_num, ostr);
}
void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable & col = static_cast<ColumnNullable &>(column);
auto action = NullDeserializer<NullSymbol::Quoted>::execute(col, istr);
if (action == Action::NONE)
return;
else if (action == Action::ADD_ORDINARY)
nested_data_type->deserializeTextQuoted(*col.getNestedColumn(), istr);
updateNullMap(col, action);
}
void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
writeCString(NullSymbol::Quoted::name, ostr);
else
nested_data_type->serializeTextCSV(*col.getNestedColumn(), row_num, ostr);
}
void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
ColumnNullable & col = static_cast<ColumnNullable &>(column);
auto action = NullDeserializer<NullSymbol::Quoted>::execute(col, istr);
if (action == Action::NONE)
return;
else if (action == Action::ADD_ORDINARY)
nested_data_type->deserializeTextCSV(*col.getNestedColumn(), istr, delimiter);
updateNullMap(col, action);
}
void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
writeCString(NullSymbol::Plain::name, ostr);
else
nested_data_type->serializeText(*col.getNestedColumn(), row_num, ostr);
}
void DataTypeNullable::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr,
bool force_quoting_64bit_integers) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
writeCString(NullSymbol::JSON::name, ostr);
else
nested_data_type->serializeTextJSON(*col.getNestedColumn(), row_num, ostr,
force_quoting_64bit_integers);
}
void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{
ColumnNullable & col = static_cast<ColumnNullable &>(column);
auto action = NullDeserializer<NullSymbol::JSON>::execute(col, istr);
if (action == Action::NONE)
return;
else if (action == Action::ADD_ORDINARY)
nested_data_type->deserializeTextJSON(*col.getNestedColumn(), istr);
updateNullMap(col, action);
}
void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
const ColumnNullable & col = static_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
writeCString(NullSymbol::XML::name, ostr);
else
nested_data_type->serializeTextXML(*col.getNestedColumn(), row_num, ostr);
}
ColumnPtr DataTypeNullable::createColumn() const
{
ColumnPtr new_col = nested_data_type->createColumn();
return std::make_shared<ColumnNullable>(new_col, std::make_shared<ColumnUInt8>());
}
ColumnPtr DataTypeNullable::createConstColumn(size_t size, const Field & field) const
{
ColumnPtr new_col = nested_data_type->createConstColumn(size, field);
return std::make_shared<ColumnNullable>(new_col, std::make_shared<ColumnUInt8>(size));
}
}

View File

@ -1,6 +1,8 @@
#include <DB/Core/FieldVisitors.h>
#include <DB/DataTypes/FieldToDataType.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/DataTypes/DataTypeNull.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <ext/size.hpp>
@ -11,7 +13,6 @@ namespace ErrorCodes
{
extern const int EMPTY_DATA_PASSED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -19,7 +20,10 @@ template <typename T>
static void convertArrayToCommonType(Array & arr)
{
for (auto & elem : arr)
elem = apply_visitor(FieldVisitorConvertToNumber<T>(), elem);
{
if (!elem.isNull())
elem = apply_visitor(FieldVisitorConvertToNumber<T>(), elem);
}
}
@ -41,10 +45,19 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
bool has_array = false;
bool has_float = false;
bool has_tuple = false;
bool has_null = false;
int max_bits = 0;
int max_signed_bits = 0;
int max_unsigned_bits = 0;
/// Wrap the specified type into an array type. If at least one element of
/// the array is nullable, first turn the input argument into a nullable type.
auto wrap_into_array = [&has_null](const DataTypePtr & type)
{
return std::make_shared<DataTypeArray>(
has_null ? std::make_shared<DataTypeNullable>(type) : type);
};
for (const Field & elem : x)
{
switch (elem.getType())
@ -99,7 +112,7 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
}
case Field::Types::Null:
{
throw Exception("NULL literals are not implemented yet", ErrorCodes::NOT_IMPLEMENTED);
has_null = true;
break;
}
}
@ -115,7 +128,7 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
throw Exception("Type inference of array of tuples is not supported", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (has_string)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
return wrap_into_array(std::make_shared<DataTypeString>());
if (has_float && max_bits == 64)
throw Exception("Incompatible types Float64 and UInt64/Int64 of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -123,7 +136,7 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
if (has_float)
{
convertArrayToCommonType<Float64>(x);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>());
return wrap_into_array(std::make_shared<DataTypeFloat64>());
}
if (max_signed_bits == 64 && max_unsigned_bits == 64)
@ -132,25 +145,25 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
if (max_signed_bits && !max_unsigned_bits)
{
if (max_signed_bits == 8)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt8>());
return wrap_into_array(std::make_shared<DataTypeInt8>());
if (max_signed_bits == 16)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt16>());
return wrap_into_array(std::make_shared<DataTypeInt16>());
if (max_signed_bits == 32)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt32>());
return wrap_into_array(std::make_shared<DataTypeInt32>());
if (max_signed_bits == 64)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt64>());
return wrap_into_array(std::make_shared<DataTypeInt64>());
}
if (!max_signed_bits && max_unsigned_bits)
{
if (max_unsigned_bits == 8)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt8>());
return wrap_into_array(std::make_shared<DataTypeUInt8>());
if (max_unsigned_bits == 16)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt16>());
return wrap_into_array(std::make_shared<DataTypeUInt16>());
if (max_unsigned_bits == 32)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>());
return wrap_into_array(std::make_shared<DataTypeUInt32>());
if (max_unsigned_bits == 64)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
return wrap_into_array(std::make_shared<DataTypeUInt64>());
}
if (max_signed_bits && max_unsigned_bits)
@ -161,11 +174,11 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
{
/// Беззнаковый тип не помещается в знаковый. Надо увеличить количество бит.
if (max_bits == 8)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt16>());
return wrap_into_array(std::make_shared<DataTypeInt16>());
if (max_bits == 16)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt32>());
return wrap_into_array(std::make_shared<DataTypeInt32>());
if (max_bits == 32)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt64>());
return wrap_into_array(std::make_shared<DataTypeInt64>());
else
throw Exception("Incompatible types UInt64 and signed integer of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -173,16 +186,23 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
{
/// Беззнаковый тип помещается в знаковый.
if (max_bits == 8)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt8>());
return wrap_into_array(std::make_shared<DataTypeInt8>());
if (max_bits == 16)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt16>());
return wrap_into_array(std::make_shared<DataTypeInt16>());
if (max_bits == 32)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt32>());
return wrap_into_array(std::make_shared<DataTypeInt32>());
if (max_bits == 64)
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt64>());
return wrap_into_array(std::make_shared<DataTypeInt64>());
}
}
if (has_null)
{
/// Special case: an array of NULLs is represented as an array
/// of Nullable(UInt8) because ColumnNull is actually ColumnConst<Null>.
return wrap_into_array(std::make_shared<DataTypeUInt8>());
}
throw Exception("Incompatible types of elements of array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -0,0 +1,18 @@
#include <DB/DataTypes/NullSymbol.h>
namespace DB
{
namespace NullSymbol
{
constexpr decltype(Plain::name) Plain::name;
constexpr decltype(Escaped::name) Escaped::name;
constexpr decltype(Quoted::name) Quoted::name;
constexpr decltype(CSV::name) CSV::name;
constexpr decltype(JSON::name) JSON::name;
constexpr decltype(XML::name) XML::name;
}
}

View File

@ -5,6 +5,7 @@
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeNullable.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/WriteHelpers.h>
@ -64,7 +65,18 @@ public:
|| ResultDataTypeDeducer<TType, DataTypeInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeFloat32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeFloat64>::execute(args, i, type_res)))
|| ResultDataTypeDeducer<TType, DataTypeFloat64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeNull>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt8> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt16> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt32> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt64> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt8> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt16> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt32> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt64> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeFloat32> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeFloat64> >::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
@ -81,6 +93,31 @@ public:
}
};
template <typename TType>
struct TypeChecker
{
static bool execute(const DataTypePtr & arg)
{
if (arg->isNullable())
return false;
return typeid_cast<const TType *>(arg.get()) != nullptr;
}
};
template <typename TType>
struct TypeChecker<Nullable<TType>>
{
static bool execute(const DataTypePtr & arg)
{
if (!arg->isNullable())
return false;
const DataTypeNullable & nullable_type = static_cast<DataTypeNullable &>(*arg);
const IDataType * nested_type = nullable_type.getNestedType().get();
return typeid_cast<const TType *>(nested_type) != nullptr;
}
};
/// Analyze the type of the branch currently being processed of a multiIf function.
/// Subsequently perform the same analysis for the remaining branches.
/// Determine the returned type if all the processed branches are numeric.
@ -94,7 +131,7 @@ private:
public:
static bool execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
if (typeid_cast<const TType *>(&*args[i]) == nullptr)
if (!TypeChecker<TType>::execute(args[i]))
return false;
if (i == elseArg(args))
@ -120,7 +157,9 @@ class FirstResultDataTypeDeducer final
public:
static void execute(const DataTypes & args, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
using Void = typename DataTypeTraits::ToEnrichedDataType<NumberTraits::Enriched::Void>::Type;
using Void = typename DataTypeTraits::ToEnrichedDataType<
NumberTraits::Enriched::Void<NumberTraits::HasNoNull>
>::Type;
size_t i = firstThen();
@ -133,7 +172,18 @@ public:
|| ResultDataTypeDeducer<Void, DataTypeInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeFloat32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeFloat64>::execute(args, i, type_res)))
|| ResultDataTypeDeducer<Void, DataTypeFloat64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeNull>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt8> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt16> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt32> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt64> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt8> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt16> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt32> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt64> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeFloat32> >::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeFloat64> >::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
@ -149,13 +199,13 @@ DataTypePtr getReturnTypeForArithmeticArgs(const DataTypes & args)
bool hasArithmeticBranches(const DataTypes & args)
{
size_t else_arg = elseArg(args);
auto check = [&](size_t i)
auto check = [&args](size_t i)
{
return args[i]->behavesAsNumber();
return args[i]->behavesAsNumber() || args[i]->isNull();
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (!check(i))
@ -167,13 +217,22 @@ bool hasArithmeticBranches(const DataTypes & args)
bool hasArrayBranches(const DataTypes & args)
{
size_t else_arg = elseArg(args);
auto check = [&](size_t i)
auto check = [&args](size_t i)
{
return typeid_cast<const DataTypeArray *>(args[i].get()) != nullptr;
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return (typeid_cast<const DataTypeArray *>(observed_type) != nullptr) || args[i]->isNull();
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < elseArg(args); i = nextThen(i))
{
if (!check(i))
@ -185,32 +244,60 @@ bool hasArrayBranches(const DataTypes & args)
bool hasIdenticalTypes(const DataTypes & args)
{
size_t else_arg = elseArg(args);
auto first_type_name = args[firstThen()]->getName();
auto check = [&](size_t i)
auto check = [&args](size_t i, std::string & first_type_name)
{
return args[i]->getName() == first_type_name;
if (!args[i]->isNull())
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
const std::string & name = observed_type->getName();
if (first_type_name.empty())
first_type_name = name;
else if (name != first_type_name)
return false;
}
return true;
};
for (size_t i = secondThen(); i < elseArg(args); i = nextThen(i))
size_t else_arg = elseArg(args);
std::string first_type_name;
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (!check(i))
if (!check(i, first_type_name))
return false;
}
return check(else_arg);
return check(else_arg, first_type_name);
}
bool hasFixedStrings(const DataTypes & args)
{
size_t else_arg = elseArg(args);
auto check = [&](size_t i)
auto check = [&args](size_t i)
{
return typeid_cast<const DataTypeFixedString *>(args[i].get()) != nullptr;
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return (typeid_cast<const DataTypeFixedString *>(observed_type) != nullptr) || (args[i]->isNull());
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < elseArg(args); i = nextThen(i))
{
if (!check(i))
@ -222,38 +309,69 @@ bool hasFixedStrings(const DataTypes & args)
bool hasFixedStringsOfIdenticalLength(const DataTypes & args)
{
size_t else_arg = elseArg(args);
auto get_length = [&](size_t i)
auto check = [&args](size_t i, bool & has_length, size_t & first_length)
{
auto fixed_str = typeid_cast<const DataTypeFixedString *>(args[i].get());
if (fixed_str == nullptr)
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
if (!args[i]->isNull())
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return fixed_str->getN();
/// Get the length of the fixed string currently being checked.
auto fixed_str = typeid_cast<const DataTypeFixedString *>(observed_type);
if (fixed_str == nullptr)
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
size_t length = fixed_str->getN();
if (!has_length)
{
has_length = true;
first_length = length;
}
else if (length != first_length)
return false;
}
return true;
};
auto first_length = get_length(firstThen());
size_t else_arg = elseArg(args);
bool has_length = false;
size_t first_length = 0;
for (size_t i = secondThen(); i < elseArg(args); i = nextThen(i))
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (get_length(i) != first_length)
if (!check(i, has_length, first_length))
return false;
}
return get_length(else_arg) == first_length;
return check(else_arg, has_length, first_length);
}
bool hasStrings(const DataTypes & args)
{
size_t else_arg = elseArg(args);
auto check = [&](size_t i)
auto check = [&args](size_t i)
{
return (typeid_cast<const DataTypeFixedString *>(args[i].get()) != nullptr) ||
(typeid_cast<const DataTypeString *>(args[i].get()) != nullptr);
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return (typeid_cast<const DataTypeFixedString *>(observed_type) != nullptr) ||
(typeid_cast<const DataTypeString *>(observed_type) != nullptr) || args[i]->isNull();
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < elseArg(args); i = nextThen(i))
{
if (!check(i))

View File

@ -3,6 +3,7 @@
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnNullable.h>
namespace DB
{
@ -18,17 +19,36 @@ namespace Conditional
{
const ColumnPtr CondSource::null_materialized_col;
const PaddedPODArray<UInt8> CondSource::empty_null_map;
CondSource::CondSource(const Block & block, const ColumnNumbers & args, size_t i)
: materialized_col{initMaterializedCol(block, args, i)},
data_array{initDataArray(block, args, i, materialized_col)}
data_array{initDataArray(block, args, i, materialized_col)},
null_map{initNullMap(block, args, i)}
{
}
const ColumnPtr CondSource::initMaterializedCol(const Block & block, const ColumnNumbers & args, size_t i)
{
const ColumnPtr & col = block.getByPosition(args[i]).column;
const auto * const_col = typeid_cast<const ColumnConst<UInt8> *>(&*col);
if (col->isNull())
{
const ColumnNull & null_col = static_cast<const ColumnNull &>(*col);
return null_col.convertToFullColumn();
}
const IColumn * observed_col;
if (col->isNullable())
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*col);
observed_col = nullable_col.getNestedColumn().get();
}
else
observed_col = col.get();
const auto * const_col = typeid_cast<const ColumnConst<UInt8> *>(observed_col);
if (const_col != nullptr)
return const_col->convertToFullColumn();
@ -49,7 +69,17 @@ const PaddedPODArray<UInt8> & CondSource::initDataArray(const Block & block, con
source_col = col.get();
}
const auto * vec_col = typeid_cast<const ColumnUInt8 *>(source_col);
const IColumn * observed_col;
if (source_col->isNullable())
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*source_col);
observed_col = nullable_col.getNestedColumn().get();
}
else
observed_col = source_col;
const auto * vec_col = typeid_cast<const ColumnUInt8 *>(observed_col);
if (vec_col == nullptr)
throw CondException{CondErrorCodes::COND_SOURCE_ILLEGAL_COLUMN,
@ -58,6 +88,21 @@ const PaddedPODArray<UInt8> & CondSource::initDataArray(const Block & block, con
return vec_col->getData();
}
const PaddedPODArray<UInt8> & CondSource::initNullMap(const Block & block, const ColumnNumbers & args, size_t i)
{
const ColumnPtr & col = block.getByPosition(args[i]).column;
if (col->isNullable())
{
const ColumnNullable & nullable_col = static_cast<const ColumnNullable &>(*col);
const ColumnPtr & null_map = nullable_col.getNullValuesByteMap();
const ColumnUInt8 & content = static_cast<const ColumnUInt8 &>(*null_map);
return content.getData();
}
else
return empty_null_map;
}
}
}

View File

@ -0,0 +1,76 @@
#include <DB/Functions/Conditional/NullMapBuilder.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace Conditional
{
Block NullMapBuilder::empty_block;
void NullMapBuilder::init(const ColumnNumbers & args)
{
null_map = std::make_shared<ColumnUInt8>(row_count);
cols_properties.resize(block.columns());
for (const auto & arg : args)
{
const auto & col = *block.unsafeGetByPosition(arg).column;
if (col.isNullable())
cols_properties[arg] = IS_NULLABLE;
else if (col.isNull())
cols_properties[arg] = IS_NULL;
else
cols_properties[arg] = IS_ORDINARY;
}
}
void NullMapBuilder::update(size_t index, size_t row)
{
const IColumn & from = *block.unsafeGetByPosition(index).column;
bool is_null;
auto property = cols_properties[index];
if (property == IS_NULL)
is_null = true;
else if (property == IS_NULLABLE)
{
const auto & nullable_col = static_cast<const ColumnNullable &>(from);
is_null = nullable_col.isNullAt(row);
}
else if (property == IS_ORDINARY)
is_null = false;
else
throw Exception{"NullMapBuilder: internal error", ErrorCodes::LOGICAL_ERROR};
auto & null_map_data = static_cast<ColumnUInt8 &>(*null_map).getData();
null_map_data[row] = is_null ? 1 : 0;
}
void NullMapBuilder::build(size_t index)
{
const IColumn & from = *block.unsafeGetByPosition(index).column;
if (from.isNull())
null_map = std::make_shared<ColumnUInt8>(row_count, 1);
else if (from.isNullable())
{
const auto & nullable_col = static_cast<const ColumnNullable &>(from);
null_map = nullable_col.getNullValuesByteMap();
}
else
null_map = std::make_shared<ColumnUInt8>(row_count, 0);
}
}
}

View File

@ -1,4 +1,5 @@
#include <DB/Functions/Conditional/NumericPerformer.h>
#include <DB/Functions/Conditional/NullMapBuilder.h>
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/ArgsInfo.h>
#include <DB/Functions/Conditional/NumericEvaluator.h>
@ -29,21 +30,21 @@ namespace Conditional
namespace
{
struct Category
{
static const auto NONE = 0;
static const auto NUMERIC = UINT8_C(1) << 0;
static const auto NUMERIC_ARRAY = UINT8_C(1) << 1;
static const auto NULL_VALUE = UINT8_C(1) << 2;
};
/// This class provides a means to collect type information on a branch
/// (then or else) of a multiIf function.
template <typename TType>
struct PredicateBase
{
protected:
enum Category
{
NONE = 0,
NUMERIC,
NUMERIC_ARRAY
};
protected:
static Category appendBranchInfo(size_t index, const Block & block,
static bool appendBranchInfo(size_t index, const Block & block,
const ColumnNumbers & args, Branches & branches)
{
const IColumn * col = block.getByPosition(args[index]).column.get();
@ -74,7 +75,7 @@ protected:
if (arr_vec_col != nullptr)
branch.is_const = false;
else
return NONE;
return false;
}
else
{
@ -91,12 +92,12 @@ protected:
using ElementType = typename DataTypeFromFieldType<TType>::Type;
if (typeid_cast<const ElementType *>(nested_type) == nullptr)
return NONE;
return false;
branch.is_const = true;
}
else
return NONE;
return false;
}
}
}
@ -104,17 +105,49 @@ protected:
branch.index = index;
branch.type = DataTypeTraits::DataTypeFromFieldTypeOrError<TType>::getDataType();
branches.push_back(branch);
if ((vec_col != nullptr) || (const_col != nullptr))
return NUMERIC;
branch.category = Category::NUMERIC;
else if ((arr_vec_col != nullptr) || (arr_const_col != nullptr))
return NUMERIC_ARRAY;
branch.category = Category::NUMERIC_ARRAY;
else
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
branches.push_back(branch);
return true;
}
};
template <>
struct PredicateBase<Null>
{
protected:
static bool appendBranchInfo(size_t index, const Block & block,
const ColumnNumbers & args, Branches & branches)
{
const IColumn * col = block.getByPosition(args[index]).column.get();
const ColumnNull * const_col = typeid_cast<const ColumnNull *>(col);
if (const_col == nullptr)
return false;
Branch branch;
branch.is_const = true;
branch.index = index;
branch.type = DataTypeTraits::DataTypeFromFieldTypeOrError<Null>::getDataType();
branch.category = Category::NULL_VALUE;
branches.push_back(branch);
return true;
}
};
template <>
struct PredicateBase<NumberTraits::Error>
{
};
/// Collect type information on the unique else branch of a multiIf function.
/// Determine the returned value type. Perform the multiIf.
template <typename TResult, typename TType>
@ -128,20 +161,29 @@ struct ElsePredicate final : public PredicateBase<TType>
>::Type;
using TFinal = typename NumberTraits::ToOrdinaryType<TCombined>::Type;
using TFinal2 = typename RemoveNullable<TFinal>::Type;
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
size_t result, NullMapBuilder & builder, Branches & branches)
{
auto category = Base::appendBranchInfo(index, block, args, branches);
if (!Base::appendBranchInfo(index, block, args, branches))
return false;
/// We have collected all the information we need.
/// Now perform the multiIf.
if (category == Base::NONE)
return false;
else if (category == Base::NUMERIC)
NumericEvaluator<TFinal>::perform(branches, block, args, result);
else if (category == Base::NUMERIC_ARRAY)
ArrayEvaluator<TFinal>::perform(branches, block, args, result);
UInt8 category = Category::NONE;
for (const auto & branch : branches)
category |= branch.category;
if (category & Category::NUMERIC)
{
if (category & Category::NUMERIC_ARRAY)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
NumericEvaluator<TFinal2>::perform(branches, block, args, result, builder);
}
else if (category & Category::NUMERIC_ARRAY)
ArrayEvaluator<TFinal2>::perform(branches, block, args, result, builder);
else
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
@ -149,12 +191,23 @@ struct ElsePredicate final : public PredicateBase<TType>
}
};
/// Specialization for incorrect type information.
template <typename TResult>
struct ElsePredicate<TResult, NumberTraits::Error>
/// We cannot have only null branches.
template <typename Nullity>
struct ElsePredicate<NumberTraits::Enriched::Void<Nullity>, Null> final : public PredicateBase<Null>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
size_t result, NullMapBuilder & builder, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for incorrect type information.
template <typename TResult>
struct ElsePredicate<TResult, NumberTraits::Error> : public PredicateBase<NumberTraits::Error>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, NullMapBuilder & builder, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
@ -165,7 +218,7 @@ template <typename TType>
struct ElsePredicate<NumberTraits::Error, TType>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
size_t result, NullMapBuilder & builder, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
@ -184,11 +237,9 @@ struct ThenPredicate final : public PredicateBase<TType>
>::Type;
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
size_t result, NullMapBuilder & builder, Branches & branches)
{
auto category = Base::appendBranchInfo(index, block, args, branches);
if (category == Base::NONE)
if (!Base::appendBranchInfo(index, block, args, branches))
return false;
/// Guess what comes after Then.
@ -197,34 +248,34 @@ struct ThenPredicate final : public PredicateBase<TType>
if (index2 != elseArg(args))
{
/// We have a pair Cond-Then. Process the next Then.
if (! (ThenPredicate<TCombined, UInt8>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, UInt16>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, UInt32>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, UInt64>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int8>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int16>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int32>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int64>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Float32>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Float64>::execute(index2 + 1, block, args, result, branches)))
throw CondException{CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN,
toString(index2 + 1)};
if (! (ThenPredicate<TCombined, UInt8>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, UInt16>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, UInt32>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, UInt64>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Int8>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Int16>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Int32>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Int64>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Float32>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Float64>::execute(index2 + 1, block, args, result, builder, branches)
|| ThenPredicate<TCombined, Null>::execute(index2 + 1, block, args, result, builder, branches)))
return false;
}
else
{
/// We have an Else which ends the multiIf. Process it.
if (! (ElsePredicate<TCombined, UInt8>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, UInt16>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, UInt32>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, UInt64>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int8>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int16>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int32>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int64>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Float32>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Float64>::execute(index2, block, args, result, branches)))
throw CondException{CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN,
toString(index2)};
if (! (ElsePredicate<TCombined, UInt8>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, UInt16>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, UInt32>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, UInt64>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Int8>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Int16>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Int32>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Int64>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Float32>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Float64>::execute(index2, block, args, result, builder, branches)
|| ElsePredicate<TCombined, Null>::execute(index2, block, args, result, builder, branches)))
return false;
}
return true;
@ -236,7 +287,7 @@ template <typename TResult>
struct ThenPredicate<TResult, NumberTraits::Error>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
size_t result, NullMapBuilder & builder, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
@ -247,7 +298,7 @@ template <typename TType>
struct ThenPredicate<NumberTraits::Error, TType>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
size_t result, NullMapBuilder & builder, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
@ -256,30 +307,31 @@ struct ThenPredicate<NumberTraits::Error, TType>
/// First Then
struct FirstThenPredicate final
{
static bool execute(Block & block, const ColumnNumbers & args, size_t result)
static bool execute(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
using Void = NumberTraits::Enriched::Void;
using Void = NumberTraits::Enriched::Void<NumberTraits::HasNoNull>;
Branches branches;
return ThenPredicate<Void, UInt8>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, UInt16>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, UInt32>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, UInt64>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int8>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int16>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int32>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int64>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Float32>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Float64>::execute(firstThen(), block, args, result, branches);
return ThenPredicate<Void, UInt8>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, UInt16>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, UInt32>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, UInt64>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Int8>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Int16>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Int32>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Int64>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Float32>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Float64>::execute(firstThen(), block, args, result, builder, branches)
|| ThenPredicate<Void, Null>::execute(firstThen(), block, args, result, builder, branches);
}
};
}
bool NumericPerformer::perform(Block & block, const ColumnNumbers & args,
size_t result)
size_t result, NullMapBuilder & builder)
{
return FirstThenPredicate::execute(block, args, result);
return FirstThenPredicate::execute(block, args, result, builder);
}
}

View File

@ -1,6 +1,7 @@
#include <DB/Functions/Conditional/StringArrayEvaluator.h>
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/NullMapBuilder.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnConst.h>
@ -65,6 +66,7 @@ public:
virtual void next() = 0;
virtual size_t getDataSize() const = 0;
virtual size_t getStringOffsetsSize() const = 0;
virtual size_t getIndex() const = 0;
};
using StringArraySourcePtr = std::unique_ptr<StringArraySource>;
@ -76,8 +78,10 @@ class VarStringArraySource : public StringArraySource
public:
VarStringArraySource(const ColumnString::Chars_t & data_,
const ColumnString::Offsets_t & string_offsets_,
const ColumnArray::Offsets_t & array_offsets_)
: data{data_}, string_offsets{string_offsets_}, array_offsets{array_offsets_}
const ColumnArray::Offsets_t & array_offsets_,
size_t index_)
: data{data_}, string_offsets{string_offsets_}, array_offsets{array_offsets_},
index{index_}
{
}
@ -115,6 +119,11 @@ public:
return string_offsets.size();
}
size_t getIndex() const override
{
return index;
}
private:
const ColumnString::Chars_t & data;
const ColumnString::Offsets_t & string_offsets;
@ -122,6 +131,7 @@ private:
ColumnArray::Offset_t array_prev_offset = 0;
ColumnString::Offset_t string_prev_offset = 0;
size_t index = 0;
size_t i = 0;
VarCallback var_callback = [&](ColumnString::Chars_t & to_data,
@ -156,8 +166,8 @@ private:
class ConstStringArraySource : public StringArraySource
{
public:
ConstStringArraySource(const Array & data_)
: data{data_}
ConstStringArraySource(const Array & data_, size_t index_)
: data{data_}, index{index_}
{
data_size = 0;
for (const auto & s : data)
@ -194,9 +204,15 @@ public:
return data.size();
}
size_t getIndex() const override
{
return index;
}
private:
const Array & data;
size_t data_size;
size_t index;
VarCallback var_callback = [&](ColumnString::Chars_t & to_data,
ColumnString::Offset_t & to_string_prev_offset,
@ -275,6 +291,8 @@ CondSources createConds(const Block & block, const ColumnNumbers & args)
return conds;
}
const Array null_array{String()};
/// Create accessors for branch values.
bool createStringArraySources(StringArraySources & sources, const Block & block,
const ColumnNumbers & args)
@ -286,15 +304,22 @@ bool createStringArraySources(StringArraySources & sources, const Block & block,
const ColumnString * var_col = col_arr ? typeid_cast<const ColumnString *>(&col_arr->getData()) : nullptr;
const ColumnConstArray * const_col = typeid_cast<const ColumnConstArray *>(col);
if ((col_arr && var_col) || const_col)
if (col->isNull())
{
StringArraySourcePtr source;
source = std::make_unique<ConstStringArraySource>(null_array, args[i]);
sources.push_back(std::move(source));
return true;
}
else if ((col_arr && var_col) || const_col)
{
StringArraySourcePtr source;
if (var_col != nullptr)
source = std::make_unique<VarStringArraySource>(var_col->getChars(),
var_col->getOffsets(), col_arr->getOffsets());
var_col->getOffsets(), col_arr->getOffsets(), args[i]);
else if (const_col)
source = std::make_unique<ConstStringArraySource>(const_col->getData());
source = std::make_unique<ConstStringArraySource>(const_col->getData(), args[i]);
else
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
@ -369,7 +394,7 @@ VarStringArraySink createSink(Block & block, const StringArraySources & sources,
}
/// Process a multiIf.
bool StringArrayEvaluator::perform(Block & block, const ColumnNumbers & args, size_t result)
bool StringArrayEvaluator::perform(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
StringArraySources sources;
if (!createStringArraySources(sources, block, args))
@ -379,6 +404,9 @@ bool StringArrayEvaluator::perform(Block & block, const ColumnNumbers & args, si
size_t row_count = conds[0].getSize();
VarStringArraySink sink = createSink(block, sources, result, row_count);
if (builder)
builder.init(args);
for (size_t cur_row = 0; cur_row < row_count; ++cur_row)
{
bool has_triggered_cond = false;
@ -389,6 +417,8 @@ bool StringArrayEvaluator::perform(Block & block, const ColumnNumbers & args, si
if (cond.get(cur_row))
{
sink.store(sources[cur_source]->get());
if (builder)
builder.update(sources[cur_source]->getIndex(), cur_row);
has_triggered_cond = true;
break;
}
@ -396,7 +426,11 @@ bool StringArrayEvaluator::perform(Block & block, const ColumnNumbers & args, si
}
if (!has_triggered_cond)
{
sink.store(sources.back()->get());
if (builder)
builder.update(sources.back()->getIndex(), cur_row);
}
for (auto & source : sources)
source->next();

View File

@ -1,5 +1,6 @@
#include <DB/Functions/Conditional/StringEvaluator.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/NullMapBuilder.h>
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeString.h>
@ -8,6 +9,7 @@
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Core/Types.h>
namespace DB
@ -53,6 +55,7 @@ public:
virtual StringChunk get() const = 0;
virtual size_t getSize() const { throw Exception{"Unsupported method", ErrorCodes::LOGICAL_ERROR}; }
virtual size_t getDataSize() const = 0;
virtual size_t getIndex() const = 0;
};
using StringSourcePtr = std::unique_ptr<StringSource>;
@ -62,8 +65,8 @@ using StringSources = std::vector<StringSourcePtr>;
class ConstStringSource final : public StringSource
{
public:
ConstStringSource(const std::string & str_, size_t size_)
: str{str_}, size{size_},
ConstStringSource(const std::string & str_, size_t size_, size_t index_)
: str{str_}, size{size_}, index{index_},
type{static_cast<UInt64>(StringType::CONSTANT | ((size > 0) ? StringType::FIXED : 0))}
{
}
@ -102,9 +105,15 @@ public:
return str.length();
}
inline size_t getIndex() const override
{
return index;
}
private:
const std::string & str;
size_t size;
size_t index;
UInt64 type;
};
@ -112,8 +121,8 @@ private:
class FixedStringSource final : public StringSource
{
public:
FixedStringSource(const ColumnFixedString::Chars_t & data_, size_t size_)
: data{data_}, size{size_}
FixedStringSource(const ColumnFixedString::Chars_t & data_, size_t size_, size_t index_)
: data{data_}, size{size_}, index{index_}
{
}
@ -153,9 +162,15 @@ public:
return data.size();
}
inline size_t getIndex() const override
{
return index;
}
private:
const ColumnFixedString::Chars_t & data;
size_t size;
size_t index;
size_t i = 0;
};
@ -164,8 +179,8 @@ class VarStringSource final : public StringSource
{
public:
VarStringSource(const ColumnString::Chars_t & data_,
const ColumnString::Offsets_t & offsets_)
: data{data_}, offsets{offsets_}
const ColumnString::Offsets_t & offsets_, size_t index_)
: data{data_}, offsets{offsets_}, index{index_}
{
}
@ -200,10 +215,16 @@ public:
return data.size();
}
inline size_t getIndex() const override
{
return index;
}
private:
const ColumnString::Chars_t & data;
const ColumnString::Offsets_t & offsets;
ColumnString::Offset_t prev_offset = 0;
size_t index;
size_t i = 0;
};
@ -324,6 +345,8 @@ CondSources createConds(const Block & block, const ColumnNumbers & args)
return conds;
}
const std::string null_string = "";
/// Create accessors for branch values.
bool createStringSources(StringSources & sources, const Block & block,
const ColumnNumbers & args)
@ -337,12 +360,14 @@ bool createStringSources(StringSources & sources, const Block & block,
StringSourcePtr source;
if (var_col != nullptr)
if (col->isNull())
source = std::make_unique<ConstStringSource>(null_string, 1, args[i]);
else if (var_col != nullptr)
source = std::make_unique<VarStringSource>(var_col->getChars(),
var_col->getOffsets());
var_col->getOffsets(), args[i]);
else if (fixed_col != nullptr)
source = std::make_unique<FixedStringSource>(fixed_col->getChars(),
fixed_col->getN());
fixed_col->getN(), args[i]);
else if (const_col != nullptr)
{
/// If we actually have a fixed string, get its capacity.
@ -355,7 +380,7 @@ bool createStringSources(StringSources & sources, const Block & block,
size = fixed->getN();
}
source = std::make_unique<ConstStringSource>(const_col->getData(), size);
source = std::make_unique<ConstStringSource>(const_col->getData(), size, args[i]);
}
else
return false;
@ -412,9 +437,13 @@ template <typename SinkType>
class SinkUpdater
{
public:
static void execute(const StringSources & sources, const CondSources & conds,
SinkType & sink, size_t row_count)
static void execute(Block & block, const StringSources & sources, const CondSources & conds,
SinkType & sink, size_t row_count, const ColumnNumbers & args, size_t result,
NullMapBuilder & builder)
{
if (builder)
builder.init(args);
for (size_t cur_row = 0; cur_row < row_count; ++cur_row)
{
bool has_triggered_cond = false;
@ -425,6 +454,8 @@ public:
if (cond.get(cur_row))
{
sink.store(sources[cur_source]->get());
if (builder)
builder.update(sources[cur_source]->getIndex(), cur_row);
has_triggered_cond = true;
break;
}
@ -432,7 +463,11 @@ public:
}
if (!has_triggered_cond)
{
sink.store(sources.back()->get());
if (builder)
builder.update(sources.back()->getIndex(), cur_row);
}
for (auto & source : sources)
source->next();
@ -449,10 +484,12 @@ class Performer<true>
{
public:
static void execute(const StringSources & sources, const CondSources & conds,
size_t row_count, Block & block, size_t result)
size_t row_count, Block & block, const ColumnNumbers & args, size_t result,
NullMapBuilder & builder)
{
FixedStringSink sink = createSink(block, sources, result, row_count);
SinkUpdater<FixedStringSink>::execute(sources, conds, sink, row_count);
SinkUpdater<FixedStringSink>::execute(block, sources, conds, sink, row_count,
args, result, builder);
}
private:
@ -481,10 +518,12 @@ class Performer<false>
{
public:
static void execute(const StringSources & sources, const CondSources & conds,
size_t row_count, Block & block, size_t result)
size_t row_count, Block & block, const ColumnNumbers & args, size_t result,
NullMapBuilder & builder)
{
VarStringSink sink = createSink(block, sources, result, row_count);
SinkUpdater<VarStringSink>::execute(sources, conds, sink, row_count);
SinkUpdater<VarStringSink>::execute(block, sources, conds, sink, row_count,
args, result, builder);
}
private:
@ -505,7 +544,7 @@ private:
}
/// Process a multiIf.
bool StringEvaluator::perform(Block & block, const ColumnNumbers & args, size_t result)
bool StringEvaluator::perform(Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
{
StringSources sources;
if (!createStringSources(sources, block, args))
@ -525,9 +564,9 @@ bool StringEvaluator::perform(Block & block, const ColumnNumbers & args, size_t
}
if (has_only_fixed_sources)
Performer<true>::execute(sources, conds, row_count, block, result);
Performer<true>::execute(sources, conds, row_count, block, args, result, builder);
else
Performer<false>::execute(sources, conds, row_count, block, result);
Performer<false>::execute(sources, conds, row_count, block, args, result, builder);
return true;
}

Some files were not shown because too many files have changed in this diff Show More