mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Fixed DateTime64 creation, and optional scale parameter for now64()
Added more tests for casting and inserting values.
This commit is contained in:
parent
31d5730513
commit
655ec351ba
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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));
|
||||
}
|
||||
};
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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))
|
Loading…
Reference in New Issue
Block a user