Skip to content

Commit

Permalink
fix bug of handling zero date in dayofweek and dayofyear (#4314)
Browse files Browse the repository at this point in the history
close #4313
  • Loading branch information
windtalker authored Mar 17, 2022
1 parent 0b22c9b commit bfe8015
Show file tree
Hide file tree
Showing 4 changed files with 102 additions and 14 deletions.
4 changes: 2 additions & 2 deletions dbms/src/Flash/Coprocessor/DAGUtils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -504,8 +504,8 @@ const std::unordered_map<tipb::ScalarFuncSig, String> scalar_func_map({

{tipb::ScalarFuncSig::DayName, "toDayName"},
{tipb::ScalarFuncSig::DayOfMonth, "toDayOfMonth"},
{tipb::ScalarFuncSig::DayOfWeek, "toDayOfWeek"},
{tipb::ScalarFuncSig::DayOfYear, "toDayOfYear"},
{tipb::ScalarFuncSig::DayOfWeek, "tidbDayOfWeek"},
{tipb::ScalarFuncSig::DayOfYear, "tidbDayOfYear"},

//{tipb::ScalarFuncSig::WeekWithMode, "cast"},
//{tipb::ScalarFuncSig::WeekWithoutMode, "cast"},
Expand Down
2 changes: 2 additions & 0 deletions dbms/src/Functions/FunctionsDateTime.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,8 @@ void registerFunctionsDateTime(FunctionFactory & factory)
factory.registerFunction<FunctionTiDBTimestampDiff>();
factory.registerFunction<FunctionExtractMyDateTime>();
factory.registerFunction<FunctionTiDBDateDiff>();
factory.registerFunction<FunctionToTiDBDayOfWeek>();
factory.registerFunction<FunctionToTiDBDayOfYear>();

factory.registerFunction<FunctionToTimeZone>();
factory.registerFunction<FunctionToLastDay>();
Expand Down
72 changes: 72 additions & 0 deletions dbms/src/Functions/FunctionsDateTime.h
Original file line number Diff line number Diff line change
Expand Up @@ -3314,6 +3314,76 @@ struct TiDBLastDayTransformerImpl
}
};

template <typename ToFieldType>
struct TiDBDayOfWeekTransformerImpl
{
static constexpr auto name = "tidbDayOfWeek";

static void execute(const Context & context,
const ColumnVector<DataTypeMyTimeBase::FieldType>::Container & vec_from,
typename ColumnVector<ToFieldType>::Container & vec_to,
typename ColumnVector<UInt8>::Container & vec_null_map)
{
for (size_t i = 0; i < vec_from.size(); ++i)
{
bool is_null = false;
MyTimeBase val(vec_from[i]);
vec_to[i] = execute(context, val, is_null);
vec_null_map[i] = is_null;
}
}

static ToFieldType execute(const Context & context, const MyTimeBase & val, bool & is_null)
{
// TiDB also considers NO_ZERO_DATE sql_mode. But sql_mode is not handled by TiFlash for now.
if (val.month == 0 || val.day == 0)
{
context.getDAGContext()->handleInvalidTime(
fmt::format("Invalid time value: month({}) or day({}) is zero", val.month, val.day),
Errors::Types::WrongValue);
is_null = true;
return 0;
}
/// Behavior differences from TiDB:
/// for date in ['0000-01-01', '0000-03-01'), dayOfWeek is the same with MySQL, while TiDB is offset by one day
/// In TiDB dayOfWeek('0000-01-01') = 7, in MySQL/TiFlash dayOfWeek('0000-01-01') = 1
return static_cast<ToFieldType>(val.weekDay() + 1);
}
};

template <typename ToFieldType>
struct TiDBDayOfYearTransformerImpl
{
static constexpr auto name = "tidbDayOfYear";

static void execute(const Context & context,
const ColumnVector<DataTypeMyTimeBase::FieldType>::Container & vec_from,
typename ColumnVector<ToFieldType>::Container & vec_to,
typename ColumnVector<UInt8>::Container & vec_null_map)
{
for (size_t i = 0; i < vec_from.size(); ++i)
{
bool is_null = false;
MyTimeBase val(vec_from[i]);
vec_to[i] = execute(context, val, is_null);
vec_null_map[i] = is_null;
}
}

static ToFieldType execute(const Context & context, const MyTimeBase & val, bool & is_null)
{
// TiDB also considers NO_ZERO_DATE sql_mode. But sql_mode is not handled by TiFlash for now.
if (val.month == 0 || val.day == 0)
{
context.getDAGContext()->handleInvalidTime(
fmt::format("Invalid time value: month({}) or day({}) is zero", val.month, val.day),
Errors::Types::WrongValue);
is_null = true;
return 0;
}
return static_cast<ToFieldType>(val.yearDay());
}
};
// Similar to FunctionDateOrDateTimeToSomething, but also handle nullable result and mysql sql mode.
template <typename ToDataType, template <typename> class Transformer, bool return_nullable>
class FunctionMyDateOrMyDateTimeToSomething : public IFunction
Expand Down Expand Up @@ -3410,6 +3480,8 @@ using FunctionToStartOfFifteenMinutes = FunctionDateOrDateTimeToSomething<DataTy
using FunctionToStartOfHour = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToStartOfHourImpl>;
using FunctionToTime = FunctionDateOrDateTimeToSomething<DataTypeDateTime, ToTimeImpl>;
using FunctionToLastDay = FunctionMyDateOrMyDateTimeToSomething<DataTypeMyDate, TiDBLastDayTransformerImpl, return_nullable>;
using FunctionToTiDBDayOfWeek = FunctionMyDateOrMyDateTimeToSomething<DataTypeUInt16, TiDBDayOfWeekTransformerImpl, return_nullable>;
using FunctionToTiDBDayOfYear = FunctionMyDateOrMyDateTimeToSomething<DataTypeUInt16, TiDBDayOfYearTransformerImpl, return_nullable>;

using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething<DataTypeUInt16, ToRelativeYearNumImpl>;
using FunctionToRelativeQuarterNum = FunctionDateOrDateTimeToSomething<DataTypeUInt32, ToRelativeQuarterNumImpl>;
Expand Down
38 changes: 26 additions & 12 deletions dbms/src/Functions/tests/gtest_dayofweekyear.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,11 @@ class TestDayOfWeekYear : public DB::tests::FunctionTest
TEST_F(TestDayOfWeekYear, TestDayOfWeek)
try
{
DAGContext * dag_context = context.getDAGContext();
UInt64 ori_flags = dag_context->getFlags();
dag_context->addFlag(TiDBSQLFlags::TRUNCATE_AS_WARNING);
/// ColumnVector(nullable)
const String func_name = "toDayOfWeek";
const String func_name = "tidbDayOfWeek";
static auto const nullable_datetime_type_ptr = makeNullable(std::make_shared<DataTypeMyDateTime>(6));
static auto const datetime_type_ptr = std::make_shared<DataTypeMyDateTime>(6);
static auto const date_type_ptr = std::make_shared<DataTypeMyDate>();
Expand All @@ -40,6 +43,8 @@ try
// FIXME: https://github.com/pingcap/tiflash/issues/4186
// MyDateTime(2022, 12, 0, 1, 1, 1, 1).toPackedUInt(),
// MyDateTime(2022, 13, 31, 1, 1, 1, 1).toPackedUInt(),
MyDateTime(0, 0, 0, 0, 0, 0, 0).toPackedUInt(),
MyDateTime(0, 1, 1, 0, 0, 0, 0).toPackedUInt(),
MyDateTime(1969, 1, 2, 1, 1, 1, 1).toPackedUInt(),
MyDateTime(2022, 3, 13, 6, 7, 8, 9).toPackedUInt(),
MyDateTime(2022, 3, 14, 9, 8, 7, 6).toPackedUInt(),
Expand All @@ -51,12 +56,13 @@ try
})
.column;
auto input_col = ColumnWithTypeAndName(data_col_ptr, nullable_datetime_type_ptr, "input");
auto output_col = createColumn<Nullable<UInt8>>({{}, 5, 1, 2, 3, 4, 5, 6, 7});
auto output_col = createColumn<Nullable<UInt16>>({{}, {}, 1, 5, 1, 2, 3, 4, 5, 6, 7});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnVector(non-null)
data_col_ptr = createColumn<DataTypeMyDateTime::FieldType>(
{
MyDateTime(0, 0, 0, 0, 0, 0, 0).toPackedUInt(),
MyDateTime(1969, 1, 2, 1, 1, 1, 1).toPackedUInt(),
MyDateTime(2022, 3, 13, 6, 7, 8, 9).toPackedUInt(),
MyDateTime(2022, 3, 14, 9, 8, 7, 6).toPackedUInt(),
Expand All @@ -68,22 +74,22 @@ try
})
.column;
input_col = ColumnWithTypeAndName(data_col_ptr, datetime_type_ptr, "input");
output_col = createColumn<UInt8>({5, 1, 2, 3, 4, 5, 6, 7});
output_col = createColumn<Nullable<UInt16>>({{}, 5, 1, 2, 3, 4, 5, 6, 7});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnConst(non-null)
input_col = ColumnWithTypeAndName(createConstColumn<DataTypeMyDateTime::FieldType>(1, MyDateTime(2022, 3, 19, 1, 1, 1, 1).toPackedUInt()).column, datetime_type_ptr, "input");
output_col = createConstColumn<UInt8>(1, {7});
output_col = createConstColumn<Nullable<UInt16>>(1, {7});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnConst(nullable)
input_col = ColumnWithTypeAndName(createConstColumn<Nullable<DataTypeMyDateTime::FieldType>>(1, MyDateTime(2022, 3, 19, 1, 1, 1, 1).toPackedUInt()).column, nullable_datetime_type_ptr, "input");
output_col = createConstColumn<Nullable<UInt8>>(1, {7});
output_col = createConstColumn<Nullable<UInt16>>(1, {7});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnConst(nullable(null))
input_col = ColumnWithTypeAndName(createConstColumn<Nullable<DataTypeMyDateTime::FieldType>>(1, {}).column, nullable_datetime_type_ptr, "input");
output_col = createConstColumn<Nullable<UInt8>>(1, {});
output_col = createConstColumn<Nullable<UInt16>>(1, {});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// MyDate ColumnVector(non-null)
Expand All @@ -100,16 +106,20 @@ try
})
.column;
input_col = ColumnWithTypeAndName(data_col_ptr, date_type_ptr, "input");
output_col = createColumn<UInt8>({5, 1, 2, 3, 4, 5, 6, 7});
output_col = createColumn<Nullable<UInt16>>({5, 1, 2, 3, 4, 5, 6, 7});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));
dag_context->setFlags(ori_flags);
}
CATCH

