Fixed DateTime64 creation, and optional scale parameter for now64()

Added more tests for casting and inserting values.
This commit is contained in:
Vasily Nemkov 2019-10-03 05:07:36 +03:00
parent 31d5730513
commit 655ec351ba
5 changed files with 81 additions and 63 deletions

View File

@ -87,7 +87,7 @@ std::string DataTypeDateTime::doGetName() const
void DataTypeDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeDateTimeText(assert_cast<const ColumnUInt32 &>(column).getData()[row_num], ostr, time_zone);
writeDateTimeText(assert_cast<const ColumnType &>(column).getData()[row_num], ostr, time_zone);
}
void DataTypeDateTime::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -104,7 +104,7 @@ void DataTypeDateTime::deserializeTextEscaped(IColumn & column, ReadBuffer & ist
{
time_t x;
::readText(x, istr, settings, time_zone, utc_time_zone);
assert_cast<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(column).getData().push_back(x);
}
void DataTypeDateTime::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -126,7 +126,7 @@ void DataTypeDateTime::deserializeTextQuoted(IColumn & column, ReadBuffer & istr
{
readIntText(x, istr);
}
assert_cast<ColumnUInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
assert_cast<ColumnType &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
}
void DataTypeDateTime::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -148,7 +148,7 @@ void DataTypeDateTime::deserializeTextJSON(IColumn & column, ReadBuffer & istr,
{
readIntText(x, istr);
}
assert_cast<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(column).getData().push_back(x);
}
void DataTypeDateTime::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -175,7 +175,7 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c
if (maybe_quote == '\'' || maybe_quote == '\"')
assertChar(maybe_quote, istr);
assert_cast<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(column).getData().push_back(x);
}
void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
@ -192,7 +192,7 @@ void DataTypeDateTime::deserializeProtobuf(IColumn & column, ProtobufReader & pr
if (!protobuf.readDateTime(t))
return;
auto & container = assert_cast<ColumnUInt32 &>(column).getData();
auto & container = assert_cast<ColumnType &>(column).getData();
if (allow_add_row)
{
container.emplace_back(t);
@ -209,7 +209,7 @@ bool DataTypeDateTime::equals(const IDataType & rhs) const
return typeid(rhs) == typeid(*this);
}
DataTypeDateTime64::DataTypeDateTime64(UInt8 scale_, const std::string & time_zone_name)
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name)
: DataTypeDecimalBase<DateTime64>(maxDecimalPrecision<DateTime64>() - scale_, scale_),
TimezoneMixin(time_zone_name)
{
@ -242,7 +242,7 @@ void DataTypeDateTime64::deserializeTextEscaped(IColumn & column, ReadBuffer & i
{
DateTime64 x;
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
assert_cast<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(column).getData().push_back(x);
}
void DataTypeDateTime64::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -264,7 +264,7 @@ void DataTypeDateTime64::deserializeTextQuoted(IColumn & column, ReadBuffer & is
{
readIntText(x, istr);
}
assert_cast<ColumnUInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
assert_cast<ColumnType &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
}
void DataTypeDateTime64::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -286,7 +286,7 @@ void DataTypeDateTime64::deserializeTextJSON(IColumn & column, ReadBuffer & istr
{
readIntText(x, istr);
}
assert_cast<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(column).getData().push_back(x);
}
void DataTypeDateTime64::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -313,7 +313,7 @@ void DataTypeDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & istr,
if (maybe_quote == '\'' || maybe_quote == '\"')
assertChar(maybe_quote, istr);
assert_cast<ColumnUInt32 &>(column).getData().push_back(x);
assert_cast<ColumnType &>(column).getData().push_back(x);
}
void DataTypeDateTime64::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
@ -368,51 +368,54 @@ static DataTypePtr create(const ASTPtr & arguments)
return std::make_shared<DataTypeDateTime>(arg->value.get<String>());
}
struct ArgumentSpec
enum class ArgumentKind
{
enum ArgumentKind
{
Optional,
Mandatory
};
size_t index;
const char * name;
ArgumentKind kind;
Optional,
Mandatory
};
template <typename T>
T getArgument(const ASTPtr & arguments, ArgumentSpec argument_spec, const std::string context_data_type_name)
template <typename T, ArgumentKind Kind>
std::conditional_t<Kind == ArgumentKind::Optional, std::optional<T>, T>
getArgument(const ASTPtr & arguments, size_t argument_index, const char * argument_name, const std::string context_data_type_name)
{
using NearestResultType = NearestFieldType<T>;
const auto fieldType = Field::TypeToEnum<NearestResultType>::value;
const ASTLiteral * argument = nullptr;
if (!arguments || arguments->children.size() <= argument_spec.index)
auto exceptionMessage = [=](const String & message)
{
if (argument_spec.kind == ArgumentSpec::Optional)
return std::string("Parameter #") + std::to_string(argument_index) + " '"
+ argument_name + "' for " + context_data_type_name
+ message
+ ", expected: " + Field::Types::toString(fieldType) + " literal.";
};
if (!arguments || arguments->children.size() <= argument_index
|| !(argument = arguments->children[argument_index]->as<ASTLiteral>()))
{
if constexpr (Kind == ArgumentKind::Optional)
return {};
else
throw Exception("Parameter #" + std::to_string(argument_spec.index) + "'" + argument_spec.name + "' for " + context_data_type_name + " is missing.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(exceptionMessage(" is missing"),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
const auto * argument = arguments->children[argument_spec.index]->as<ASTLiteral>();
if (!argument || argument->value.getType() != fieldType)
throw Exception("'" + std::string(argument_spec.name) + "' parameter for " +
context_data_type_name + " must be " + Field::Types::toString(fieldType) +
" literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (argument->value.getType() != fieldType)
throw Exception(exceptionMessage(String(" has wrong type: ") + argument->value.getTypeName()),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return argument->value.get<NearestResultType>();
}
static DataTypePtr create64(const ASTPtr & arguments)
{
if (!arguments)
if (!arguments || arguments->size() == 0)
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale);
const auto scale = getArgument<UInt64>(arguments, ArgumentSpec{0, "scale", ArgumentSpec::Mandatory}, "DateType64");
const auto timezone = getArgument<String>(arguments, ArgumentSpec{0, "timezone", ArgumentSpec::Optional}, "DateType64");
const auto scale = getArgument<UInt64, ArgumentKind::Optional>(arguments, 0, "scale", "DateType64");
const auto timezone = getArgument<String, ArgumentKind::Optional>(arguments, !!scale, "timezone", "DateType64");
return std::make_shared<DataTypeDateTime64>(scale, timezone);
return std::make_shared<DataTypeDateTime64>(scale.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{}));
}
void registerDataTypeDateTime(DataTypeFactory & factory)

View File

@ -83,7 +83,7 @@ class DataTypeDateTime64 final : public DataTypeDecimalBase<DateTime64>, public
public:
static constexpr UInt8 default_scale = 3;
explicit DataTypeDateTime64(UInt8 scale_, const std::string & time_zone_name = "");
explicit DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name = "");
const char * getFamilyName() const override { return "DateTime64"; }
std::string doGetName() const override;

View File

@ -61,24 +61,27 @@ public:
return name;
}
size_t getNumberOfArguments() const override { return 0; }
size_t getNumberOfArguments() const override { return 1; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return ColumnNumbers{0}; }
// Return type depends on argument value.
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
UInt64 scale = DataTypeDateTime64::default_scale;
// Type check is similar to the validateArgumentType, trying to keep error codes and messages as close to the said function as possible.
if (arguments.size() <= 1)
throw Exception("Incorrect number of arguments of function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments.size() >= 1)
{
const auto & argument = arguments[0];
if (!isInteger(argument.type) || !isColumnConst(*argument.column))
throw Exception("Illegal type " + argument.type->getName() +
" of 0" +
" argument of function " + getName() +
". Expected const integer.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto & argument = arguments[0];
if (!isInteger(argument.type) || !isColumnConst(*argument.column))
throw Exception("Illegal type " + argument.type->getName() +
" of 0" +
" argument of function " + getName() +
". Expected const integer.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const UInt64 scale = argument.column->get64(0);
scale = argument.column->get64(0);
}
return std::make_shared<DataTypeDateTime64>(scale);
}
@ -87,13 +90,18 @@ public:
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const IColumn * scale_column = block.getByPosition(arguments[0]).column.get();
if (!isColumnConst(*scale_column))
throw Exception("Unsupported argument type: " + scale_column->getName() +
+ " for function " + getName() + ". Expected const integer.",
ErrorCodes::ILLEGAL_COLUMN);
UInt64 scale = DataTypeDateTime64::default_scale;
if (arguments.size() == 1)
{
const IColumn * scale_column = block.getByPosition(arguments[0]).column.get();
if (!isColumnConst(*scale_column))
throw Exception("Unsupported argument type: " + scale_column->getName() +
+ " for function " + getName() + ". Expected const integer.",
ErrorCodes::ILLEGAL_COLUMN);
scale = scale_column->get64(0);
}
const UInt64 scale = scale_column->get64(0);
block.getByPosition(result).column = DataTypeDateTime64(scale).createColumnConst(input_rows_count, nowSubsecond(scale));
}
};

