Support TIME type and timediff function (#1505)

This commit is contained in:
HangyuanLiu
2019-07-23 13:42:40 +08:00
committed by ZHAO Chun
parent 221cd2e103
commit 4aedaea84e
37 changed files with 318 additions and 20 deletions

View File

@ -46,6 +46,7 @@
#include "exprs/timestamp_functions.h"
#include "exprs/decimal_operators.h"
#include "exprs/decimalv2_operators.h"
#include "exprs/time_operators.h"
#include "exprs/utility_functions.h"
#include "exprs/json_functions.h"
#include "exprs/hll_hash_function.h"
@ -260,6 +261,7 @@ void init_daemon(int argc, char** argv, const std::vector<StorePath>& paths) {
TimestampFunctions::init();
DecimalOperators::init();
DecimalV2Operators::init();
TimeOperators::init();
UtilityFunctions::init();
CompoundPredicate::init();
JsonFunctions::init();

View File

@ -37,6 +37,7 @@ add_library(Exprs
conditional_functions_ir.cpp
decimal_operators.cpp
decimalv2_operators.cpp
time_operators.cpp
es_functions.cpp
literal.cpp
expr.cpp

View File

@ -76,6 +76,7 @@ AnyVal* create_any_val(ObjectPool* pool, const TypeDescriptor& type) {
case TYPE_FLOAT:
return pool->add(new FloatVal);
case TYPE_TIME:
case TYPE_DOUBLE:
return pool->add(new DoubleVal);
@ -125,6 +126,7 @@ FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescrip
case TYPE_FLOAT:
out.type = FunctionContext::TYPE_FLOAT;
break;
case TYPE_TIME:
case TYPE_DOUBLE:
out.type = FunctionContext::TYPE_DOUBLE;
break;

View File

@ -369,6 +369,7 @@ public:
reinterpret_cast<doris_udf::FloatVal*>(dst)->val =
*reinterpret_cast<const float*>(slot);
return;
case TYPE_TIME:
case TYPE_DOUBLE:
reinterpret_cast<doris_udf::DoubleVal*>(dst)->val =
*reinterpret_cast<const double*>(slot);

View File

@ -137,6 +137,7 @@ Expr::Expr(const TypeDescriptor& type) :
case TYPE_FLOAT:
case TYPE_DOUBLE:
case TYPE_TIME:
_node_type = (TExprNodeType::FLOAT_LITERAL);
break;
@ -195,6 +196,7 @@ Expr::Expr(const TypeDescriptor& type, bool is_slotref) :
case TYPE_FLOAT:
case TYPE_DOUBLE:
case TYPE_TIME:
_node_type = (TExprNodeType::FLOAT_LITERAL);
break;
@ -743,7 +745,8 @@ doris_udf::AnyVal* Expr::get_const_val(ExprContext* context) {
_constant_val.reset(new FloatVal(get_float_val(context, NULL)));
break;
}
case TYPE_DOUBLE: {
case TYPE_DOUBLE:
case TYPE_TIME: {
_constant_val.reset(new DoubleVal(get_double_val(context, NULL)));
break;
}

View File

@ -324,6 +324,7 @@ void* ExprContext::get_value(Expr* e, TupleRow* row) {
_result.float_val = v.val;
return &_result.float_val;
}
case TYPE_TIME:
case TYPE_DOUBLE: {
doris_udf::DoubleVal v = e->get_double_val(this, row);
if (v.is_null) {

View File

@ -78,6 +78,7 @@ Literal::Literal(const TExprNode& node) :
_value.float_val = node.float_literal.value;
break;
case TYPE_DOUBLE:
case TYPE_TIME:
DCHECK_EQ(node.node_type, TExprNodeType::FLOAT_LITERAL);
DCHECK(node.__isset.float_literal);
_value.double_val = node.float_literal.value;

View File

@ -821,7 +821,7 @@ FloatVal ScalarFnCall::get_float_val(ExprContext* context, TupleRow* row) {
}
DoubleVal ScalarFnCall::get_double_val(ExprContext* context, TupleRow* row) {
DCHECK_EQ(_type.type, TYPE_DOUBLE);
DCHECK(_type.type == TYPE_DOUBLE || _type.type == TYPE_TIME);
DCHECK(context != NULL);
if (_scalar_fn_wrapper == NULL) {
return interpret_eval<DoubleVal>(context, row);
@ -871,7 +871,6 @@ DecimalV2Val ScalarFnCall::get_decimalv2_val(ExprContext* context, TupleRow* row
return fn(context, row);
}
std::string ScalarFnCall::debug_string() const {
std::stringstream out;
out << "ScalarFnCall(udf_type=" << _fn.binary_type

View File

@ -0,0 +1,75 @@
// 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.
#include "exprs/time_operators.h"
#include <iomanip>
#include <sstream>
#include <math.h>
#include "exprs/anyval_util.h"
#include "exprs/case_expr.h"
#include "exprs/expr.h"
#include "runtime/tuple_row.h"
#include "util/string_parser.hpp"
#include "util/date_func.h"
namespace doris {
void TimeOperators::init() {
}
#define CAST_TIME_TO_INT(to_type, type_name) \
to_type TimeOperators::cast_to_##type_name( \
FunctionContext* context, const DoubleVal& val) { \
if (val.is_null) return to_type::null(); \
int time = (int) val.val ; \
int second = time % 60; \
int minute = time / 60 % 60; \
int hour = time / 3600; \
return to_type(hour * 10000 + minute * 100 + second); \
}
#define CAST_FROM_TIME() \
CAST_TIME_TO_INT(BooleanVal, boolean_val);\
CAST_TIME_TO_INT(TinyIntVal, tiny_int_val);\
CAST_TIME_TO_INT(SmallIntVal, small_int_val);\
CAST_TIME_TO_INT(IntVal, int_val);\
CAST_TIME_TO_INT(BigIntVal, big_int_val);\
CAST_TIME_TO_INT(LargeIntVal, large_int_val);\
CAST_TIME_TO_INT(FloatVal, float_val);\
CAST_TIME_TO_INT(DoubleVal, double_val);
CAST_FROM_TIME();
StringVal TimeOperators::cast_to_string_val(
FunctionContext* ctx, const DoubleVal& val) {
if (val.is_null) {
return StringVal::null();
}
return AnyValUtil::from_string_temp(ctx, time_str_from_int(val.val));
}
DateTimeVal TimeOperators::cast_to_datetime_val(
FunctionContext* context, const DoubleVal& val) {
return DateTimeVal::null();
}
DecimalVal TimeOperators::cast_to_decimal_val(
FunctionContext* context, const DoubleVal& val) {
return DecimalVal::null();
}
}

View File

@ -0,0 +1,48 @@
// 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.
#ifndef DORIS_BE_SRC_EXPRS_TIME_OPERATORS_H
#define DORIS_BE_SRC_EXPRS_TIME_OPERATORS_H
#include <stdint.h>
#include "udf/udf.h"
namespace doris {
class Expr;
struct ExprValue;
class TupleRow;
/// Implementation of the time operators. These include the cast,
/// arithmetic and binary operators.
class TimeOperators {
public:
static void init();
static BooleanVal cast_to_boolean_val(FunctionContext*, const DoubleVal&);
static TinyIntVal cast_to_tiny_int_val(FunctionContext*, const DoubleVal&);
static SmallIntVal cast_to_small_int_val(FunctionContext*, const DoubleVal&);
static IntVal cast_to_int_val(FunctionContext*, const DoubleVal&);
static BigIntVal cast_to_big_int_val(FunctionContext*, const DoubleVal&);
static LargeIntVal cast_to_large_int_val(FunctionContext*, const DoubleVal&);
static FloatVal cast_to_float_val(FunctionContext*, const DoubleVal&);
static DoubleVal cast_to_double_val(FunctionContext*, const DoubleVal&);
static StringVal cast_to_string_val(FunctionContext*, const DoubleVal&);
static DateTimeVal cast_to_datetime_val(FunctionContext*, const DoubleVal&);
static DecimalVal cast_to_decimal_val(FunctionContext*, const DoubleVal&);
};
}
#endif

View File

@ -463,18 +463,16 @@ IntVal TimestampFunctions::to_days(
}
// TODO(dhc): implement this funciton really
DateTimeVal TimestampFunctions::time_diff(
DoubleVal TimestampFunctions::time_diff(
FunctionContext* ctx, const DateTimeVal& ts_val1, const DateTimeVal& ts_val2) {
if (ts_val1.is_null || ts_val2.is_null) {
return DateTimeVal::null();
return DoubleVal::null();
}
const DateTimeValue& ts_value1 = DateTimeValue::from_datetime_val(ts_val1);
const DateTimeValue& ts_value2 = DateTimeValue::from_datetime_val(ts_val2);
DateTimeValue ts = ts_value1 - ts_value2;
ts.cast_to_time();
DateTimeVal result;
ts.to_datetime_val(&result);
return result;
int64_t timediff = ts_value1.unix_timestamp() - ts_value2.unix_timestamp();
return DoubleVal(timediff);
}
IntVal TimestampFunctions::date_diff(

View File

@ -84,7 +84,7 @@ public:
static doris_udf::IntVal date_diff(
doris_udf::FunctionContext* ctx, const doris_udf::DateTimeVal& ts_val1,
const doris_udf::DateTimeVal& ts_val2);
static doris_udf::DateTimeVal time_diff(
static doris_udf::DoubleVal time_diff(
doris_udf::FunctionContext* ctx, const doris_udf::DateTimeVal& ts_val1,
const doris_udf::DateTimeVal& ts_val2);

View File

@ -68,6 +68,9 @@ PrimitiveType thrift_to_type(TPrimitiveType::type ttype) {
case TPrimitiveType::DATETIME:
return TYPE_DATETIME;
case TPrimitiveType::TIME:
return TYPE_TIME;
case TPrimitiveType::VARCHAR:
return TYPE_VARCHAR;
@ -129,6 +132,9 @@ TPrimitiveType::type to_thrift(PrimitiveType ptype) {
case TYPE_DATETIME:
return TPrimitiveType::DATETIME;
case TYPE_TIME:
return TPrimitiveType::TIME;
case TYPE_VARCHAR:
return TPrimitiveType::VARCHAR;
@ -190,6 +196,9 @@ std::string type_to_string(PrimitiveType t) {
case TYPE_DATETIME:
return "DATETIME";
case TYPE_TIME:
return "TIME";
case TYPE_VARCHAR:
return "VARCHAR";

View File

@ -53,7 +53,9 @@ enum PrimitiveType {
TYPE_ARRAY, /* 17 */
TYPE_MAP, /* 18 */
TYPE_HLL, /* 19 */
TYPE_DECIMALV2 /* 20 */
TYPE_DECIMALV2, /* 20 */
TYPE_TIME, /* 21 */
};
inline bool is_enumeration_type(PrimitiveType type) {
@ -114,6 +116,7 @@ inline int get_byte_size(PrimitiveType type) {
return 4;
case TYPE_BIGINT:
case TYPE_TIME:
case TYPE_DOUBLE:
return 8;
@ -153,6 +156,7 @@ inline int get_real_byte_size(PrimitiveType type) {
return 4;
case TYPE_BIGINT:
case TYPE_TIME:
case TYPE_DOUBLE:
return 8;

View File

@ -262,6 +262,7 @@ void RawValue::write(const void* value, void* dst, const TypeDescriptor& type, M
break;
}
case TYPE_TIME:
case TYPE_DOUBLE: {
*reinterpret_cast<double*>(dst) = *reinterpret_cast<const double*>(value);
break;

View File

@ -25,6 +25,7 @@
#include "runtime/buffer_control_block.h"
#include "util/mysql_row_buffer.h"
#include "util/types.h"
#include "util/date_func.h"
#include "gen_cpp/PaloInternalService_types.h"
@ -104,6 +105,13 @@ Status ResultWriter::add_one_row(TupleRow* row) {
buf_ret = _row_buffer->push_double(*static_cast<double*>(item));
break;
case TYPE_TIME: {
double time = *static_cast<double *>(item);
std::string time_str = time_str_from_int((int) time);
buf_ret = _row_buffer->push_string(time_str.c_str(), time_str.size());
break;
}
case TYPE_DATE:
case TYPE_DATETIME: {
char buf[64];

View File

@ -261,6 +261,7 @@ struct TypeDescriptor {
case TYPE_BIGINT:
case TYPE_DOUBLE:
case TYPE_TIME:
return 8;
case TYPE_LARGEINT:

View File

@ -63,6 +63,22 @@ static uint24_t timestamp_from_date(const std::string& date_str) {
return uint24_t(value);
}
static std::string time_str_from_int(int time) {
std::stringstream time_ss;
if (time < 0) {
time_ss << "-";
time = -time;
}
int hour = time / 60 / 60;
int minute = time / 60 % 60;
int second = time % 60;
time_ss << std::setw(2) << std::setfill('0') << hour
<< ":" << std::setw(2) << std::setfill('0') << minute
<< ":" << std::setw(2) << std::setfill('0') << second;
return time_ss.str();
}
} // namespace doris
#endif // DORIS_BE_SRC_UTIL_DATE_FUNC_H

View File

@ -145,6 +145,7 @@ static void append_any_val_type(
case TYPE_FLOAT:
append_mangled_token("FloatVal", s);
break;
case TYPE_TIME:
case TYPE_DOUBLE:
append_mangled_token("DoubleVal", s);
break;

View File

@ -37,6 +37,22 @@ TEST_F(TimestampFunctionsTest, day_of_week_test) {
ASSERT_EQ(7, TimestampFunctions::day_of_week(context, tv).val);
}
TEST_F(TimestampFunctionsTest, time_diff_test) {
doris_udf::FunctionContext *context = new doris_udf::FunctionContext();
DateTimeValue dt1(20190718120000);
dt1.set_type(TIME_DATETIME);
doris_udf::DateTimeVal tv1;
dt1.to_datetime_val(&tv1);
DateTimeValue dt2(20190718130102);
dt2.set_type(TIME_DATETIME);
doris_udf::DateTimeVal tv2;
dt2.to_datetime_val(&tv2);
ASSERT_EQ(-3662, TimestampFunctions::time_diff(context, tv1, tv2).val);
}
}
int main(int argc, char** argv) {

View File

@ -2,7 +2,7 @@
## Syntax
`DATETIME DATEDIFF(expr1,expr2)`
`DATETIME DATEDIFF(DATETIME expr1,DATETIME expr2)`
## Description
@ -28,4 +28,4 @@ mysql> select datediff(CAST('2010-11-30 23:59:59' AS DATETIME), CAST('2010-12-31
+-----------------------------------------------------------------------------------+
| -31 |
+-----------------------------------------------------------------------------------+
```
```

View File

@ -0,0 +1,46 @@
# timediff
## Syntax
`TIME TIMEDIFF(DATETIME expr1, DATETIME expr2)`
## Description
TIMEDIFF��������DATETIME֮���IJ�ֵ
TIMEDIFF�������ر�ʾΪʱ��ֵ��expr1 - expr2�Ľ���������ֵΪTIME����
�������������ڴ�-838:59:59��838:59:59֮����TIMEֵ��Χ��
## Examples
```
mysql> SELECT TIMEDIFF(now(),utc_timestamp());
+----------------------------------+
| timediff(now(), utc_timestamp()) |
+----------------------------------+
| 08:00:00 |
+----------------------------------+
mysql> SELECT TIMEDIFF('2019-07-11 16:59:30','2019-07-11 16:59:21');
+--------------------------------------------------------+
| timediff('2019-07-11 16:59:30', '2019-07-11 16:59:21') |
+--------------------------------------------------------+
| 00:00:09 |
+--------------------------------------------------------+
mysql> SELECT TIMEDIFF('2019-01-01 00:00:00', '2009-01-01 00:00:00');
+--------------------------------------------------------+
| timediff('2019-01-01 00:00:00', '2009-01-01 00:00:00') |
+--------------------------------------------------------+
| 838:59:59 |
+--------------------------------------------------------+
mysql> SELECT TIMEDIFF('2019-01-01 00:00:00', NULL);
+---------------------------------------+
| timediff('2019-01-01 00:00:00', NULL) |
+---------------------------------------+
| NULL |
+---------------------------------------+
```

View File

@ -195,7 +195,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A
KW_CANCEL, KW_CASE, KW_CAST, KW_CHAIN, KW_CHAR, KW_CHARSET, KW_CLUSTER, KW_CLUSTERS,
KW_COLLATE, KW_COLLATION, KW_COLUMN, KW_COLUMNS, KW_COMMENT, KW_COMMIT, KW_COMMITTED,
KW_CONFIG, KW_CONNECTION, KW_CONNECTION_ID, KW_CONSISTENT, KW_COUNT, KW_CREATE, KW_CROSS, KW_CURRENT, KW_CURRENT_USER,
KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE,
KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_TIME, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE,
KW_DELETE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE,
KW_ELSE, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT,
KW_FALSE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FILE, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION,
@ -3202,6 +3202,8 @@ type ::=
{: RESULT = Type.DATE; :}
| KW_DATETIME
{: RESULT = Type.DATETIME; :}
| KW_TIME
{: RESULT = Type.TIME; :}
| KW_STRING
{: RESULT = ScalarType.createVarcharType(-1); :}
| KW_VARCHAR LPAREN INTEGER_LITERAL:len RPAREN

View File

@ -196,7 +196,7 @@ public class ArithmeticExpr extends Expr {
private Type findCommonType(Type t1, Type t2) {
PrimitiveType pt1 = t1.getPrimitiveType();
PrimitiveType pt2 = t2.getPrimitiveType();
if (pt1 == PrimitiveType.DOUBLE || pt2 == PrimitiveType.DOUBLE) {
return Type.DOUBLE;
} else if (pt1 == PrimitiveType.DECIMALV2 || pt2 == PrimitiveType.DECIMALV2) {

View File

@ -114,7 +114,11 @@ public class CastExpr extends Expr {
continue;
}
String beClass = toType.isDecimalV2() || fromType.isDecimalV2() ? "DecimalV2Operators" : "CastFunctions";
if (toType.isDecimal() || fromType.isDecimal()) beClass = "DecimalOperators";
if (toType.isDecimal() || fromType.isDecimal()) {
beClass = "DecimalOperators";
} else if (fromType.isTime()) {
beClass = "TimeOperators";
}
String typeName = Function.getUdfTypeName(toType.getPrimitiveType());
if (toType.getPrimitiveType() == PrimitiveType.DATE) {
typeName = "date_val";

View File

@ -337,4 +337,3 @@ public class IntLiteral extends LiteralExpr {
return literal;
}
}

View File

@ -449,6 +449,7 @@ public class Function implements Writable {
case FLOAT:
return "float_val";
case DOUBLE:
case TIME:
return "double_val";
case VARCHAR:
case CHAR:
@ -486,6 +487,7 @@ public class Function implements Writable {
case FLOAT:
return "FloatVal";
case DOUBLE:
case TIME:
return "DoubleVal";
case VARCHAR:
case CHAR:

View File

@ -53,6 +53,7 @@ public enum PrimitiveType {
DECIMALV2("DECIMALV2", 16, TPrimitiveType.DECIMALV2),
HLL("HLL", 16, TPrimitiveType.HLL),
TIME("TIME", 8, TPrimitiveType.TIME),
// Unsupported scalar types.
BINARY("BINARY", -1, TPrimitiveType.BINARY);
@ -79,6 +80,7 @@ public enum PrimitiveType {
builder.put(NULL_TYPE, DECIMALV2);
builder.put(NULL_TYPE, CHAR);
builder.put(NULL_TYPE, VARCHAR);
builder.put(NULL_TYPE, TIME);
// Boolean
builder.put(BOOLEAN, BOOLEAN);
builder.put(BOOLEAN, TINYINT);
@ -266,6 +268,10 @@ public enum PrimitiveType {
builder.put(HLL, HLL);
builder.put(HLL, VARCHAR);
//TIME
builder.put(TIME, TIME);
builder.put(TIME, DOUBLE);
implicitCastMap = builder.build();
}
@ -307,6 +313,7 @@ public enum PrimitiveType {
supportedTypes.add(CHAR);
supportedTypes.add(DATE);
supportedTypes.add(DATETIME);
supportedTypes.add(TIME);
supportedTypes.add(DECIMAL);
supportedTypes.add(DECIMALV2);
}
@ -360,6 +367,7 @@ public enum PrimitiveType {
compatibilityMatrix[NULL_TYPE.ordinal()][VARCHAR.ordinal()] = VARCHAR;
compatibilityMatrix[NULL_TYPE.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[NULL_TYPE.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[NULL_TYPE.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[BOOLEAN.ordinal()][BOOLEAN.ordinal()] = BOOLEAN;
compatibilityMatrix[BOOLEAN.ordinal()][TINYINT.ordinal()] = TINYINT;
@ -375,6 +383,7 @@ public enum PrimitiveType {
compatibilityMatrix[BOOLEAN.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[BOOLEAN.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[BOOLEAN.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[BOOLEAN.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[TINYINT.ordinal()][TINYINT.ordinal()] = TINYINT;
compatibilityMatrix[TINYINT.ordinal()][SMALLINT.ordinal()] = SMALLINT;
@ -389,6 +398,7 @@ public enum PrimitiveType {
compatibilityMatrix[TINYINT.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[TINYINT.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[TINYINT.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[TINYINT.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[SMALLINT.ordinal()][SMALLINT.ordinal()] = SMALLINT;
compatibilityMatrix[SMALLINT.ordinal()][INT.ordinal()] = INT;
@ -402,6 +412,7 @@ public enum PrimitiveType {
compatibilityMatrix[SMALLINT.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[SMALLINT.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[SMALLINT.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[SMALLINT.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[INT.ordinal()][INT.ordinal()] = INT;
compatibilityMatrix[INT.ordinal()][BIGINT.ordinal()] = BIGINT;
@ -414,6 +425,7 @@ public enum PrimitiveType {
compatibilityMatrix[INT.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[INT.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[INT.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[INT.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[BIGINT.ordinal()][BIGINT.ordinal()] = BIGINT;
compatibilityMatrix[BIGINT.ordinal()][LARGEINT.ordinal()] = LARGEINT;
@ -425,6 +437,7 @@ public enum PrimitiveType {
compatibilityMatrix[BIGINT.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[BIGINT.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[BIGINT.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[BIGINT.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[LARGEINT.ordinal()][LARGEINT.ordinal()] = LARGEINT;
compatibilityMatrix[LARGEINT.ordinal()][FLOAT.ordinal()] = DOUBLE;
@ -435,6 +448,7 @@ public enum PrimitiveType {
compatibilityMatrix[LARGEINT.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[LARGEINT.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[LARGEINT.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[LARGEINT.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[FLOAT.ordinal()][FLOAT.ordinal()] = FLOAT;
compatibilityMatrix[FLOAT.ordinal()][DOUBLE.ordinal()] = DOUBLE;
@ -444,6 +458,7 @@ public enum PrimitiveType {
compatibilityMatrix[FLOAT.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[FLOAT.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[FLOAT.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[FLOAT.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[DOUBLE.ordinal()][DOUBLE.ordinal()] = DOUBLE;
compatibilityMatrix[DOUBLE.ordinal()][DATE.ordinal()] = INVALID_TYPE;
@ -452,6 +467,7 @@ public enum PrimitiveType {
compatibilityMatrix[DOUBLE.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DOUBLE.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[DOUBLE.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[DOUBLE.ordinal()][TIME.ordinal()] = TIME;
compatibilityMatrix[DATE.ordinal()][DATE.ordinal()] = DATE;
compatibilityMatrix[DATE.ordinal()][DATETIME.ordinal()] = DATETIME;
@ -459,27 +475,35 @@ public enum PrimitiveType {
compatibilityMatrix[DATE.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATE.ordinal()][DECIMAL.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATE.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATE.ordinal()][TIME.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATETIME.ordinal()][DATETIME.ordinal()] = DATETIME;
compatibilityMatrix[DATETIME.ordinal()][CHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATETIME.ordinal()][VARCHAR.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATETIME.ordinal()][DECIMAL.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATETIME.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DATETIME.ordinal()][TIME.ordinal()] = INVALID_TYPE;
compatibilityMatrix[CHAR.ordinal()][CHAR.ordinal()] = CHAR;
compatibilityMatrix[CHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
compatibilityMatrix[CHAR.ordinal()][DECIMAL.ordinal()] = INVALID_TYPE;
compatibilityMatrix[CHAR.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
compatibilityMatrix[CHAR.ordinal()][TIME.ordinal()] = INVALID_TYPE;
compatibilityMatrix[VARCHAR.ordinal()][VARCHAR.ordinal()] = VARCHAR;
compatibilityMatrix[VARCHAR.ordinal()][DECIMAL.ordinal()] = INVALID_TYPE;
compatibilityMatrix[VARCHAR.ordinal()][DECIMALV2.ordinal()] = INVALID_TYPE;
compatibilityMatrix[VARCHAR.ordinal()][TIME.ordinal()] = INVALID_TYPE;
compatibilityMatrix[DECIMAL.ordinal()][DECIMAL.ordinal()] = DECIMAL;
compatibilityMatrix[DECIMALV2.ordinal()][DECIMALV2.ordinal()] = DECIMALV2;
compatibilityMatrix[DECIMALV2.ordinal()][DECIMAL.ordinal()] = DECIMALV2;
compatibilityMatrix[DECIMALV2.ordinal()][TIME.ordinal()] = INVALID_TYPE;
compatibilityMatrix[HLL.ordinal()][HLL.ordinal()] = HLL;
compatibilityMatrix[HLL.ordinal()][TIME.ordinal()] = INVALID_TYPE;
compatibilityMatrix[TIME.ordinal()][TIME.ordinal()] = TIME;
}
private static PrimitiveType[][] schemaChangeCompatibilityMatrix;
@ -659,6 +683,7 @@ public enum PrimitiveType {
case DATETIME:
case CHAR:
case VARCHAR:
case TIME:
return VARCHAR;
case DECIMAL:
return DECIMAL;
@ -738,6 +763,8 @@ public enum PrimitiveType {
return MysqlColType.MYSQL_TYPE_FLOAT;
case DOUBLE:
return MysqlColType.MYSQL_TYPE_DOUBLE;
case TIME:
return MysqlColType.MYSQL_TYPE_TIME;
case DATE:
return MysqlColType.MYSQL_TYPE_DATE;
case DATETIME: {

View File

@ -160,6 +160,7 @@ public class ScalarFunction extends Function {
beFn += "_float_val";
break;
case DOUBLE:
case TIME:
beFn += "_double_val";
break;
case CHAR:

View File

@ -125,6 +125,8 @@ public class ScalarType extends Type {
return DATE;
case DATETIME:
return DATETIME;
case TIME:
return TIME;
case DECIMAL:
return (ScalarType) createDecimalType();
case DECIMALV2:
@ -168,6 +170,8 @@ public class ScalarType extends Type {
return DATE;
case "DATETIME":
return DATETIME;
case "TIME":
return TIME;
case "DECIMAL":
return (ScalarType) createDecimalType();
case "DECIMALV2":
@ -721,6 +725,7 @@ public class ScalarType extends Type {
case INT:
return 4;
case BIGINT:
case TIME:
return 8;
case LARGEINT:
return 16;

View File

@ -62,6 +62,7 @@ public abstract class Type {
public static final ScalarType DOUBLE = new ScalarType(PrimitiveType.DOUBLE);
public static final ScalarType DATE = new ScalarType(PrimitiveType.DATE);
public static final ScalarType DATETIME = new ScalarType(PrimitiveType.DATETIME);
public static final ScalarType TIME = new ScalarType(PrimitiveType.TIME);
public static final ScalarType DEFAULT_DECIMAL = (ScalarType)
ScalarType.createDecimalType(ScalarType.DEFAULT_PRECISION,
ScalarType.DEFAULT_SCALE);
@ -116,6 +117,7 @@ public abstract class Type {
supportedTypes.add(DATETIME);
supportedTypes.add(DECIMAL);
supportedTypes.add(DECIMALV2);
supportedTypes.add(TIME);
}
public static ArrayList<ScalarType> getIntegerTypes() {
@ -236,6 +238,10 @@ public abstract class Type {
public boolean isDatetime() {
return isScalarType(PrimitiveType.DATETIME);
}
public boolean isTime() {
return isScalarType(PrimitiveType.TIME);
}
public boolean isComplexType() {
return isStructType() || isCollectionType();
@ -460,6 +466,8 @@ public abstract class Type {
return Type.DATE;
case DATETIME:
return Type.DATETIME;
case TIME:
return Type.TIME;
case DECIMAL:
return Type.DECIMAL;
case DECIMALV2:
@ -857,6 +865,7 @@ public abstract class Type {
if (t1 == PrimitiveType.NULL_TYPE || t2 == PrimitiveType.NULL_TYPE) continue;
if (t1 == PrimitiveType.DECIMAL || t2 == PrimitiveType.DECIMAL) continue;
if (t1 == PrimitiveType.DECIMALV2 || t2 == PrimitiveType.DECIMALV2) continue;
if (t1 == PrimitiveType.TIME || t2 == PrimitiveType.TIME) continue;
Preconditions.checkNotNull(compatibilityMatrix[i][j]);
}
}
@ -878,6 +887,7 @@ public abstract class Type {
return DOUBLE;
case DATE:
case DATETIME:
case TIME:
case CHAR:
case VARCHAR:
case HLL:
@ -943,6 +953,7 @@ public abstract class Type {
case DOUBLE:
case DATE:
case DATETIME:
case TIME:
case CHAR:
case VARCHAR:
case HLL:

View File

@ -63,6 +63,9 @@ public class TimeUtils {
public static Date MIN_DATETIME = null;
public static Date MAX_DATETIME = null;
public static int MIN_TIME;
public static int MAX_TIME;
static {
TIME_ZONE = new SimpleTimeZone(8 * 3600 * 1000, "");
@ -81,6 +84,7 @@ public class TimeUtils {
MIN_DATETIME = DATETIME_FORMAT.parse("1900-01-01 00:00:00");
MAX_DATETIME = DATETIME_FORMAT.parse("9999-12-31 23:59:59");
} catch (ParseException e) {
LOG.error("invalid date format", e);
System.exit(-1);

View File

@ -53,6 +53,11 @@ public class FEFunctions {
/**
* date and time function
*/
@FEFunction(name = "timediff", argTypes = { "DATETIME", "DATETIME" }, returnType = "TIME")
public static FloatLiteral timeDiff(LiteralExpr first, LiteralExpr second) throws AnalysisException {
long timediff = (getTime(first) - getTime(second)) / 1000;
return new FloatLiteral((double)timediff, Type.TIME);
}
@FEFunction(name = "datediff", argTypes = { "DATETIME", "DATETIME" }, returnType = "INT")
public static IntLiteral dateDiff(LiteralExpr first, LiteralExpr second) throws AnalysisException {

View File

@ -198,7 +198,7 @@ visible_functions = [
[['datediff'], 'INT', ['DATETIME', 'DATETIME'],
'_ZN5doris18TimestampFunctions9date_diffEPN9doris_udf'
'15FunctionContextERKNS1_11DateTimeValES6_'],
[['timediff'], 'DATETIME', ['DATETIME', 'DATETIME'],
[['timediff'], 'TIME', ['DATETIME', 'DATETIME'],
'_ZN5doris18TimestampFunctions9time_diffEPN9doris_udf'
'15FunctionContextERKNS1_11DateTimeValES6_'],

View File

@ -520,6 +520,7 @@ native_types = {
'VARCHAR': 'StringValue',
'DATE': 'Date',
'DATETIME': 'DateTime',
'TIME': 'double',
'DECIMAL': 'DecimalValue',
'DECIMALV2': 'DecimalV2Value',
}
@ -537,6 +538,7 @@ implemented_types = {
'VARCHAR': 'StringValue',
'DATE': 'DateTimeValue',
'DATETIME': 'DateTimeValue',
'TIME': 'double',
'DECIMAL': 'DecimalValue',
'DECIMALV2': 'DecimalV2Value',
}
@ -552,6 +554,7 @@ result_fields = {
'VARCHAR': 'string_val',
'DATE': 'datetime_val',
'DATETIME': 'datetime_val',
'TIME': 'double_val',
'DECIMAL': 'decimal_val',
'DECIMALV2': 'decimalv2_val',
}

View File

@ -62,6 +62,7 @@ native_types = {
'DATETIME': 'DateTime',
'DECIMAL': 'DecimalValue',
'DECIMALV2': 'DecimalV2Value',
'TIME': 'double'
}
thrift_preamble = '\

View File

@ -72,7 +72,8 @@ enum TPrimitiveType {
LARGEINT,
VARCHAR,
HLL,
DECIMALV2
DECIMALV2,
TIME
}
enum TTypeNodeType {