TEST_F(TestDayOfWeekYear, TestDayOfYear)
try
{
DAGContext * dag_context = context.getDAGContext();
UInt64 ori_flags = dag_context->getFlags();
dag_context->addFlag(TiDBSQLFlags::TRUNCATE_AS_WARNING);
/// ColumnVector(nullable)
const String func_name = "toDayOfYear";
const String func_name = "tidbDayOfYear";
static auto const nullable_datetime_type_ptr = makeNullable(std::make_shared<DataTypeMyDateTime>(6));
static auto const datetime_type_ptr = std::make_shared<DataTypeMyDateTime>(6);
static auto const date_type_ptr = std::make_shared<DataTypeMyDate>();
Expand All @@ -119,6 +129,8 @@ try
// FIXME: https://github.com/pingcap/tiflash/issues/4186
// MyDateTime(2022, 12, 0, 1, 1, 1, 1).toPackedUInt(),
// MyDateTime(2022, 13, 31, 1, 1, 1, 1).toPackedUInt(),
MyDateTime(0, 0, 0, 0, 0, 0, 0).toPackedUInt(),
MyDateTime(0, 1, 1, 0, 0, 0, 0).toPackedUInt(),
MyDateTime(1969, 1, 2, 1, 1, 1, 1).toPackedUInt(),
MyDateTime(2022, 3, 13, 6, 7, 8, 9).toPackedUInt(),
MyDateTime(2022, 3, 14, 9, 8, 7, 6).toPackedUInt(),
Expand All @@ -133,12 +145,13 @@ try
})
.column;
auto input_col = ColumnWithTypeAndName(data_col_ptr, nullable_datetime_type_ptr, "input");
auto output_col = createColumn<Nullable<UInt16>>({{}, 2, 72, 73, 74, 75, 76, 77, 78, 365, 366, 365});
auto output_col = createColumn<Nullable<UInt16>>({{}, {}, 1, 2, 72, 73, 74, 75, 76, 77, 78, 365, 366, 365});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnVector(non-null)
data_col_ptr = createColumn<DataTypeMyDateTime::FieldType>(
{
MyDateTime(0, 0, 0, 0, 0, 0, 0).toPackedUInt(),
MyDateTime(1969, 1, 2, 1, 1, 1, 1).toPackedUInt(),
MyDateTime(2022, 3, 13, 6, 7, 8, 9).toPackedUInt(),
MyDateTime(2022, 3, 14, 9, 8, 7, 6).toPackedUInt(),
Expand All @@ -153,12 +166,12 @@ try
})
.column;
input_col = ColumnWithTypeAndName(data_col_ptr, datetime_type_ptr, "input");
output_col = createColumn<UInt16>({2, 72, 73, 74, 75, 76, 77, 78, 365, 366, 365});
output_col = createColumn<Nullable<UInt16>>({{}, 2, 72, 73, 74, 75, 76, 77, 78, 365, 366, 365});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnConst(non-null)
input_col = ColumnWithTypeAndName(createConstColumn<DataTypeMyDateTime::FieldType>(1, MyDateTime(2022, 3, 19, 1, 1, 1, 1).toPackedUInt()).column, datetime_type_ptr, "input");
output_col = createConstColumn<UInt16>(1, {78});
output_col = createConstColumn<Nullable<UInt16>>(1, {78});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));

/// ColumnConst(nullable)
Expand Down Expand Up @@ -188,8 +201,9 @@ try
})
.column;
input_col = ColumnWithTypeAndName(data_col_ptr, date_type_ptr, "input");
output_col = createColumn<UInt16>({2, 72, 73, 74, 75, 76, 77, 78, 365, 366, 365});
output_col = createColumn<Nullable<UInt16>>({2, 72, 73, 74, 75, 76, 77, 78, 365, 366, 365});
ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col));
dag_context->setFlags(ori_flags);
}
CATCH

Expand Down

0 comments on commit bfe8015

Please sign in to comment.