read and write datetime64

This commit is contained in:
Martijn Bakker 2019-05-01 23:42:17 +01:00
parent f6319cff9b
commit fc0e8d3658
4 changed files with 66 additions and 91 deletions

View File

@ -30,6 +30,10 @@ struct TypeGetter<UInt32> {
using Column = ColumnUInt32;
static constexpr TypeIndex Index = TypeIndex::DateTime;
static constexpr const char * Name = "DateTime";
static void write(Type datetime, WriteBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) {
writeDateTimeText(datetime, buf, date_lut);
}
};
template<>
@ -38,6 +42,10 @@ struct TypeGetter<UInt64> {
using Column = ColumnUInt64;
static constexpr TypeIndex Index = TypeIndex::DateTime64;
static constexpr const char * Name = "DateTime64";
static void write(Type datetime64, WriteBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance()) {
writeDateTime64Text(datetime64, buf, date_lut);
}
};
template<typename NumberBase>
@ -48,23 +56,6 @@ DataTypeDateTimeBase<NumberBase>::DataTypeDateTimeBase(const std::string & time_
{
}
DataTypeDateTime64::Precision parsePrecision(const std::string & precision_name)
{
if (precision_name == "MILLI")
return DataTypeDateTime64::Precision::Millis;
else if (precision_name == "MICRO")
return DataTypeDateTime64::Precision::Micros;
return DataTypeDateTime64::Precision::Nanos;
}
DataTypeDateTime64::DataTypeDateTime64(const std::string & time_zone_name, const std::string & precision_name)
: DataTypeDateTimeBase(time_zone_name),
precision(parsePrecision(precision_name))
{
}
template<typename NumberBase>
const char * DataTypeDateTimeBase<NumberBase>::getFamilyName() const
{
@ -91,44 +82,8 @@ TypeIndex DataTypeDateTimeBase<NumberBase>::getTypeId() const
template<typename NumberBase>
void DataTypeDateTimeBase<NumberBase>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeDateTimeText(static_cast<const typename TypeGetter<NumberBase>::Column &>(column).getData()[row_num], ostr, time_zone);
}
void DataTypeDateTime64::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
time_t base_time;
auto full_time = static_cast<const ColumnUInt64 &>(column).getData()[row_num];
UInt32 time_fraction;
int pad_length = 0;
switch(precision) {
case DataTypeDateTime64::Precision::Millis: {
base_time = full_time / MILLIS_PER_SECOND;
time_fraction = full_time % MILLIS_PER_SECOND;
pad_length = 3;
break;
}
case DataTypeDateTime64::Precision::Micros: {
base_time = full_time / MICROS_PER_SECOND;
time_fraction = full_time % MICROS_PER_SECOND;
pad_length = 6;
break;
}
case DataTypeDateTime64::Precision::Nanos: {
base_time = full_time / NANOS_PER_SECOND;
time_fraction = full_time % NANOS_PER_SECOND;
pad_length = 9;
break;
}
}
writeDateTimeText(base_time, ostr, time_zone);
writeText(".", 1, ostr);
/// TODO make this efficient
std::stringstream ss;
ss << std::setfill('0') << std::setw(pad_length) << time_fraction;
writeText(ss.str(), ostr);
using TG = TypeGetter<NumberBase>;
TG::write(static_cast<const typename TG::Column &>(column).getData()[row_num], ostr, time_zone);
}
template<typename NumberBase>
@ -151,25 +106,22 @@ static inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings
}
}
static inline void readText(UInt64 & /*x*/, ReadBuffer & /*istr*/, const FormatSettings & /*settings*/, const DateLUTImpl & /*time_zone*/, const DateLUTImpl & /*utc_time_zone*/)
static inline void readText(UInt64 & x, ReadBuffer & istr, const FormatSettings & settings, const DateLUTImpl & time_zone, const DateLUTImpl & /*utc_time_zone*/)
{
// TODO implement this
// return;
// switch (settings.date_time_input_format)
// {
// case FormatSettings::DateTimeInputFormat::Basic:
// readDateTimeText(x, istr, time_zone);
// return;
// case FormatSettings::DateTimeInputFormat::BestEffort:
// parseDateTimeBestEffort(x, istr, time_zone, utc_time_zone);
// return;
// }
switch (settings.date_time_input_format)
{
case FormatSettings::DateTimeInputFormat::Basic:
readDateTime64Text(x, istr, time_zone);
return;
default:
return;
}
}
template<typename NumberBase>
void DataTypeDateTimeBase<NumberBase>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
typename TypeGetter<NumberBase>::Type x;
typename TypeGetter<NumberBase>::Type x = 0;
readText(x, istr, settings, time_zone, utc_time_zone);
static_cast<typename TypeGetter<NumberBase>::Column &>(column).getData().push_back(x);
}
@ -233,7 +185,7 @@ void DataTypeDateTimeBase<NumberBase>::serializeTextCSV(const IColumn & column,
template<typename NumberBase>
void DataTypeDateTimeBase<NumberBase>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
typename TypeGetter<NumberBase>::Type x;
typename TypeGetter<NumberBase>::Type x = 0;
if (istr.eof())
throwReadAfterEOF();
@ -313,18 +265,14 @@ static DataTypePtr create64(const ASTPtr & arguments)
if (!arguments)
return std::make_shared<DataTypeDateTime64>();
if (arguments->children.size() != 2)
throw Exception("DateTime64 data type can optionally have 2 arguments - precision and time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments->children.size() != 1)
throw Exception("DateTime64 data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto * timezone_arg = arguments->children[0]->as<ASTLiteral>();
if (!timezone_arg || timezone_arg->value.getType() != Field::Types::String)
throw Exception("Timezone parameter for DateTime64 data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto * precision_arg = arguments->children[1]->as<ASTLiteral>();
if (!precision_arg || precision_arg->value.getType() != Field::Types::String)
throw Exception("Precision parameter for DateTime64 data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeDateTime64>(timezone_arg->value.get<String>(), precision_arg->value.get<String>());
return std::make_shared<DataTypeDateTime64>(timezone_arg->value.get<String>());
}
void registerDataTypeDateTime(DataTypeFactory & factory)

View File

@ -68,21 +68,7 @@ struct DataTypeDateTime : DataTypeDateTimeBase<UInt32> {
};
struct DataTypeDateTime64 : DataTypeDateTimeBase<UInt64> {
enum class Precision {
Millis,
Micros,
Nanos,
};
static constexpr UInt32 MILLIS_PER_SECOND = 1000;
static constexpr UInt32 MICROS_PER_SECOND = 1000 * 1000;
static constexpr UInt32 NANOS_PER_SECOND = 1000 * 1000 * 1000;
DataTypeDateTime64(const std::string & time_zone_name = "", const std::string & precision_name = "");
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
private:
const Precision precision;
using DataTypeDateTimeBase::DataTypeDateTimeBase;
};
}

View File

@ -622,6 +622,15 @@ inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTI
readDateTimeTextImpl<void>(datetime, buf, date_lut);
}
inline void readDateTime64Text(UInt64 & datetime64, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
time_t datetime = 0;
readDateTimeTextImpl<void>(datetime, buf, date_lut);
buf.ignore(); // ignore the "."
readIntText(datetime64, buf);
datetime64 += 1000 * 1000 * 1000 * datetime;
}
inline bool tryReadDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
return readDateTimeTextImpl<bool>(datetime, buf, date_lut);

View File

@ -669,6 +669,38 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTI
date_lut.toHour(datetime), date_lut.toMinute(datetime), date_lut.toSecond(datetime)), buf);
}
/// In the format YYYY-MM-DD HH:MM:SS.NNNNNNNNN, according to the specified time zone.
template <char date_delimeter = '-', char time_delimeter = ':', char between_date_time_delimiter = ' ', char fractional_time_delimiter = '.'>
inline void writeDateTime64Text(UInt64 datetime64, WriteBuffer & buf, const DateLUTImpl & date_lut = DateLUT::instance())
{
if (unlikely(!datetime64))
{
static const char s[] =
{
'0', '0', '0', '0', date_delimeter, '0', '0', date_delimeter, '0', '0',
between_date_time_delimiter,
'0', '0', time_delimeter, '0', '0', time_delimeter, '0', '0',
fractional_time_delimiter,
'0', '0', '0', '0', '0', '0', '0', '0', '0'
};
buf.write(s, sizeof(s));
return;
}
const UInt32 NANOS_PER_SECOND = 1000 * 1000 * 1000;
time_t datetime = datetime64 / NANOS_PER_SECOND;
auto nanos_since_second = static_cast<UInt32>(datetime64 % NANOS_PER_SECOND);
const auto & values = date_lut.getValues(datetime64);
writeDateTimeText<date_delimeter, time_delimeter, between_date_time_delimiter>(
LocalDateTime(values.year, values.month, values.day_of_month,
date_lut.toHour(datetime), date_lut.toMinute(datetime), date_lut.toSecond(datetime)), buf);
buf.write(fractional_time_delimiter);
writeIntText(nanos_since_second, buf);
}
/// Methods for output in binary format.
template <typename T>