Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 13 additions & 4 deletions be/src/vec/functions/date_time_transforms.h
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ namespace doris::vectorized {
#define TO_TIME_FUNCTION(CLASS, UNIT) TIME_FUNCTION_IMPL(CLASS, UNIT, UNIT())

TO_TIME_FUNCTION(ToYearImpl, year);
TO_TIME_FUNCTION(ToYearOfWeekImpl, year_of_week);
TO_TIME_FUNCTION(ToQuarterImpl, quarter);
TO_TIME_FUNCTION(ToMonthImpl, month);
TO_TIME_FUNCTION(ToDayImpl, day);
Expand Down Expand Up @@ -377,8 +378,8 @@ struct Transformer {
}
};

template <typename FromType, typename ToType>
struct Transformer<FromType, ToType, ToYearImpl<FromType>> {
template <typename FromType, typename ToType, template <typename> typename Impl>
struct TransformerYear {
static void vector(const PaddedPODArray<FromType>& vec_from, PaddedPODArray<ToType>& vec_to,
NullMap& null_map) {
size_t size = vec_from.size();
Expand All @@ -390,7 +391,7 @@ struct Transformer<FromType, ToType, ToYearImpl<FromType>> {
auto* __restrict null_map_ptr = null_map.data();

for (size_t i = 0; i < size; ++i) {
to_ptr[i] = ToYearImpl<FromType>::execute(from_ptr[i]);
to_ptr[i] = Impl<FromType>::execute(from_ptr[i]);
}

for (size_t i = 0; i < size; ++i) {
Expand All @@ -406,11 +407,19 @@ struct Transformer<FromType, ToType, ToYearImpl<FromType>> {
auto* __restrict from_ptr = vec_from.data();

for (size_t i = 0; i < size; ++i) {
to_ptr[i] = ToYearImpl<FromType>::execute(from_ptr[i]);
to_ptr[i] = Impl<FromType>::execute(from_ptr[i]);
}
}
};

template <typename FromType, typename ToType>
struct Transformer<FromType, ToType, ToYearImpl<FromType>>
: public TransformerYear<FromType, ToType, ToYearImpl> {};

template <typename FromType, typename ToType>
struct Transformer<FromType, ToType, ToYearOfWeekImpl<FromType>>
: public TransformerYear<FromType, ToType, ToYearOfWeekImpl> {};

template <typename FromType, typename ToType, typename Transform>
struct DateTimeTransformImpl {
static Status execute(Block& block, const ColumnNumbers& arguments, uint32_t result,
Expand Down
10 changes: 3 additions & 7 deletions be/src/vec/functions/to_time_function.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,8 @@
// specific language governing permissions and limitations
// under the License.

#include <algorithm>
#include <boost/iterator/iterator_facade.hpp>
#include <memory>
#include <utility>

#include "vec/core/types.h"
#include "vec/data_types/data_type.h"
#include "vec/data_types/data_type_date_time.h"
#include "vec/data_types/data_type_nullable.h"
#include "vec/data_types/data_type_number.h"
#include "vec/data_types/data_type_time_v2.h"
#include "vec/functions/date_time_transforms.h"
Expand All @@ -35,6 +28,8 @@ namespace doris::vectorized {

using FunctionYear = FunctionDateOrDateTimeToSomething<DataTypeInt16, ToYearImpl<Int64>>;
using FunctionYearV2 = FunctionDateOrDateTimeToSomething<DataTypeInt16, ToYearImpl<UInt32>>;
using FunctionYearOfWeek =
FunctionDateOrDateTimeToSomething<DataTypeInt16, ToYearOfWeekImpl<UInt32>>;
using FunctionQuarter = FunctionDateOrDateTimeToSomething<DataTypeInt8, ToQuarterImpl<Int64>>;
using FunctionQuarterV2 = FunctionDateOrDateTimeToSomething<DataTypeInt8, ToQuarterImpl<UInt32>>;
using FunctionMonth = FunctionDateOrDateTimeToSomething<DataTypeInt8, ToMonthImpl<Int64>>;
Expand Down Expand Up @@ -102,6 +97,7 @@ void register_function_to_time_function(SimpleFunctionFactory& factory) {
factory.register_function<FunctionWeekV2>();
factory.register_function<FunctionMonthV2>();
factory.register_function<FunctionYearV2>();
factory.register_function<FunctionYearOfWeek>();
factory.register_function<FunctionQuarterV2>();
factory.register_function<FunctionToDaysV2>();
factory.register_function<FunctionToDateV2>();
Expand Down
26 changes: 26 additions & 0 deletions be/src/vec/runtime/vdatetime_value.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -3735,6 +3735,32 @@ bool DateV2Value<T>::from_date_int64(int64_t value) {
}
}

// An ISO week-numbering year (also called ISO year informally) has 52 or 53 full weeks. That is 364 or 371 days instead of the usual 365 or 366 days. These 53-week years occur on all years that have Thursday as 1 January and on leap years that start on Wednesday. The extra week is sometimes referred to as a leap week, although ISO 8601 does not use this term. https://en.wikipedia.org/wiki/ISO_week_date
template <typename T>
uint16_t DateV2Value<T>::year_of_week() const {
constexpr uint8_t THURSDAY = 3;

if (date_v2_value_.month_ == 1) {
constexpr uint8_t MAX_DISTANCE_WITH_THURSDAY = 6 - THURSDAY;
if (date_v2_value_.day_ <= MAX_DISTANCE_WITH_THURSDAY) {
auto weekday = calc_weekday(daynr(), false);
// if the current day is after Thursday and Thursday is in the previous year, return the previous year
return date_v2_value_.year_ -
(weekday > THURSDAY && weekday - THURSDAY > date_v2_value_.day_ - 1);
}
} else if (date_v2_value_.month_ == 12) {
constexpr uint8_t MAX_DISTANCE_WITH_THURSDAY = THURSDAY - 0;
if (S_DAYS_IN_MONTH[12] - date_v2_value_.day_ <= MAX_DISTANCE_WITH_THURSDAY) {
auto weekday = calc_weekday(daynr(), false);
// if the current day is before Thursday and Thursday is in the next year, return the next year
return date_v2_value_.year_ +
(weekday < THURSDAY &&
(THURSDAY - weekday) > S_DAYS_IN_MONTH[12] - date_v2_value_.day_);
}
}
return date_v2_value_.year_;
}

template <typename T>
uint8_t DateV2Value<T>::calc_week(const uint32_t& day_nr, const uint16_t& year,
const uint8_t& month, const uint8_t& day, uint8_t mode,
Expand Down
1 change: 1 addition & 0 deletions be/src/vec/runtime/vdatetime_value.h
Original file line number Diff line number Diff line change
Expand Up @@ -965,6 +965,7 @@ class DateV2Value {
}

uint16_t year() const { return date_v2_value_.year_; }
uint16_t year_of_week() const;
uint8_t month() const { return date_v2_value_.month_; }
int quarter() const { return (date_v2_value_.month_ - 1) / 3 + 1; }
int week() const { return week(mysql_week_mode(0)); } //00-53
Expand Down
12 changes: 12 additions & 0 deletions be/test/vec/function/function_time_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1819,4 +1819,16 @@ TEST(VTimestampFunctionsTest, seconds_sub_v2_test) {
}
}

TEST(VTimestampFunctionsTest, year_of_week_test) {
std::string func_name = "year_of_week";
{
InputTypeSet input_types = {TypeIndex::DateV2};
DataSet data_set = {{{std::string("2005-01-01")}, int16_t(2004)},
{{std::string("2008-12-30")}, int16_t(2009)},
{{std::string("12008-12-30")}, Null()},
{{Null()}, Null()}};
static_cast<void>(check_function<DataTypeInt16, true>(func_name, input_types, data_set));
}
}

} // namespace doris::vectorized
Original file line number Diff line number Diff line change
Expand Up @@ -479,6 +479,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.Year;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearCeil;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearFloor;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearOfWeek;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearWeek;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsDiff;
Expand Down Expand Up @@ -974,6 +975,7 @@ public class BuiltinScalarFunctions implements FunctionHelper {
scalar(Year.class, "year"),
scalar(YearCeil.class, "year_ceil"),
scalar(YearFloor.class, "year_floor"),
scalar(YearOfWeek.class, "year_of_week", "yow"),
scalar(YearWeek.class, "yearweek"),
scalar(YearsAdd.class, "years_add"),
scalar(YearsDiff.class, "years_diff"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.WeekOfYear;
import org.apache.doris.nereids.trees.expressions.functions.scalar.WeeksDiff;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Year;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearOfWeek;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsDiff;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsSub;
Expand Down Expand Up @@ -405,6 +406,21 @@ public ColumnStatistic visitYear(Year year, Statistics context) {
.setMaxValue(maxYear).setMinExpr(null).build();
}

@Override
public ColumnStatistic visitYearOfWeek(YearOfWeek yearOfWeek, Statistics context) {
ColumnStatistic childStat = yearOfWeek.child().accept(this, context);
double rowCount = context.getRowCount();
long minYear = 1970;
long maxYear = 2038;
return new ColumnStatisticBuilder()
.setNdv(maxYear - minYear + 1)
.setAvgSizeByte(4)
.setNumNulls(childStat.numNulls)
.setDataSize(4 * rowCount)
.setMinValue(minYear)
.setMaxValue(maxYear).setMinExpr(null).build();
}

@Override
public ColumnStatistic visitWeekOfYear(WeekOfYear weekOfYear, Statistics context) {
ColumnStatistic childStat = weekOfYear.child().accept(this, context);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.nereids.trees.expressions.functions.scalar;

import org.apache.doris.catalog.FunctionSignature;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.functions.Monotonic;
import org.apache.doris.nereids.trees.expressions.functions.PropagateNullableOnDateLikeV2Args;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DateV2Type;
import org.apache.doris.nereids.types.SmallIntType;

import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;

import java.util.List;

/**
* ScalarFunction 'year_of_week'. This class is generated by GenerateFunction.
*/
public class YearOfWeek extends ScalarFunction
implements UnaryExpression, ExplicitlyCastableSignature, PropagateNullableOnDateLikeV2Args, Monotonic {

private static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.ret(SmallIntType.INSTANCE).args(DateV2Type.INSTANCE));

/**
* constructor with 1 argument.
*/
public YearOfWeek(Expression arg) {
super("year_of_week", arg);
}

/**
* withChildren.
*/
@Override
public YearOfWeek withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() == 1);
return new YearOfWeek(children.get(0));
}

@Override
public List<FunctionSignature> getSignatures() {
return SIGNATURES;
}

@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitYearOfWeek(this, context);
}

@Override
public boolean isPositive() {
return true;
}

@Override
public int getMonotonicFunctionChildIndex() {
return 0;
}

@Override
public Expression withConstantArgs(Expression literal) {
return new YearOfWeek(literal);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -476,6 +476,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.Year;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearCeil;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearFloor;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearOfWeek;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearWeek;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsAdd;
import org.apache.doris.nereids.trees.expressions.functions.scalar.YearsDiff;
Expand Down Expand Up @@ -2261,6 +2262,10 @@ default R visitYear(Year year, C context) {
return visitScalarFunction(year, context);
}

default R visitYearOfWeek(YearOfWeek yearOfWeek, C context) {
return visitScalarFunction(yearOfWeek, context);
}

default R visitYearCeil(YearCeil yearCeil, C context) {
return visitScalarFunction(yearCeil, context);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -415,6 +415,68 @@ February
2019-08-01T13:21:03 2019
9999-08-01T13:21:03 9999

-- !sql --
1987

-- !sql --
2049

-- !sql --
0000-08-01T13:21:03 0
2019-08-01T13:21:03 2019
9999-08-01T13:21:03 9999

-- !sql --
1987

-- !sql --
2004

-- !sql --
2004

-- !sql --
2005

-- !sql --
2007

-- !sql --
2007

-- !sql --
2008

-- !sql --
2008

-- !sql --
2008

-- !sql --
2009

-- !sql --
2009

-- !sql --
2009

-- !sql --
2009

-- !sql --
2009

-- !sql --
2009

-- !sql --
2009

-- !sql --
2009

-- !sql --
202052

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -475,6 +475,30 @@ suite("test_date_function") {
qt_sql """ select year('2050-01-01') """
qt_sql """ select test_datetime, year(test_datetime) from ${tableName} order by test_datetime """

// YEAROFWEEK
qt_sql """ select year_of_week('1987-01-01') """
qt_sql """ select year_of_week('2050-01-01') """
qt_sql """ select test_datetime, year_of_week(test_datetime) from ${tableName} order by test_datetime """

qt_sql """ select yow('1987-01-01') """

qt_sql "select year_of_week('2005-01-01')" // 2004-W53-6
qt_sql "select year_of_week('2005-01-02')" // 2004-W53-7
qt_sql "select year_of_week('2005-12-31')" // 2005-W52-6
qt_sql "select year_of_week('2007-01-01')" // 2007-W01-1
qt_sql "select year_of_week('2007-12-30')" // 2007-W52-7
qt_sql "select year_of_week('2007-12-31')" // 2008-W01-1
qt_sql "select year_of_week('2008-01-01')" // 2008-W01-2
qt_sql "select year_of_week('2008-12-28')" // 2008-W52-7
qt_sql "select year_of_week('2008-12-29')" // 2009-W01-1
qt_sql "select year_of_week('2008-12-30')" // 2009-W01-2
qt_sql "select year_of_week('2008-12-31')" // 2009-W01-3
qt_sql "select year_of_week('2009-01-01')" // 2009-W01-4
qt_sql "select year_of_week('2009-12-31')" // 2009-W53-4
qt_sql "select year_of_week('2010-01-01')" // 2009-W53-5
qt_sql "select year_of_week('2010-01-02')" // 2009-W53-6
qt_sql "select year_of_week('2010-01-03')" // 2009-W53-7

// YEARWEEK
qt_sql """ select yearweek('2021-1-1') """
qt_sql """ select yearweek('2020-7-1') """
Expand Down
Loading