View File

@ -2,9 +2,13 @@ USE test;
DROP TABLE IF EXISTS A;
CREATE TABLE A(t DateTime64) ENGINE = MergeTree() ORDER BY t;
INSERT INTO A(t) VALUES (1556879125123456789);
INSERT INTO A(t) VALUES ('2019-05-03 11:25:25.123456789');
SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type
SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale
SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type
SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone
CREATE TABLE A(t DateTime64(3, 'UTC')) ENGINE = MergeTree() ORDER BY t;
INSERT INTO A(t) VALUES (1556879125123456789), ('2019-05-03 11:25:25.123456789'), (now64(3)), (now64(6)), (now64(0));
SELECT toString(t, 'UTC'), toDate(t), toStartOfDay(t), toStartOfQuarter(t), toTime(t), toStartOfMinute(t) FROM A ORDER BY t;

View File

@ -70,6 +70,7 @@ toYYYYMMDDhhmmss({datetime})
# subtractSeconds({datetime}, 1)
# subtractQuarters({datetime}, 1)
CAST({datetime} as DateTime)
CAST({datetime} as Date)
CAST({datetime} as UInt64)
formatDateTime({datetime}, '%C %d %D %e %F %H %I %j %m %M %n %p %R %S %t %T %u %V %w %y %Y %%')
""".splitlines()
@ -112,7 +113,7 @@ extra_ops =\
'> ',
'>='
],
'arg': ['now()'],
'arg': ['now()', 'toDate(now())'],
}
),
# With arithmetic types
@ -148,9 +149,11 @@ for f, args in extra_ops:
func = f.format(**dict(zip(args_keys, args_vals)))
functions.append(func)
datetime64_args = ['now64(0)', 'now64(3)', 'now64(6)', 'now64(9)', 'now64(12)', 'now64(15)']
# TODO: use string.Template here to allow lines that do not contain type, like: SELECT CAST(toDateTime64(1234567890), 'DateTime64')
for func in functions:
f = func.format(datetime="now64()")
print("""SELECT {function};""".format(function=f))
print("SELECT CAST( )")
for dt in datetime64_args:
f = func.format(datetime=dt)
print("""SELECT '{function}';""".format(function=f))
print("""SELECT {function};""".format(function=f))