various formatters are working, though there is still an issue with toDate for some reason, maybe the data is unaligned?

This commit is contained in:
Martijn Bakker 2019-05-04 13:13:59 +01:00
parent 118e498eab
commit 81ca591b23
6 changed files with 77 additions and 20 deletions

View File

@ -3,6 +3,7 @@
#include <utility> #include <utility>
#include <Core/Types.h> #include <Core/Types.h>
#include <DataTypes/DataTypeDateTime.h>
namespace DB namespace DB
{ {
@ -71,6 +72,7 @@ bool callOnBasicType(TypeIndex number, F && f)
{ {
case TypeIndex::Date: return f(TypePair<T, UInt16>()); case TypeIndex::Date: return f(TypePair<T, UInt16>());
case TypeIndex::DateTime: return f(TypePair<T, UInt32>()); case TypeIndex::DateTime: return f(TypePair<T, UInt32>());
case TypeIndex::DateTime64: return f(TypePair<T, DateTime64>());
default: default:
break; break;
} }
@ -145,8 +147,6 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
class DataTypeDate; class DataTypeDate;
class DataTypeDateTime;
class DataTypeDateTime64;
class DataTypeString; class DataTypeString;
class DataTypeFixedString; class DataTypeFixedString;
class DataTypeUUID; class DataTypeUUID;

View File

@ -5,6 +5,13 @@
class DateLUTImpl; 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 namespace DB
{ {

View File

@ -5,7 +5,7 @@
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Functions/FunctionHelpers.h> #include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h> #include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <DataTypes/DataTypeDateTime.h>
namespace DB namespace DB
{ {
@ -47,10 +47,12 @@ struct ToDateImpl
static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone) static inline UInt16 execute(UInt32 t, const DateLUTImpl & time_zone)
{ {
std::cout << "converting UInt32 t=" << t << " " << name << std::endl;
return UInt16(time_zone.toDayNum(t)); return UInt16(time_zone.toDayNum(t));
} }
static inline UInt16 execute(UInt16 d, const DateLUTImpl &) static inline UInt16 execute(UInt16 d, const DateLUTImpl &)
{ {
std::cout << "converting UInt16 d=" << d << " " << name << std::endl;
return d; return d;
} }

View File

@ -15,6 +15,17 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
template<class Transform>
struct WithDateTime64Converter : public Transform {
static inline auto execute(DataTypeDateTime64::FieldType t, const DateLUTImpl & time_zone)
{
auto x = DateTime64(t);
auto res = Transform::execute(static_cast<UInt32>(x.split().datetime), time_zone);
std::cout << "calling through datetime64 wrapper v=" << x.get() << "tz= " << time_zone.getTimeZone() << " result=" << res << std::endl;
return res;
}
};
/// See DateTimeTransforms.h /// See DateTimeTransforms.h
template <typename ToDataType, typename Transform> template <typename ToDataType, typename Transform>
@ -67,8 +78,8 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// For DateTime, if time zone is specified, attach it to type. /// For DateTime, if time zone is specified, attach it to type.
if (std::is_same_v<ToDataType, DataTypeDateTime>) if constexpr (is_instance<ToDataType, DataTypeDateTimeBase>{})
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); return std::make_shared<ToDataType>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
else else
return std::make_shared<ToDataType>(); return std::make_shared<ToDataType>();
} }
@ -85,6 +96,8 @@ public:
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count); DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
else if (which.isDateTime()) else if (which.isDateTime())
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count); DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
else if (which.isDateTime64())
DateTimeTransformImpl<DataTypeDateTime64::FieldType, typename ToDataType::FieldType, WithDateTime64Converter<Transform>>::execute(block, arguments, result, input_rows_count);
else else
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -7,12 +7,43 @@
namespace DB namespace DB
{ {
template<typename DT>
struct TypeGetter;
template<>
struct TypeGetter<DataTypeDateTime64> {
using Type = DataTypeDateTime64;
static constexpr auto name = "now64";
static DateTime64::Type now() {
long int ns;
time_t sec;
timespec spec;
clock_gettime(CLOCK_REALTIME, &spec);
sec = spec.tv_sec;
ns = spec.tv_nsec;
return 1000 * 1000 * 1000 * sec + ns;
}
};
template<>
struct TypeGetter<DataTypeDateTime> {
using Type = DataTypeDateTime;
static constexpr auto name = "now";
static UInt64 now() {
return static_cast<UInt64>(time(nullptr));
}
};
/// Get the current time. (It is a constant, it is evaluated once for the entire query.) /// Get the current time. (It is a constant, it is evaluated once for the entire query.)
template<typename TG>
class FunctionNow : public IFunction class FunctionNow : public IFunction
{ {
public: public:
static constexpr auto name = "now"; static constexpr auto name = TypeGetter<TG>::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow>(); } static FunctionPtr create(const Context &) { return std::make_shared<FunctionNow<TG>>(); }
String getName() const override String getName() const override
{ {
@ -23,22 +54,21 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
{ {
return std::make_shared<DataTypeDateTime>(); return std::make_shared<typename TypeGetter<TG>::Type>();
} }
bool isDeterministic() const override { return false; } bool isDeterministic() const override { return false; }
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{ {
block.getByPosition(result).column = DataTypeDateTime().createColumnConst( block.getByPosition(result).column = typename TypeGetter<TG>::Type().createColumnConst(input_rows_count, TypeGetter<TG>::now());
input_rows_count,
static_cast<UInt64>(time(nullptr)));
} }
}; };
void registerFunctionNow(FunctionFactory & factory) void registerFunctionNow(FunctionFactory & factory)
{ {
factory.registerFunction<FunctionNow>(FunctionFactory::CaseInsensitive); factory.registerFunction<FunctionNow<DataTypeDateTime64>>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionNow<DataTypeDateTime>>(FunctionFactory::CaseInsensitive);
} }
} }

View File

@ -1,16 +1,21 @@
USE test; USE test;
DROP TABLE IF EXISTS A; DROP TABLE IF EXISTS A;
DROP TABLE IF EXISTS B; -- DROP TABLE IF EXISTS B;
CREATE TABLE A(k UInt32, t DateTime64, a Float64) ENGINE = MergeTree() ORDER BY (k, t); CREATE TABLE A(t DateTime64, a Float64) ENGINE = MergeTree() ORDER BY t;
INSERT INTO A(k,t,a) VALUES (2,1,1),(2,50000,3); -- INSERT INTO A(t,a) VALUES (1,1),(50000,3);
INSERT INTO A(k,t,a) VALUES (2,'2019-05-03 00:25:25.123456789',5); INSERT INTO A(t,a) VALUES ('2019-05-03 11:25:25.123456789',5);
-- INSERT INTO A(t,a) VALUES (1556841600034000001,5);
-- INSERT INTO A(t,a) VALUES (now64(),5);
CREATE TABLE B(k UInt32, t DateTime64, b Float64) ENGINE = MergeTree() ORDER BY (k, t); -- 1556841600034
INSERT INTO B(k,t,b) VALUES (2,40000,3);
SELECT k, toString(A.t, 'Europe/Moscow'), a, b FROM A ASOF LEFT JOIN B USING(k,t) ORDER BY (k,t);
DROP TABLE B; -- CREATE TABLE B(k UInt32, t DateTime64, b Float64) ENGINE = MergeTree() ORDER BY (k, t);
-- INSERT INTO B(k,t,b) VALUES (2,40000,3);
SELECT toString(t, 'UTC'), toDate(t), toStartOfDay(t), toStartOfQuarter(t), toTime(t), toStartOfMinute(t), a FROM A ORDER BY t;
-- DROP TABLE B;
DROP TABLE A; DROP TABLE A;