mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 17:44:23 +00:00
Post-PR fixes
* More precise overflow check in readIntTextImpl * writeDateTimeText now always writes sub-second part for DateTime64 * comment for validateFunctionArgumentTypes * DateTime64-related fixes for FunctionConvertFromString * other minoe fixes: comments, removed commented out code, variable renamings, etc.
This commit is contained in:
parent
f56b2b64f2
commit
c781908e6d
@ -136,7 +136,7 @@ void validateArgumentsImpl(const IFunction & func,
|
|||||||
|
|
||||||
const auto & arg = arguments[i + argument_offset];
|
const auto & arg = arguments[i + argument_offset];
|
||||||
const auto validator = validators[i];
|
const auto validator = validators[i];
|
||||||
if (validator.validator_func(*arg.type) == false)
|
if (!validator.validator_func(*arg.type))
|
||||||
throw Exception("Illegal type " + arg.type->getName() +
|
throw Exception("Illegal type " + arg.type->getName() +
|
||||||
" of " + std::to_string(i) +
|
" of " + std::to_string(i) +
|
||||||
" argument of function " + func.getName() +
|
" argument of function " + func.getName() +
|
||||||
|
@ -24,17 +24,6 @@ const Type * checkAndGetDataType(const IDataType * data_type)
|
|||||||
return typeid_cast<const Type *>(data_type);
|
return typeid_cast<const Type *>(data_type);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename Type>
|
|
||||||
std::shared_ptr<const Type> checkAndGetDataTypePtr(const DataTypePtr & data_type)
|
|
||||||
{
|
|
||||||
if (typeid_cast<const Type *>(data_type.get()))
|
|
||||||
{
|
|
||||||
return std::static_pointer_cast<const Type>(data_type);
|
|
||||||
}
|
|
||||||
|
|
||||||
return std::shared_ptr<const Type>();
|
|
||||||
}
|
|
||||||
|
|
||||||
template <typename Type>
|
template <typename Type>
|
||||||
const ColumnConst * checkAndGetColumnConst(const IColumn * column)
|
const ColumnConst * checkAndGetColumnConst(const IColumn * column)
|
||||||
{
|
{
|
||||||
@ -109,6 +98,12 @@ struct FunctionArgumentTypeValidator
|
|||||||
|
|
||||||
using FunctionArgumentTypeValidators = std::vector<FunctionArgumentTypeValidator>;
|
using FunctionArgumentTypeValidators = std::vector<FunctionArgumentTypeValidator>;
|
||||||
|
|
||||||
|
/** Validate that function arguments match specification.
|
||||||
|
* first, check that mandatory args present and have valid type.
|
||||||
|
* second, check optional arguents types, skipping ones that are missing.
|
||||||
|
*
|
||||||
|
* If any mandatory arg is missing, throw an exception, with explicit description of expected arguments.
|
||||||
|
*/
|
||||||
void validateFunctionArgumentTypes(const IFunction & func, const ColumnsWithTypeAndName & arguments, const FunctionArgumentTypeValidators & mandatory_args, const FunctionArgumentTypeValidators & optional_args = {});
|
void validateFunctionArgumentTypes(const IFunction & func, const ColumnsWithTypeAndName & arguments, const FunctionArgumentTypeValidators & mandatory_args, const FunctionArgumentTypeValidators & optional_args = {});
|
||||||
|
|
||||||
/// Checks if a list of array columns have equal offsets. Return a pair of nested columns and offsets if true, otherwise throw.
|
/// Checks if a list of array columns have equal offsets. Return a pair of nested columns and offsets if true, otherwise throw.
|
||||||
|
@ -254,7 +254,7 @@ template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDateTime64
|
|||||||
template <typename Transform>
|
template <typename Transform>
|
||||||
struct FromDateTime64Transform
|
struct FromDateTime64Transform
|
||||||
{
|
{
|
||||||
static constexpr auto name = "toDateTime64";
|
static constexpr auto name = Transform::name;
|
||||||
|
|
||||||
const DateTime64::NativeType scale_multiplier = 1;
|
const DateTime64::NativeType scale_multiplier = 1;
|
||||||
|
|
||||||
@ -934,6 +934,7 @@ public:
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
// Optional second argument with time zone for DateTime.
|
||||||
UInt8 timezone_arg_position = 1;
|
UInt8 timezone_arg_position = 1;
|
||||||
UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale;
|
UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale;
|
||||||
|
|
||||||
@ -1079,8 +1080,7 @@ public:
|
|||||||
static constexpr bool to_decimal =
|
static constexpr bool to_decimal =
|
||||||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
|
||||||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
|
||||||
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>> ||
|
std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>;
|
||||||
std::is_same_v<ToDataType, DataTypeDateTime64>;
|
|
||||||
|
|
||||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvertFromString>(); }
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvertFromString>(); }
|
||||||
|
|
||||||
@ -1144,7 +1144,7 @@ public:
|
|||||||
res = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
|
res = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
|
||||||
else if constexpr (to_decimal)
|
else if constexpr (to_decimal)
|
||||||
{
|
{
|
||||||
UInt64 scale [[maybe_unused]] = extractToDecimalScale(arguments[1]);
|
UInt64 scale = extractToDecimalScale(arguments[1]);
|
||||||
|
|
||||||
if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>>)
|
if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>>)
|
||||||
res = createDecimal<DataTypeDecimal>(9, scale);
|
res = createDecimal<DataTypeDecimal>(9, scale);
|
||||||
@ -1156,6 +1156,12 @@ public:
|
|||||||
if (!res)
|
if (!res)
|
||||||
throw Exception("Someting wrong with toDecimalNNOrZero() or toDecimalNNOrNull()", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Someting wrong with toDecimalNNOrZero() or toDecimalNNOrNull()", ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
else if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||||
|
{
|
||||||
|
UInt64 scale = extractToDecimalScale(arguments[1]);
|
||||||
|
const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0);
|
||||||
|
res = std::make_shared<DataTypeDateTime64>(scale, timezone);
|
||||||
|
}
|
||||||
else
|
else
|
||||||
res = std::make_shared<ToDataType>();
|
res = std::make_shared<ToDataType>();
|
||||||
|
|
||||||
@ -1170,7 +1176,7 @@ public:
|
|||||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||||
|
|
||||||
bool ok = true;
|
bool ok = true;
|
||||||
if constexpr (to_decimal)
|
if constexpr (to_decimal || std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||||
{
|
{
|
||||||
if (arguments.size() != 2)
|
if (arguments.size() != 2)
|
||||||
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
|
throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include <common/LocalDate.h>
|
#include <common/LocalDate.h>
|
||||||
#include <common/LocalDateTime.h>
|
#include <common/LocalDateTime.h>
|
||||||
#include <common/StringRef.h>
|
#include <common/StringRef.h>
|
||||||
|
#include <common/arithmeticOverflow.h>
|
||||||
|
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
#include <Core/DecimalFunctions.h>
|
#include <Core/DecimalFunctions.h>
|
||||||
@ -256,14 +257,14 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
enum ReadIntTextCheckOverflow
|
enum class ReadIntTextCheckOverflow
|
||||||
{
|
{
|
||||||
READ_INT_DO_NOT_CHECK_OVERFLOW,
|
DO_NOT_CHECK_OVERFLOW,
|
||||||
READ_INT_CHECK_OVERFLOW,
|
CHECK_OVERFLOW,
|
||||||
};
|
};
|
||||||
|
|
||||||
template <typename T, typename ReturnType = void>
|
template <typename T, typename ReturnType = void, ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW>
|
||||||
ReturnType readIntTextImpl(T & x, ReadBuffer & buf, ReadIntTextCheckOverflow check_overflow = READ_INT_DO_NOT_CHECK_OVERFLOW)
|
ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
|
||||||
{
|
{
|
||||||
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
|
||||||
|
|
||||||
@ -277,7 +278,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf, ReadIntTextCheckOverflow che
|
|||||||
return ReturnType(false);
|
return ReturnType(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
size_t initial = buf.count();
|
const size_t initial_pos = buf.count();
|
||||||
while (!buf.eof())
|
while (!buf.eof())
|
||||||
{
|
{
|
||||||
switch (*buf.position())
|
switch (*buf.position())
|
||||||
@ -285,7 +286,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf, ReadIntTextCheckOverflow che
|
|||||||
case '+':
|
case '+':
|
||||||
break;
|
break;
|
||||||
case '-':
|
case '-':
|
||||||
if (is_signed_v<T>)
|
if constexpr (is_signed_v<T>)
|
||||||
negative = true;
|
negative = true;
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -305,44 +306,48 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf, ReadIntTextCheckOverflow che
|
|||||||
case '7': [[fallthrough]];
|
case '7': [[fallthrough]];
|
||||||
case '8': [[fallthrough]];
|
case '8': [[fallthrough]];
|
||||||
case '9':
|
case '9':
|
||||||
|
if constexpr (check_overflow == ReadIntTextCheckOverflow::CHECK_OVERFLOW)
|
||||||
|
{
|
||||||
|
// perform relativelly slow overflow check only when number of decimal digits so far is close to the max for given type.
|
||||||
|
if (buf.count() - initial_pos >= std::numeric_limits<T>::max_digits10)
|
||||||
|
{
|
||||||
|
if (common::mulOverflow(res, static_cast<decltype(res)>(10), res)
|
||||||
|
|| common::addOverflow(res, static_cast<decltype(res)>(*buf.position() - '0'), res))
|
||||||
|
return ReturnType(false);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
res *= 10;
|
res *= 10;
|
||||||
res += *buf.position() - '0';
|
res += *buf.position() - '0';
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
goto end;
|
goto end;
|
||||||
// x = negative ? -res : res;
|
|
||||||
// return ReturnType(true);
|
|
||||||
}
|
}
|
||||||
++buf.position();
|
++buf.position();
|
||||||
}
|
}
|
||||||
|
|
||||||
end:
|
end:
|
||||||
x = negative ? -res : res;
|
x = negative ? -res : res;
|
||||||
if (check_overflow && buf.count() - initial > std::numeric_limits<T>::digits10)
|
|
||||||
{
|
|
||||||
// the int literal is too big and x overflowed
|
|
||||||
return ReturnType(false);
|
|
||||||
}
|
|
||||||
|
|
||||||
return ReturnType(true);
|
return ReturnType(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
|
||||||
void readIntText(T & x, ReadBuffer & buf, ReadIntTextCheckOverflow check_overflow = READ_INT_DO_NOT_CHECK_OVERFLOW)
|
void readIntText(T & x, ReadBuffer & buf)
|
||||||
{
|
{
|
||||||
readIntTextImpl<T, void>(x, buf, check_overflow);
|
readIntTextImpl<T, void, check_overflow>(x, buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::CHECK_OVERFLOW, typename T>
|
||||||
bool tryReadIntText(T & x, ReadBuffer & buf, ReadIntTextCheckOverflow check_overflow = READ_INT_DO_NOT_CHECK_OVERFLOW)
|
bool tryReadIntText(T & x, ReadBuffer & buf)
|
||||||
{
|
{
|
||||||
return readIntTextImpl<T, bool>(x, buf, check_overflow);
|
return readIntTextImpl<T, bool, check_overflow>(x, buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::DO_NOT_CHECK_OVERFLOW, typename T>
|
||||||
void readIntText(Decimal<T> & x, ReadBuffer & buf, ReadIntTextCheckOverflow check_overflow = READ_INT_DO_NOT_CHECK_OVERFLOW)
|
void readIntText(Decimal<T> & x, ReadBuffer & buf)
|
||||||
{
|
{
|
||||||
readIntText(x.value, buf, check_overflow);
|
readIntText<check_overflow>(x.value, buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** More efficient variant (about 1.5 times on real dataset).
|
/** More efficient variant (about 1.5 times on real dataset).
|
||||||
@ -642,7 +647,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
/// Why not readIntTextUnsafe? Because for needs of AdFox, parsing of unix timestamp with leading zeros is supported: 000...NNNN.
|
/// Why not readIntTextUnsafe? Because for needs of AdFox, parsing of unix timestamp with leading zeros is supported: 000...NNNN.
|
||||||
return readIntTextImpl<time_t, ReturnType>(datetime, buf, READ_INT_CHECK_OVERFLOW);
|
return readIntTextImpl<time_t, ReturnType, ReadIntTextCheckOverflow::CHECK_OVERFLOW>(datetime, buf);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
return readDateTimeTextFallback<ReturnType>(datetime, buf, date_lut);
|
return readDateTimeTextFallback<ReturnType>(datetime, buf, date_lut);
|
||||||
@ -662,8 +667,8 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
|
|||||||
if (!buf.eof() && *buf.position() == '.')
|
if (!buf.eof() && *buf.position() == '.')
|
||||||
{
|
{
|
||||||
buf.ignore(1); // skip separator
|
buf.ignore(1); // skip separator
|
||||||
const auto count1 = buf.count();
|
const auto pos_before_fractional = buf.count();
|
||||||
if (!tryReadIntText(c.fractional, buf, READ_INT_CHECK_OVERFLOW))
|
if (!tryReadIntText<ReadIntTextCheckOverflow::CHECK_OVERFLOW>(c.fractional, buf))
|
||||||
{
|
{
|
||||||
return ReturnType(false);
|
return ReturnType(false);
|
||||||
}
|
}
|
||||||
@ -674,7 +679,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re
|
|||||||
|
|
||||||
// If scale is 3, but we read '12', promote fractional part to '120'.
|
// If scale is 3, but we read '12', promote fractional part to '120'.
|
||||||
// And vice versa: if we read '1234', denote it to '123'.
|
// And vice versa: if we read '1234', denote it to '123'.
|
||||||
const auto fractional_length = static_cast<Int32>(buf.count() - count1);
|
const auto fractional_length = static_cast<Int32>(buf.count() - pos_before_fractional);
|
||||||
if (const auto adjust_scale = static_cast<Int32>(scale) - fractional_length; adjust_scale > 0)
|
if (const auto adjust_scale = static_cast<Int32>(scale) - fractional_length; adjust_scale > 0)
|
||||||
{
|
{
|
||||||
c.fractional *= common::exp10_i64(adjust_scale);
|
c.fractional *= common::exp10_i64(adjust_scale);
|
||||||
@ -935,11 +940,11 @@ void readAndThrowException(ReadBuffer & buf, const String & additional_message =
|
|||||||
|
|
||||||
/** Helper function for implementation.
|
/** Helper function for implementation.
|
||||||
*/
|
*/
|
||||||
template <typename T>
|
template <ReadIntTextCheckOverflow check_overflow = ReadIntTextCheckOverflow::CHECK_OVERFLOW, typename T>
|
||||||
static inline const char * tryReadIntText(T & x, const char * pos, const char * end)
|
static inline const char * tryReadIntText(T & x, const char * pos, const char * end)
|
||||||
{
|
{
|
||||||
ReadBufferFromMemory in(pos, end - pos);
|
ReadBufferFromMemory in(pos, end - pos);
|
||||||
tryReadIntText(x, in);
|
tryReadIntText<check_overflow>(x, in);
|
||||||
return pos + in.count();
|
return pos + in.count();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -732,7 +732,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer &
|
|||||||
LocalDateTime(values.year, values.month, values.day_of_month,
|
LocalDateTime(values.year, values.month, values.day_of_month,
|
||||||
date_lut.toHour(c.whole), date_lut.toMinute(c.whole), date_lut.toSecond(c.whole)), buf);
|
date_lut.toHour(c.whole), date_lut.toMinute(c.whole), date_lut.toSecond(c.whole)), buf);
|
||||||
|
|
||||||
if (scale > 0 && c.fractional)
|
if (scale > 0)
|
||||||
{
|
{
|
||||||
buf.write(fractional_time_delimiter);
|
buf.write(fractional_time_delimiter);
|
||||||
|
|
||||||
@ -740,7 +740,7 @@ inline void writeDateTimeText(DateTime64 datetime64, UInt32 scale, WriteBuffer &
|
|||||||
static_assert(sizeof(data) >= MaxScale);
|
static_assert(sizeof(data) >= MaxScale);
|
||||||
|
|
||||||
auto fractional = c.fractional;
|
auto fractional = c.fractional;
|
||||||
for (Int32 pos = scale - 1; pos >= 0; --pos, fractional /= DateTime64(10))
|
for (Int32 pos = scale - 1; pos >= 0 && fractional; --pos, fractional /= DateTime64(10))
|
||||||
data[pos] += fractional % DateTime64(10);
|
data[pos] += fractional % DateTime64(10);
|
||||||
|
|
||||||
writeString(&data[0], static_cast<size_t>(scale), buf);
|
writeString(&data[0], static_cast<size_t>(scale), buf);
|
||||||
|
@ -1,3 +1,3 @@
|
|||||||
2019-09-16 19:20:11
|
2019-09-16 19:20:11.000
|
||||||
2019-05-03 11:25:25.123 2019-05-03 2019-05-02 21:00:00 2019-04-01 1970-01-02 11:25:25 2019-05-03 11:25:00
|
2019-05-03 11:25:25.123 2019-05-03 2019-05-02 21:00:00 2019-04-01 1970-01-02 11:25:25 2019-05-03 11:25:00
|
||||||
2019-09-16 19:20:11.234
|
2019-09-16 19:20:11.234
|
||||||
|
Loading…
Reference in New Issue
Block a user