mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 11:02:08 +00:00
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:
parent
118e498eab
commit
81ca591b23
@ -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;
|
||||||
|
@ -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
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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;
|
||||||
|
Loading…
Reference in New Issue
Block a user