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:
Vasily Nemkov 2019-12-11 11:56:32 +03:00
parent f56b2b64f2
commit c781908e6d
6 changed files with 55 additions and 49 deletions

View File

@ -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() +

View File

@ -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.

View File

@ -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};

View File

@ -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();
} }

View File

@ -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);

View File

@ -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