mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
Moved DataTypeDateTime64 and now64() to separate source files.
This commit is contained in:
parent
8648ebf75c
commit
db166af95a
@ -3,7 +3,6 @@
|
||||
#include <utility>
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -151,6 +150,8 @@ class DataTypeDate;
|
||||
class DataTypeString;
|
||||
class DataTypeFixedString;
|
||||
class DataTypeUUID;
|
||||
class DataTypeDateTime;
|
||||
class DataTypeDateTime64;
|
||||
template <typename T> class DataTypeEnum;
|
||||
template <typename T> class DataTypeNumber;
|
||||
template <typename T> class DataTypeDecimal;
|
||||
|
@ -17,7 +17,7 @@
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <iomanip>
|
||||
#include <optional>
|
||||
|
||||
namespace
|
||||
{
|
||||
@ -34,49 +34,23 @@ static inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
static inline void readText(DateTime64 & x, UInt32 scale, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone)
|
||||
{
|
||||
switch (settings.date_time_input_format)
|
||||
{
|
||||
case FormatSettings::DateTimeInputFormat::Basic:
|
||||
readDateTime64Text(x, scale, istr, time_zone);
|
||||
return;
|
||||
case FormatSettings::DateTimeInputFormat::BestEffort:
|
||||
parseDateTime64BestEffort(x, scale, istr, time_zone, utc_time_zone);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
bool protobufReadDateTime(ProtobufReader & protobuf, T & date_time)
|
||||
{
|
||||
return protobuf.readDateTime(date_time);
|
||||
}
|
||||
|
||||
template <>
|
||||
bool protobufReadDateTime<DateTime64>(ProtobufReader & protobuf, DateTime64 & date_time)
|
||||
{
|
||||
// TODO (vnemkov): protobuf.readDecimal ?
|
||||
return protobuf.readDateTime(date_time.value);
|
||||
}
|
||||
|
||||
TimezoneMixin::TimezoneMixin(const std::string & time_zone_name)
|
||||
TimezoneMixin::TimezoneMixin(const String & time_zone_name)
|
||||
: has_explicit_time_zone(!time_zone_name.empty()),
|
||||
time_zone(DateLUT::instance(time_zone_name)),
|
||||
utc_time_zone(DateLUT::instance("UTC"))
|
||||
{}
|
||||
|
||||
DataTypeDateTime::DataTypeDateTime(const std::string & time_zone_name)
|
||||
DataTypeDateTime::DataTypeDateTime(const String & time_zone_name)
|
||||
: TimezoneMixin(time_zone_name)
|
||||
{
|
||||
}
|
||||
|
||||
std::string DataTypeDateTime::doGetName() const
|
||||
String DataTypeDateTime::doGetName() const
|
||||
{
|
||||
if (!has_explicit_time_zone)
|
||||
return "DateTime";
|
||||
@ -210,149 +184,6 @@ bool DataTypeDateTime::equals(const IDataType & rhs) const
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name)
|
||||
: DataTypeDecimalBase<DateTime64>(maxDecimalPrecision<DateTime64>() - scale_, scale_),
|
||||
TimezoneMixin(time_zone_name)
|
||||
{
|
||||
}
|
||||
|
||||
std::string DataTypeDateTime64::doGetName() const
|
||||
{
|
||||
if (!has_explicit_time_zone)
|
||||
return std::string(getFamilyName()) + "(" + std::to_string(this->scale) + ")";
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
out << "DateTime64(" << this->scale << ", " << quote << time_zone.getTimeZone() << ")";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & /*settings*/) const
|
||||
{
|
||||
writeDateTimeText(assert_cast<const ColumnType &>(column).getData()[row_num], scale, ostr, time_zone);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeText(IColumn & /*column*/, ReadBuffer & /*istr*/, const FormatSettings &) const
|
||||
{}
|
||||
|
||||
void DataTypeDateTime64::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextEscaped(column, istr, settings);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('\'', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('\'', ostr);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808'
|
||||
{
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
assertChar('\'', istr);
|
||||
}
|
||||
else /// Just 1504193808 or 01504193808
|
||||
{
|
||||
readIntText(x, istr);
|
||||
}
|
||||
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
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
if (checkChar('"', istr))
|
||||
{
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
assertChar('"', istr);
|
||||
}
|
||||
else
|
||||
{
|
||||
readIntText(x, istr);
|
||||
}
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
|
||||
if (istr.eof())
|
||||
throwReadAfterEOF();
|
||||
|
||||
char maybe_quote = *istr.position();
|
||||
|
||||
if (maybe_quote == '\'' || maybe_quote == '\"')
|
||||
++istr.position();
|
||||
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
|
||||
if (maybe_quote == '\'' || maybe_quote == '\"')
|
||||
assertChar(maybe_quote, istr);
|
||||
|
||||
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
|
||||
{
|
||||
if (value_index)
|
||||
return;
|
||||
value_index = static_cast<bool>(protobuf.writeDateTime64(assert_cast<const ColumnType &>(column).getData()[row_num], scale));
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const
|
||||
{
|
||||
row_added = false;
|
||||
DateTime64 t;
|
||||
if (!protobuf.readDateTime64(t, scale))
|
||||
return;
|
||||
|
||||
auto & container = assert_cast<ColumnType &>(column).getData();
|
||||
if (allow_add_row)
|
||||
{
|
||||
container.emplace_back(t);
|
||||
row_added = true;
|
||||
}
|
||||
else
|
||||
container.back() = t;
|
||||
}
|
||||
|
||||
bool DataTypeDateTime64::equals(const IDataType & rhs) const
|
||||
{
|
||||
if (auto * ptype = typeid_cast<const DataTypeDateTime64 *>(&rhs))
|
||||
return this->scale == ptype->getScale();
|
||||
return false;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
@ -374,61 +205,9 @@ static DataTypePtr create(const ASTPtr & arguments)
|
||||
return std::make_shared<DataTypeDateTime>(arg->value.get<String>());
|
||||
}
|
||||
|
||||
enum class ArgumentKind
|
||||
{
|
||||
Optional,
|
||||
Mandatory
|
||||
};
|
||||
|
||||
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;
|
||||
|
||||
auto exceptionMessage = [=](const String & message)
|
||||
{
|
||||
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(exceptionMessage(" is missing"),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
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 || arguments->size() == 0)
|
||||
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale);
|
||||
|
||||
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.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{}));
|
||||
}
|
||||
|
||||
void registerDataTypeDateTime(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive);
|
||||
factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive);
|
||||
|
||||
factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
|
@ -2,23 +2,16 @@
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
|
||||
class DateLUTImpl;
|
||||
|
||||
//template <class, template <class, class...> class>
|
||||
//struct is_instance : public std::false_type {};
|
||||
|
||||
//template <class...Ts, template <class, class...> class U>
|
||||
//struct is_instance<U<Ts...>, U> : public std::true_type {};
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TimezoneMixin
|
||||
{
|
||||
public:
|
||||
explicit TimezoneMixin(const std::string & time_zone_name = "");
|
||||
explicit TimezoneMixin(const String & time_zone_name = "");
|
||||
|
||||
const DateLUTImpl & getTimeZone() const { return time_zone; }
|
||||
|
||||
@ -51,10 +44,10 @@ protected:
|
||||
class DataTypeDateTime final : public DataTypeNumberBase<UInt32>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
explicit DataTypeDateTime(const std::string & time_zone_name = "");
|
||||
explicit DataTypeDateTime(const String & time_zone_name = "");
|
||||
|
||||
const char * getFamilyName() const override { return "DateTime"; }
|
||||
std::string doGetName() const override;
|
||||
String doGetName() const override;
|
||||
TypeIndex getTypeId() const override { return TypeIndex::DateTime; }
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
@ -76,39 +69,5 @@ public:
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
};
|
||||
|
||||
/** DateTime64 is same as DateTime, but it stores values as UInt64 and has configurable sub-second part.
|
||||
*
|
||||
* `scale` determines number of decimal places for sub-second part of the DateTime64.
|
||||
*/
|
||||
class DataTypeDateTime64 final : public DataTypeDecimalBase<DateTime64>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
static constexpr UInt8 default_scale = 3;
|
||||
|
||||
explicit DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name = "");
|
||||
|
||||
const char * getFamilyName() const override { return "DateTime64"; }
|
||||
std::string doGetName() const override;
|
||||
TypeIndex getTypeId() const override { return TypeIndex::DateTime64; }
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
};
|
||||
|
||||
template <> inline constexpr bool IsDataTypeDecimal<DataTypeDateTime64> = true;
|
||||
|
||||
}
|
||||
|
||||
|
251
dbms/src/DataTypes/DataTypeDateTime64.cpp
Normal file
251
dbms/src/DataTypes/DataTypeDateTime64.cpp
Normal file
@ -0,0 +1,251 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <iomanip>
|
||||
|
||||
namespace
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
static inline void readText(DateTime64 & x, UInt32 scale, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & utc_time_zone)
|
||||
{
|
||||
switch (settings.date_time_input_format)
|
||||
{
|
||||
case FormatSettings::DateTimeInputFormat::Basic:
|
||||
readDateTime64Text(x, scale, istr, time_zone);
|
||||
return;
|
||||
case FormatSettings::DateTimeInputFormat::BestEffort:
|
||||
parseDateTime64BestEffort(x, scale, istr, time_zone, utc_time_zone);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name)
|
||||
: DataTypeDecimalBase<DateTime64>(maxDecimalPrecision<DateTime64>() - scale_, scale_),
|
||||
TimezoneMixin(time_zone_name)
|
||||
{
|
||||
}
|
||||
|
||||
std::string DataTypeDateTime64::doGetName() const
|
||||
{
|
||||
if (!has_explicit_time_zone)
|
||||
return std::string(getFamilyName()) + "(" + std::to_string(this->scale) + ")";
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
out << "DateTime64(" << this->scale << ", " << quote << time_zone.getTimeZone() << ")";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & /*settings*/) const
|
||||
{
|
||||
writeDateTimeText(assert_cast<const ColumnType &>(column).getData()[row_num], scale, ostr, time_zone);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
DateTime64 result;
|
||||
readDateTime64Text(result, this->getScale(), istr, time_zone);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(result);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
deserializeTextEscaped(column, istr, settings);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('\'', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('\'', ostr);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
if (checkChar('\'', istr)) /// Cases: '2017-08-31 18:36:48' or '1504193808'
|
||||
{
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
assertChar('\'', istr);
|
||||
}
|
||||
else /// Just 1504193808 or 01504193808
|
||||
{
|
||||
readIntText(x, istr);
|
||||
}
|
||||
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
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
if (checkChar('"', istr))
|
||||
{
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
assertChar('"', istr);
|
||||
}
|
||||
else
|
||||
{
|
||||
readIntText(x, istr);
|
||||
}
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeChar('"', ostr);
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
DateTime64 x;
|
||||
|
||||
if (istr.eof())
|
||||
throwReadAfterEOF();
|
||||
|
||||
char maybe_quote = *istr.position();
|
||||
|
||||
if (maybe_quote == '\'' || maybe_quote == '\"')
|
||||
++istr.position();
|
||||
|
||||
::readText(x, scale, istr, settings, time_zone, utc_time_zone);
|
||||
|
||||
if (maybe_quote == '\'' || maybe_quote == '\"')
|
||||
assertChar(maybe_quote, istr);
|
||||
|
||||
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
|
||||
{
|
||||
if (value_index)
|
||||
return;
|
||||
value_index = static_cast<bool>(protobuf.writeDateTime64(assert_cast<const ColumnType &>(column).getData()[row_num], scale));
|
||||
}
|
||||
|
||||
void DataTypeDateTime64::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const
|
||||
{
|
||||
row_added = false;
|
||||
DateTime64 t;
|
||||
if (!protobuf.readDateTime64(t, scale))
|
||||
return;
|
||||
|
||||
auto & container = assert_cast<ColumnType &>(column).getData();
|
||||
if (allow_add_row)
|
||||
{
|
||||
container.emplace_back(t);
|
||||
row_added = true;
|
||||
}
|
||||
else
|
||||
container.back() = t;
|
||||
}
|
||||
|
||||
bool DataTypeDateTime64::equals(const IDataType & rhs) const
|
||||
{
|
||||
if (auto * ptype = typeid_cast<const DataTypeDateTime64 *>(&rhs))
|
||||
return this->scale == ptype->getScale();
|
||||
return false;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
enum class ArgumentKind
|
||||
{
|
||||
Optional,
|
||||
Mandatory
|
||||
};
|
||||
|
||||
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;
|
||||
|
||||
auto exceptionMessage = [=](const String & message)
|
||||
{
|
||||
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(exceptionMessage(" is missing"),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
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 || arguments->size() == 0)
|
||||
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale);
|
||||
|
||||
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.value_or(DataTypeDateTime64::default_scale), timezone.value_or(String{}));
|
||||
}
|
||||
|
||||
void registerDataTypeDateTime64(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
47
dbms/src/DataTypes/DataTypeDateTime64.h
Normal file
47
dbms/src/DataTypes/DataTypeDateTime64.h
Normal file
@ -0,0 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
|
||||
class DateLUTImpl;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** DateTime64 is same as DateTime, but it stores values as UInt64 and has configurable sub-second part.
|
||||
*
|
||||
* `scale` determines number of decimal places for sub-second part of the DateTime64.
|
||||
*/
|
||||
class DataTypeDateTime64 final : public DataTypeDecimalBase<DateTime64>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
static constexpr UInt8 default_scale = 3;
|
||||
|
||||
explicit DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name = "");
|
||||
|
||||
const char * getFamilyName() const override { return "DateTime64"; }
|
||||
std::string doGetName() const override;
|
||||
TypeIndex getTypeId() const override { return TypeIndex::DateTime64; }
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override;
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
};
|
||||
|
||||
template <> inline constexpr bool IsDataTypeDecimal<DataTypeDateTime64> = true;
|
||||
|
||||
}
|
||||
|
@ -161,6 +161,7 @@ void registerDataTypeNumbers(DataTypeFactory & factory);
|
||||
void registerDataTypeDecimal(DataTypeFactory & factory);
|
||||
void registerDataTypeDate(DataTypeFactory & factory);
|
||||
void registerDataTypeDateTime(DataTypeFactory & factory);
|
||||
void registerDataTypeDateTime64(DataTypeFactory & factory);
|
||||
void registerDataTypeString(DataTypeFactory & factory);
|
||||
void registerDataTypeFixedString(DataTypeFactory & factory);
|
||||
void registerDataTypeEnum(DataTypeFactory & factory);
|
||||
@ -183,6 +184,7 @@ DataTypeFactory::DataTypeFactory()
|
||||
registerDataTypeDecimal(*this);
|
||||
registerDataTypeDate(*this);
|
||||
registerDataTypeDateTime(*this);
|
||||
registerDataTypeDateTime64(*this);
|
||||
registerDataTypeString(*this);
|
||||
registerDataTypeFixedString(*this);
|
||||
registerDataTypeEnum(*this);
|
||||
|
@ -594,7 +594,9 @@ struct ConvertThroughParsing
|
||||
{
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
{
|
||||
readDateTimeText(vec_to[i], vec_to.getScale(), read_buffer, local_time_zone);
|
||||
DateTime64 value;
|
||||
readDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
|
||||
vec_to[i] = value;
|
||||
}
|
||||
else if constexpr (IsDataTypeDecimal<ToDataType>)
|
||||
ToDataType::readText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Core/Block.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <common/DateLUT.h>
|
||||
|
||||
|
@ -16,14 +16,6 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
DateTime64::NativeType nowSubsecond(UInt32 scale)
|
||||
{
|
||||
timespec spec;
|
||||
clock_gettime(CLOCK_REALTIME, &spec);
|
||||
|
||||
return decimalFromComponents<DateTime64>(spec.tv_sec, spec.tv_nsec, scale).value;
|
||||
}
|
||||
|
||||
/// Get the current time. (It is a constant, it is evaluated once for the entire query.)
|
||||
class FunctionNow : public IFunction
|
||||
{
|
||||
@ -51,65 +43,8 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
class FunctionNow64 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "now64";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow64>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return ColumnNumbers{0}; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
// Return type depends on argument value.
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
UInt32 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)
|
||||
{
|
||||
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);
|
||||
|
||||
scale = argument.column->get64(0);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeDateTime64>(scale);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
UInt32 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);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = DataTypeDateTime64(scale).createColumnConst(input_rows_count, nowSubsecond(scale));
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionNow(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionNow64>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionNow>(FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
|
88
dbms/src/Functions/now64.cpp
Normal file
88
dbms/src/Functions/now64.cpp
Normal file
@ -0,0 +1,88 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
#include <time.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
DateTime64::NativeType nowSubsecond(UInt32 scale)
|
||||
{
|
||||
timespec spec;
|
||||
clock_gettime(CLOCK_REALTIME, &spec);
|
||||
|
||||
return decimalFromComponents<DateTime64>(spec.tv_sec, spec.tv_nsec, scale).value;
|
||||
}
|
||||
|
||||
class FunctionNow64 : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "now64";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow64>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return ColumnNumbers{0}; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
// Return type depends on argument value.
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
UInt32 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)
|
||||
{
|
||||
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);
|
||||
|
||||
scale = argument.column->get64(0);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeDateTime64>(scale);
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
UInt32 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);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = DataTypeDateTime64(scale).createColumnConst(input_rows_count, nowSubsecond(scale));
|
||||
}
|
||||
};
|
||||
|
||||
void registerFunctionNow64(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionNow64>(FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -37,6 +37,7 @@ void registerFunctionToRelativeMinuteNum(FunctionFactory &);
|
||||
void registerFunctionToRelativeSecondNum(FunctionFactory &);
|
||||
void registerFunctionToTime(FunctionFactory &);
|
||||
void registerFunctionNow(FunctionFactory &);
|
||||
void registerFunctionNow64(FunctionFactory &);
|
||||
void registerFunctionToday(FunctionFactory &);
|
||||
void registerFunctionYesterday(FunctionFactory &);
|
||||
void registerFunctionTimeSlot(FunctionFactory &);
|
||||
@ -101,6 +102,7 @@ void registerFunctionsDateTime(FunctionFactory & factory)
|
||||
registerFunctionToRelativeSecondNum(factory);
|
||||
registerFunctionToTime(factory);
|
||||
registerFunctionNow(factory);
|
||||
registerFunctionNow64(factory);
|
||||
registerFunctionToday(factory);
|
||||
registerFunctionYesterday(factory);
|
||||
registerFunctionTimeSlot(factory);
|
||||
|
@ -30,6 +30,7 @@
|
||||
#include <Formats/FormatSettings.h>
|
||||
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
Loading…
Reference in New Issue
Block a user