Bit shift operations for FixedString and String data types.

This commit is contained in:
zhangxiao871 2021-08-30 14:37:23 +08:00
parent 243b845dc0
commit 3b39b82fb7
35 changed files with 298 additions and 15 deletions

View File

@ -72,6 +72,7 @@ struct DivideIntegralImpl
{
using ResultType = typename NumberTraits::ResultOfIntegerDivision<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -126,6 +127,7 @@ struct ModuloImpl
using IntegerBType = typename NumberTraits::ToInteger<B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -15,11 +15,13 @@
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/Native.h>
#include <DataTypes/NumberTraits.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnAggregateFunction.h>
#include "Core/DecimalFunctions.h"
@ -197,6 +199,7 @@ struct BinaryOperation
{
using ResultType = OpResultType;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <OpCase op_case>
static void NO_INLINE process(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size)
@ -213,6 +216,80 @@ struct BinaryOperation
static ResultType process(A a, B b) { return Op::template apply<ResultType>(a, b); }
};
template <typename B, typename Op>
struct StringIntegerOperationImpl
{
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = true;
template <OpCase op_case, bool is_fixed_string>
static void NO_INLINE process(const UInt8 * __restrict in_vec, const UInt64 * __restrict n_or_in_offsets, const B * __restrict b, ColumnString::Chars & out_vec, ColumnString::Offsets & out_offsets, size_t size)
{
if constexpr (is_fixed_string)
processFixedString<op_case>(in_vec, *n_or_in_offsets, b, out_vec, out_offsets, size);
else
processString<op_case>(in_vec, n_or_in_offsets, b, out_vec, out_offsets, size);
}
template <OpCase op_case>
static void NO_INLINE processFixedString(const UInt8 * __restrict in_vec, const UInt64 n, const B * __restrict b, ColumnString::Chars & out_vec, ColumnString::Offsets & out_offsets, size_t size)
{
size_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if constexpr (op_case == OpCase::LeftConstant)
{
Op::apply(&in_vec[0], &in_vec[n], b[i], out_vec, out_offsets);
}
else
{
size_t new_offset = prev_offset + n;
if constexpr (op_case == OpCase::Vector)
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset], b[i], out_vec, out_offsets);
}
else
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset], b[0], out_vec, out_offsets);
}
prev_offset = new_offset;
}
}
}
/// if is_fixed_string n_or_in_offsets is n, otherwise n_or_in_offsets is in_offsets
template <OpCase op_case>
static void NO_INLINE processString(const UInt8 * __restrict in_vec, const UInt64 * __restrict in_offsets, const B * __restrict b, ColumnString::Chars & out_vec, ColumnString::Offsets & out_offsets, size_t size)
{
size_t prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
if constexpr (op_case == OpCase::LeftConstant)
{
Op::apply(&in_vec[0], &in_vec[in_offsets[0] - 1], b[i], out_vec, out_offsets);
}
else
{
size_t new_offset = in_offsets[i];
if constexpr (op_case == OpCase::Vector)
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset - 1], b[i], out_vec, out_offsets);
}
else
{
Op::apply(&in_vec[prev_offset], &in_vec[new_offset - 1], b[0], out_vec, out_offsets);
}
prev_offset = new_offset;
}
}
}
};
template <typename Op>
struct FixedStringOperationImpl
{
@ -540,7 +617,8 @@ class FunctionBinaryArithmetic : public IFunction
DataTypeDecimal<Decimal64>,
DataTypeDecimal<Decimal128>,
DataTypeDecimal<Decimal256>,
DataTypeFixedString
DataTypeFixedString,
DataTypeString
>(type, std::forward<F>(f));
}
@ -566,7 +644,8 @@ class FunctionBinaryArithmetic : public IFunction
DataTypeDecimal<Decimal64>,
DataTypeDecimal<Decimal128>,
DataTypeDecimal<Decimal256>,
DataTypeFixedString
DataTypeFixedString,
DataTypeString
>(type, std::forward<F>(f));
}
@ -1012,19 +1091,31 @@ public:
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> ||
std::is_same_v<DataTypeFixedString, RightDataType>)
if constexpr ((std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeString, LeftDataType>) ||
(std::is_same_v<DataTypeFixedString, RightDataType> || std::is_same_v<DataTypeString, RightDataType>))
{
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else if constexpr (std::is_same_v<LeftDataType, RightDataType>)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> &&
std::is_same_v<DataTypeFixedString, RightDataType>)
{
if (left.getN() == right.getN())
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else
{
type_res = std::make_shared<LeftDataType>(left.getN());
return true;
if (left.getN() == right.getN())
{
type_res = std::make_shared<LeftDataType>(left.getN());
return true;
}
}
}
if constexpr (!Op<LeftDataType, RightDataType>::allow_string_integer)
return false;
else if constexpr (!IsIntegral<RightDataType>)
return false;
else
type_res = std::make_shared<DataTypeString>();
return true;
}
else
{
@ -1161,6 +1252,99 @@ public:
return nullptr;
}
template <typename LeftColumnType, typename A, typename B>
ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A & left, const B & right) const
{
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
using ResultDataType = DataTypeString;
const auto * const col_left_raw = arguments[0].column.get();
const auto * const col_right_raw = arguments[1].column.get();
using T1 = typename RightDataType::FieldType;
using ColVecT1 = ColumnVector<T1>;
const ColVecT1 * const col_right = checkAndGetColumn<ColVecT1>(col_right_raw);
const ColumnConst * const col_right_const = checkAndGetColumnConst<ColVecT1>(col_right_raw);
ColumnString::MutablePtr col_res = ColumnString::create();
ColumnString::Chars & out_vec = col_res->getChars();
ColumnString::Offsets & out_offsets = col_res->getOffsets();
using OpImpl = StringIntegerOperationImpl<T1, Op<LeftDataType, T1>>;
const ColumnConst * const col_left_const = checkAndGetColumnConst<LeftColumnType>(col_left_raw);
const auto * col_left = col_left_const ? checkAndGetColumn<LeftColumnType>(col_left_const->getDataColumn())
: checkAndGetColumn<LeftColumnType>(col_left_raw);
const typename LeftColumnType::Chars & in_vec = col_left->getChars();
UInt64 n;
const UInt64 * n_or_in_offsets;
if constexpr (std::is_same_v<LeftDataType, DataTypeFixedString>)
{
n = col_left->getN();
n_or_in_offsets = &n;
}
else
n_or_in_offsets = col_left->getOffsets().data();
if (col_left_const && col_right_const)
{
const T1 value = col_right_const->template getValue<T1>();
OpImpl::template process<OpCase::Vector, std::is_same_v<LeftDataType, DataTypeFixedString>>(
in_vec.data(),
n_or_in_offsets,
&value,
out_vec,
out_offsets,
1);
return ResultDataType().createColumnConst(col_left_const->size(), col_res->size() ? (*col_res)[0] : Field(""));
}
else if (!col_left_const && !col_right_const && col_left && col_right)
{
out_offsets.reserve(col_left->size());
OpImpl::template process<OpCase::Vector, std::is_same_v<LeftDataType, DataTypeFixedString>>(
in_vec.data(),
n_or_in_offsets,
col_right->getData().data(),
out_vec,
out_offsets,
col_left->size());
}
else if (col_left_const && col_right)
{
out_offsets.reserve(col_right->size());
OpImpl::template process<OpCase::LeftConstant, std::is_same_v<LeftDataType, DataTypeFixedString>>(
in_vec.data(),
n_or_in_offsets,
col_right->getData().data(),
out_vec,
out_offsets,
col_right->size());
}
else if (col_left && col_right_const)
{
const T1 value = col_right_const->template getValue<T1>();
OpImpl::template process<OpCase::RightConstant, std::is_same_v<LeftDataType, DataTypeFixedString>>(
in_vec.data(),
n_or_in_offsets,
&value,
out_vec,
out_offsets,
col_left->size());
}
else
return nullptr;
return col_res;
}
template <typename A, typename B>
ColumnPtr executeNumeric(const ColumnsWithTypeAndName & arguments, const A & left, const B & right) const
{
@ -1287,13 +1471,28 @@ public:
using LeftDataType = std::decay_t<decltype(left)>;
using RightDataType = std::decay_t<decltype(right)>;
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> ||
std::is_same_v<DataTypeFixedString, RightDataType>)
if constexpr ((std::is_same_v<DataTypeFixedString, LeftDataType> || std::is_same_v<DataTypeString, LeftDataType>) ||
(std::is_same_v<DataTypeFixedString, RightDataType> || std::is_same_v<DataTypeString, RightDataType>))
{
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType> &&
std::is_same_v<DataTypeFixedString, RightDataType>)
{
if constexpr (!Op<DataTypeFixedString, DataTypeFixedString>::allow_fixed_string)
return false;
else
return (res = executeFixedString(arguments)) != nullptr;
}
if constexpr (!Op<LeftDataType, RightDataType>::allow_string_integer)
return false;
else
return (res = executeFixedString(arguments)) != nullptr;
else if constexpr (!IsIntegral<RightDataType>)
return false;
else if constexpr (std::is_same_v<DataTypeFixedString, LeftDataType>)
{
return (res = executeStringInteger<ColumnFixedString>(arguments, left, right)) != nullptr;
}
else if constexpr (std::is_same_v<DataTypeString, LeftDataType>)
return (res = executeStringInteger<ColumnString>(arguments, left, right)) != nullptr;
}
else
return (res = executeNumeric(arguments, left, right)) != nullptr;

