2016-04-18 14:31:32 +00:00
|
|
|
#pragma once
|
|
|
|
|
2016-05-03 23:19:14 +00:00
|
|
|
#include <DB/Functions/Conditional/CondException.h>
|
2016-04-18 14:31:32 +00:00
|
|
|
#include <DB/Functions/Conditional/common.h>
|
2016-08-18 14:25:02 +00:00
|
|
|
#include <DB/Functions/Conditional/NullMapBuilder.h>
|
2016-04-18 14:31:32 +00:00
|
|
|
#include <DB/Functions/Conditional/CondSource.h>
|
|
|
|
#include <DB/Functions/NumberTraits.h>
|
|
|
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
|
|
|
#include <DB/Columns/ColumnVector.h>
|
|
|
|
#include <DB/Columns/ColumnConst.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace Conditional
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
/// This class provides type-independent access to the values of a numeric branch
|
|
|
|
/// (then, else) column. Returned values have the type TResult.
|
|
|
|
template <typename TResult>
|
|
|
|
class NumericSource
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
virtual TResult get(size_t i) const = 0;
|
|
|
|
virtual ~NumericSource() = default;
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename TResult>
|
|
|
|
using NumericSourcePtr = std::unique_ptr<NumericSource<TResult> >;
|
|
|
|
|
|
|
|
template <typename TResult>
|
|
|
|
using NumericSources = std::vector<NumericSourcePtr<TResult> >;
|
|
|
|
|
|
|
|
/// Column type-specific implementation of NumericSource.
|
2016-05-05 20:59:03 +00:00
|
|
|
template <typename TResult, typename TType, bool IsConst>
|
|
|
|
class NumericSourceImpl;
|
|
|
|
|
2016-04-18 14:31:32 +00:00
|
|
|
template <typename TResult, typename TType>
|
2016-05-05 20:59:03 +00:00
|
|
|
class NumericSourceImpl<TResult, TType, true> final : public NumericSource<TResult>
|
2016-04-18 14:31:32 +00:00
|
|
|
{
|
|
|
|
public:
|
|
|
|
NumericSourceImpl(const Block & block, const ColumnNumbers & args, const Branch & br)
|
|
|
|
{
|
|
|
|
size_t index = br.index;
|
|
|
|
|
2016-05-05 20:59:03 +00:00
|
|
|
const ColumnPtr & col = block.getByPosition(args[index]).column;
|
|
|
|
const auto * const_col = typeid_cast<const ColumnConst<TType> *>(&*col);
|
|
|
|
if (const_col == nullptr)
|
|
|
|
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
|
|
|
data = const_col->getData();
|
2016-04-18 14:31:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
NumericSourceImpl(const NumericSourceImpl &) = delete;
|
|
|
|
NumericSourceImpl & operator=(const NumericSourceImpl &) = delete;
|
|
|
|
|
|
|
|
NumericSourceImpl(NumericSourceImpl &&) = default;
|
|
|
|
NumericSourceImpl & operator=(NumericSourceImpl &&) = default;
|
|
|
|
|
|
|
|
TResult get(size_t i) const override
|
|
|
|
{
|
2016-05-05 20:59:03 +00:00
|
|
|
return static_cast<TResult>(data);
|
|
|
|
};
|
|
|
|
|
|
|
|
private:
|
|
|
|
TType data;
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename TResult, typename TType>
|
|
|
|
class NumericSourceImpl<TResult, TType, false> final : public NumericSource<TResult>
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
NumericSourceImpl(const Block & block, const ColumnNumbers & args, const Branch & br)
|
|
|
|
: data_array{initDataArray(block, args, br)}
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
NumericSourceImpl(const NumericSourceImpl &) = delete;
|
|
|
|
NumericSourceImpl & operator=(const NumericSourceImpl &) = delete;
|
|
|
|
|
|
|
|
NumericSourceImpl(NumericSourceImpl &&) = default;
|
|
|
|
NumericSourceImpl & operator=(NumericSourceImpl &&) = default;
|
|
|
|
|
|
|
|
TResult get(size_t i) const override
|
|
|
|
{
|
|
|
|
return static_cast<TResult>(data_array[i]);
|
2016-04-18 14:31:32 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
private:
|
|
|
|
static const PaddedPODArray<TType> & initDataArray(const Block & block,
|
|
|
|
const ColumnNumbers & args, const Branch & br)
|
|
|
|
{
|
2016-05-05 20:59:03 +00:00
|
|
|
size_t index = br.index;
|
|
|
|
const ColumnPtr & col = block.getByPosition(args[index]).column;
|
|
|
|
const auto * vec_col = typeid_cast<const ColumnVector<TType> *>(&*col);
|
|
|
|
if (vec_col == nullptr)
|
|
|
|
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
|
|
|
return vec_col->getData();
|
2016-04-18 14:31:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
const PaddedPODArray<TType> & data_array;
|
|
|
|
};
|
|
|
|
|
|
|
|
/// Create a numeric column accessor if TType is the type registered
|
|
|
|
/// in the specified branch info.
|
|
|
|
template <typename TResult, typename TType>
|
|
|
|
class NumericSourceCreator final
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
static bool execute(NumericSourcePtr<TResult> & source, const Block & block,
|
|
|
|
const ColumnNumbers & args, const Branch & br)
|
|
|
|
{
|
|
|
|
auto type_name = br.type->getName();
|
|
|
|
if (TypeName<TType>::get() == type_name)
|
|
|
|
{
|
2016-05-05 20:59:03 +00:00
|
|
|
if (br.is_const)
|
|
|
|
source = std::make_unique<NumericSourceImpl<TResult, TType, true> >(block, args, br);
|
|
|
|
else
|
|
|
|
source = std::make_unique<NumericSourceImpl<TResult, TType, false> >(block, args, br);
|
2016-04-18 14:31:32 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Processing of multiIf in the case of scalar numeric types.
|
|
|
|
template <typename TResult>
|
|
|
|
class NumericEvaluator final
|
|
|
|
{
|
|
|
|
public:
|
2016-08-18 14:25:02 +00:00
|
|
|
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result, NullMapBuilder & builder)
|
2016-04-18 14:31:32 +00:00
|
|
|
{
|
|
|
|
const CondSources conds = createConds(block, args);
|
|
|
|
const NumericSources<TResult> sources = createNumericSources(block, args, branches);
|
2016-04-22 10:21:52 +00:00
|
|
|
size_t row_count = conds[0].getSize();
|
2016-04-18 14:31:32 +00:00
|
|
|
PaddedPODArray<TResult> & res = createSink(block, result, row_count);
|
|
|
|
|
2016-08-18 14:25:02 +00:00
|
|
|
if (builder)
|
|
|
|
builder.init(args);
|
2016-08-04 15:06:27 +00:00
|
|
|
|
2016-04-18 14:31:32 +00:00
|
|
|
for (size_t cur_row = 0; cur_row < row_count; ++cur_row)
|
|
|
|
{
|
|
|
|
bool has_triggered_cond = false;
|
|
|
|
|
|
|
|
size_t cur_source = 0;
|
|
|
|
for (const auto & cond : conds)
|
|
|
|
{
|
|
|
|
if (cond.get(cur_row))
|
|
|
|
{
|
|
|
|
res[cur_row] = sources[cur_source]->get(cur_row);
|
2016-08-18 14:25:02 +00:00
|
|
|
if (builder)
|
|
|
|
builder.update(args[branches[cur_source].index], cur_row);
|
2016-04-18 14:31:32 +00:00
|
|
|
has_triggered_cond = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
++cur_source;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!has_triggered_cond)
|
2016-08-04 15:06:27 +00:00
|
|
|
{
|
2016-04-18 14:31:32 +00:00
|
|
|
res[cur_row] = sources.back()->get(cur_row);
|
2016-08-18 14:25:02 +00:00
|
|
|
if (builder)
|
|
|
|
builder.update(args[branches.back().index], cur_row);
|
2016-08-04 15:06:27 +00:00
|
|
|
}
|
2016-04-18 14:31:32 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
/// Create the result column.
|
|
|
|
static PaddedPODArray<TResult> & createSink(Block & block, size_t result, size_t size)
|
|
|
|
{
|
2016-05-28 05:31:36 +00:00
|
|
|
std::shared_ptr<ColumnVector<TResult>> col_res = std::make_shared<ColumnVector<TResult>>();
|
2016-04-18 14:31:32 +00:00
|
|
|
block.getByPosition(result).column = col_res;
|
|
|
|
|
|
|
|
typename ColumnVector<TResult>::Container_t & vec_res = col_res->getData();
|
|
|
|
vec_res.resize(size);
|
|
|
|
|
|
|
|
return vec_res;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Create accessors for condition values.
|
|
|
|
static CondSources createConds(const Block & block, const ColumnNumbers & args)
|
|
|
|
{
|
|
|
|
CondSources conds;
|
|
|
|
conds.reserve(getCondCount(args));
|
|
|
|
|
|
|
|
for (size_t i = firstCond(); i < elseArg(args); i = nextCond(i))
|
|
|
|
conds.emplace_back(block, args, i);
|
|
|
|
return conds;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Create accessors for branch values.
|
|
|
|
static NumericSources<TResult> createNumericSources(const Block & block,
|
|
|
|
const ColumnNumbers & args, const Branches & branches)
|
|
|
|
{
|
|
|
|
NumericSources<TResult> sources;
|
|
|
|
sources.reserve(branches.size());
|
|
|
|
|
|
|
|
for (const auto & br : branches)
|
|
|
|
{
|
|
|
|
NumericSourcePtr<TResult> source;
|
|
|
|
|
2016-05-03 23:19:14 +00:00
|
|
|
if (! (NumericSourceCreator<TResult, UInt8>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, UInt16>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, UInt32>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, UInt64>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, Int8>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, Int16>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, Int32>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, Int64>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, Float32>::execute(source, block, args, br)
|
2016-08-04 15:06:27 +00:00
|
|
|
|| NumericSourceCreator<TResult, Float64>::execute(source, block, args, br)
|
|
|
|
|| NumericSourceCreator<TResult, Null>::execute(source, block, args, br)))
|
2016-05-03 23:19:14 +00:00
|
|
|
throw CondException{CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT, toString(br.index)};
|
2016-04-18 14:31:32 +00:00
|
|
|
|
|
|
|
sources.push_back(std::move(source));
|
|
|
|
}
|
|
|
|
|
|
|
|
return sources;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
/// Processing of multiIf in the case of an invalid return type.
|
|
|
|
template <>
|
|
|
|
class NumericEvaluator<NumberTraits::Error>
|
|
|
|
{
|
|
|
|
public:
|
2016-08-18 14:25:02 +00:00
|
|
|
/// 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)
|
2016-04-18 14:31:32 +00:00
|
|
|
{
|
|
|
|
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|