mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Fix toWeek monotonicity
This commit is contained in:
parent
9edfc1641a
commit
6ba29af871
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <common/types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -26,43 +27,6 @@ namespace ErrorCodes
|
||||
* CustomWeek Transformations.
|
||||
*/
|
||||
|
||||
static inline UInt32 dateIsNotSupported(const char * name)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
/// This factor transformation will say that the function is monotone everywhere.
|
||||
struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(Int64, UInt8, const DateLUTImpl &) { return 0; }
|
||||
};
|
||||
|
||||
struct ToWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToYearWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toYearWeek";
|
||||
@ -110,10 +74,34 @@ struct ToStartOfWeekImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
struct ToWeekImpl
|
||||
{
|
||||
explicit Transformer(Transform transform_)
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct WeekTransformer
|
||||
{
|
||||
explicit WeekTransformer(Transform transform_)
|
||||
: transform(std::move(transform_))
|
||||
{}
|
||||
|
||||
@ -139,7 +127,7 @@ struct CustomWeekTransformImpl
|
||||
template <typename Transform>
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, Transform transform = {})
|
||||
{
|
||||
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
|
||||
UInt8 week_mode = DEFAULT_WEEK_MODE;
|
||||
if (arguments.size() > 1)
|
||||
|
@ -10,9 +10,9 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
@ -143,15 +143,15 @@ public:
|
||||
|
||||
if (checkAndGetDataType<DataTypeDate>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
|
@ -0,0 +1,4 @@
|
||||
1 2020-11-11
|
||||
1 2021-01-01
|
||||
1 2020-11-11
|
||||
1 2021-01-01
|
11
tests/queries/0_stateless/01881_to_week_monotonic_fix.sql
Normal file
11
tests/queries/0_stateless/01881_to_week_monotonic_fix.sql
Normal file
@ -0,0 +1,11 @@
|
||||
drop table if exists test_tbl;
|
||||
|
||||
create table test_tbl (vend_nm String, ship_dt Date) engine MergeTree partition by toWeek(ship_dt) order by vend_nm;
|
||||
|
||||
insert into test_tbl values('1', '2020-11-11'), ('1', '2021-01-01');
|
||||
|
||||
select * From test_tbl where ship_dt >= toDate('2020-11-01') and ship_dt <= toDate('2021-05-05') order by ship_dt;
|
||||
|
||||
select * From test_tbl where ship_dt >= toDate('2020-01-01') and ship_dt <= toDate('2021-05-05') order by ship_dt;
|
||||
|
||||
drop table test_tbl;
|
Loading…
Reference in New Issue
Block a user