View File

@ -25,6 +25,7 @@ struct GCDLCMImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -11,6 +11,7 @@ struct AbsImpl
{
using ResultType = std::conditional_t<IsDecimalNumber<A>, A, typename NumberTraits::ResultOfAbs<A>::Type>;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{

View File

@ -17,6 +17,7 @@ struct BitAndImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static constexpr const bool allow_fixed_string = true;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -22,6 +22,7 @@ struct BitBoolMaskAndImpl
{
using ResultType = UInt8;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply([[maybe_unused]] A left, [[maybe_unused]] B right)

View File

@ -22,6 +22,7 @@ struct BitBoolMaskOrImpl
{
using ResultType = UInt8;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply([[maybe_unused]] A left, [[maybe_unused]] B right)

View File

@ -14,6 +14,7 @@ struct BitCountImpl
{
using ResultType = UInt8;
static constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType apply(A a)
{

View File

@ -18,6 +18,7 @@ struct BitNotImpl
{
using ResultType = typename NumberTraits::ResultOfBitNot<A>::Type;
static const constexpr bool allow_fixed_string = true;
static const constexpr bool allow_string_integer = false;
static inline ResultType apply(A a)
{

View File

@ -16,6 +16,7 @@ struct BitOrImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static constexpr const bool allow_fixed_string = true;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -17,6 +17,7 @@ struct BitRotateLeftImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a [[maybe_unused]], B b [[maybe_unused]])

View File

@ -17,6 +17,7 @@ struct BitRotateRightImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a [[maybe_unused]], B b [[maybe_unused]])

View File

@ -17,6 +17,7 @@ struct BitShiftLeftImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a [[maybe_unused]], B b [[maybe_unused]])

View File

@ -1,5 +1,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionBinaryArithmetic.h>
#include <Common/hex.h>
namespace DB
{
@ -17,6 +18,7 @@ struct BitShiftRightImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = true;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a [[maybe_unused]], B b [[maybe_unused]])
@ -29,6 +31,51 @@ struct BitShiftRightImpl
return static_cast<Result>(a) >> static_cast<Result>(b);
}
static inline NO_SANITIZE_UNDEFINED void apply(const UInt8 * pos [[maybe_unused]], const UInt8 * end [[maybe_unused]], const B & b [[maybe_unused]], ColumnString::Chars & out_vec, ColumnString::Offsets & out_offsets)
{
if constexpr (is_big_int_v<B>)
throw Exception("BitShiftRight is not implemented for big integers as second argument", ErrorCodes::NOT_IMPLEMENTED);
else
{
UInt8 word_size = 8;
if (b >= static_cast<B>((end - pos) * word_size))
{
// insert default value
out_vec.push_back(0);
out_offsets.push_back(out_offsets.back() + 1);
return;
}
size_t shift_right_bytes = b / word_size;
size_t shift_right_bits = b % word_size;
const UInt8 * begin = pos;
const UInt8 * shift_right_end = end - shift_right_bytes;
const size_t old_size = out_vec.size();
size_t length = shift_right_end - begin;
const size_t new_size = old_size + length + 1;
out_vec.resize(new_size);
UInt8 * op_pointer = const_cast<UInt8 *>(shift_right_end);
out_vec[old_size + length] = 0;
UInt8 * out = out_vec.data() + old_size + length;
while (op_pointer > begin)
{
op_pointer--;
out--;
UInt8 temp_value = *op_pointer >> shift_right_bits;
if (op_pointer - 1 >= begin)
{
*out = UInt8(UInt8(*(op_pointer - 1) << (8 - shift_right_bits)) | temp_value);
}
else
*out = temp_value;
}
out_offsets.push_back(new_size);
}
}
#if USE_EMBEDDED_COMPILER
static constexpr bool compilable = true;
@ -41,6 +88,7 @@ struct BitShiftRightImpl
#endif
};
struct NameBitShiftRight { static constexpr auto name = "bitShiftRight"; };
using FunctionBitShiftRight = BinaryArithmeticOverloadResolver<BitShiftRightImpl, NameBitShiftRight, true, false>;

View File

@ -20,6 +20,7 @@ struct BitSwapLastTwoImpl
{
using ResultType = UInt8;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType NO_SANITIZE_UNDEFINED apply([[maybe_unused]] A a)
{

View File

@ -18,6 +18,7 @@ struct BitTestImpl
{
using ResultType = UInt8;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
NO_SANITIZE_UNDEFINED static inline Result apply(A a [[maybe_unused]], B b [[maybe_unused]])

View File

@ -20,6 +20,7 @@ struct BitWrapperFuncImpl
{
using ResultType = UInt8;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType NO_SANITIZE_UNDEFINED apply(A a [[maybe_unused]])
{

View File

@ -16,6 +16,7 @@ struct BitXorImpl
{
using ResultType = typename NumberTraits::ResultOfBit<A, B>::Type;
static constexpr bool allow_fixed_string = true;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -13,6 +13,7 @@ struct DivideFloatingImpl
{
using ResultType = typename NumberTraits::ResultOfFloatingPointDivision<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a [[maybe_unused]], B b [[maybe_unused]])

View File

@ -12,6 +12,7 @@ struct GreatestBaseImpl
{
using ResultType = NumberTraits::ResultOfGreatest<A, B>;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -42,6 +43,7 @@ struct GreatestSpecialImpl
{
using ResultType = make_unsigned_t<A>;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -23,6 +23,7 @@ struct DivideIntegralByConstantImpl
using Op = DivideIntegralImpl<A, B>;
using ResultType = typename Op::ResultType;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <OpCase op_case>
static void NO_INLINE process(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size)

View File

@ -10,6 +10,7 @@ struct DivideIntegralOrZeroImpl
{
using ResultType = typename NumberTraits::ResultOfIntegerDivision<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -18,6 +18,7 @@ struct IntExp10Impl
{
using ResultType = UInt64;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType apply([[maybe_unused]] A a)
{

View File

@ -19,6 +19,7 @@ struct IntExp2Impl
{
using ResultType = UInt64;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType apply([[maybe_unused]] A a)
{

View File

@ -12,6 +12,7 @@ struct LeastBaseImpl
{
using ResultType = NumberTraits::ResultOfLeast<A, B>;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)
@ -42,6 +43,7 @@ struct LeastSpecialImpl
{
using ResultType = std::make_signed_t<A>;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -10,6 +10,7 @@ struct MinusImpl
{
using ResultType = typename NumberTraits::ResultOfSubtraction<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -27,6 +27,7 @@ struct ModuloByConstantImpl
using Op = ModuloImpl<A, B>;
using ResultType = typename Op::ResultType;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <OpCase op_case>
static void NO_INLINE process(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size)

View File

@ -12,6 +12,7 @@ struct ModuloOrZeroImpl
{
using ResultType = typename NumberTraits::ResultOfModulo<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline Result apply(A a, B b)

View File

@ -11,6 +11,7 @@ struct MultiplyImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
template <typename Result = ResultType>
static inline NO_SANITIZE_UNDEFINED Result apply(A a, B b)

View File

@ -10,6 +10,7 @@ struct NegateImpl
{
using ResultType = std::conditional_t<IsDecimalNumber<A>, A, typename NumberTraits::ResultOfNegate<A>::Type>;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{

View File

@ -10,6 +10,7 @@ struct PlusImpl
{
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<A, B>::Type;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static const constexpr bool is_commutative = true;
template <typename Result = ResultType>

View File

@ -11,6 +11,7 @@ struct RoundAgeImpl
{
using ResultType = UInt8;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType apply(A x)
{

View File

@ -11,6 +11,7 @@ struct RoundDurationImpl
{
using ResultType = UInt16;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline ResultType apply(A x)
{

View File

@ -64,6 +64,7 @@ struct RoundToExp2Impl
{
using ResultType = T;
static constexpr const bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline T apply(T x)
{

View File

@ -10,6 +10,7 @@ struct SignImpl
{
using ResultType = Int8;
static const constexpr bool allow_fixed_string = false;
static const constexpr bool allow_string_integer = false;
static inline NO_SANITIZE_UNDEFINED ResultType apply(A a)
{