blob: ab8846e94f5da6f1a5726fa1576919b729b151ce [file] [log] [blame]
// 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 "vec/exprs/vexpr.h"
#include <gen_cpp/Descriptors_types.h>
#include <gen_cpp/PaloInternalService_types.h>
#include <gtest/gtest-message.h>
#include <gtest/gtest-test-part.h>
#include <gtest/gtest.h>
#include <stdint.h>
#include <string.h>
#include <sys/types.h>
#include <cmath>
#include <limits>
#include <new>
#include <string>
#include <type_traits>
#include "common/object_pool.h"
#include "exec/schema_scanner.h"
#include "gen_cpp/Exprs_types.h"
#include "gen_cpp/Types_types.h"
#include "gtest/gtest_pred_impl.h"
#include "runtime/define_primitive_type.h"
#include "runtime/descriptors.h"
#include "runtime/jsonb_value.h"
#include "runtime/large_int_value.h"
#include "runtime/runtime_state.h"
#include "testutil/desc_tbl_builder.h"
#include "util/timezone_utils.h"
#include "vec/core/block.h"
#include "vec/core/field.h"
#include "vec/core/types.h"
#include "vec/data_types/data_type_factory.hpp"
#include "vec/exprs/vexpr_context.h"
#include "vec/exprs/vliteral.h"
#include "vec/runtime/time_value.h"
#include "vec/runtime/timestamptz_value.h"
#include "vec/runtime/vdatetime_value.h"
#include "vec/utils/util.hpp"
TEST(TEST_VEXPR, ABSTEST) {
doris::ObjectPool object_pool;
doris::DescriptorTblBuilder builder(&object_pool);
builder.declare_tuple() << doris::vectorized::DataTypeFactory::instance().create_data_type(
doris::TYPE_INT, false)
<< doris::vectorized::DataTypeFactory::instance().create_data_type(
doris::TYPE_DOUBLE, false);
doris::DescriptorTbl* desc_tbl = builder.build();
auto tuple_desc = const_cast<doris::TupleDescriptor*>(desc_tbl->get_tuple_descriptor(0));
doris::RowDescriptor row_desc(tuple_desc);
std::string expr_json =
R"|({"1":{"lst":["rec",2,{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"abs"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"5":{"tf":0},"7":{"str":"abs(INT)"},"9":{"rec":{"1":{"str":"_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_6IntValE"}}},"11":{"i64":0}}}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0}}},"20":{"i32":-1},"23":{"i32":-1}}]}})|";
doris::TExpr exprx = apache::thrift::from_json_string<doris::TExpr>(expr_json);
doris::vectorized::VExprContextSPtr context;
static_cast<void>(doris::vectorized::VExpr::create_expr_tree(exprx, context));
doris::RuntimeState runtime_stat;
runtime_stat.set_desc_tbl(desc_tbl);
auto state = doris::Status::OK();
state = context->prepare(&runtime_stat, row_desc);
ASSERT_TRUE(state.ok());
state = context->open(&runtime_stat);
ASSERT_TRUE(state.ok());
}
// Only the unit test depend on this, but it is wrong, should not use TTupleDesc to create tuple desc, not
// use columndesc
static doris::TupleDescriptor* create_tuple_desc(
doris::ObjectPool* pool, std::vector<doris::SchemaScanner::ColumnDesc>& column_descs) {
using namespace doris;
int null_column = 0;
for (int i = 0; i < column_descs.size(); ++i) {
if (column_descs[i].is_null) {
null_column++;
}
}
int offset = (null_column + 7) / 8;
std::vector<SlotDescriptor*> slots;
int null_byte = 0;
int null_bit = 0;
for (int i = 0; i < column_descs.size(); ++i) {
TSlotDescriptor t_slot_desc;
if (column_descs[i].type == TYPE_DECIMALV2) {
t_slot_desc.__set_slotType(vectorized::DataTypeFactory::instance()
.create_data_type(TYPE_DECIMALV2, false, 27, 9)
->to_thrift());
} else {
auto descriptor = vectorized::DataTypeFactory::instance().create_data_type(
column_descs[i].type, false,
column_descs[i].precision >= 0 ? column_descs[i].precision : 0,
column_descs[i].scale >= 0 ? column_descs[i].scale : 0);
t_slot_desc.__set_slotType(descriptor->to_thrift());
}
t_slot_desc.__set_colName(column_descs[i].name);
t_slot_desc.__set_columnPos(i);
t_slot_desc.__set_byteOffset(offset);
if (column_descs[i].is_null) {
t_slot_desc.__set_nullIndicatorByte(null_byte);
t_slot_desc.__set_nullIndicatorBit(null_bit);
null_bit = (null_bit + 1) % 8;
if (0 == null_bit) {
null_byte++;
}
} else {
t_slot_desc.__set_nullIndicatorByte(0);
t_slot_desc.__set_nullIndicatorBit(-1);
}
t_slot_desc.id = i;
t_slot_desc.__set_slotIdx(i);
t_slot_desc.__set_isMaterialized(true);
SlotDescriptor* slot = pool->add(new (std::nothrow) SlotDescriptor(t_slot_desc));
slots.push_back(slot);
offset += column_descs[i].size;
}
TTupleDescriptor t_tuple_desc;
t_tuple_desc.__set_byteSize(offset);
t_tuple_desc.__set_numNullBytes(0);
doris::TupleDescriptor* tuple_desc =
pool->add(new (std::nothrow) doris::TupleDescriptor(t_tuple_desc));
for (int i = 0; i < slots.size(); ++i) {
tuple_desc->add_slot(slots[i]);
}
return tuple_desc;
}
TEST(TEST_VEXPR, ABSTEST2) {
using namespace doris;
std::vector<doris::SchemaScanner::ColumnDesc> column_descs = {
{"k1", TYPE_INT, sizeof(int32_t), false}};
ObjectPool object_pool;
doris::TupleDescriptor* tuple_desc = create_tuple_desc(&object_pool, column_descs);
RowDescriptor row_desc(tuple_desc);
std::string expr_json =
R"|({"1":{"lst":["rec",2,{"1":{"i32":20},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"4":{"i32":1},"20":{"i32":-1},"26":{"rec":{"1":{"rec":{"2":{"str":"abs"}}},"2":{"i32":0},"3":{"lst":["rec",1,{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}]},"4":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":6}}}}]}}},"5":{"tf":0},"7":{"str":"abs(INT)"},"9":{"rec":{"1":{"str":"_ZN5doris13MathFunctions3absEPN9doris_udf15FunctionContextERKNS1_6IntValE"}}},"11":{"i64":0}}}},{"1":{"i32":16},"2":{"rec":{"1":{"lst":["rec",1,{"1":{"i32":0},"2":{"rec":{"1":{"i32":5}}}}]}}},"4":{"i32":0},"15":{"rec":{"1":{"i32":0},"2":{"i32":0}}},"20":{"i32":-1},"23":{"i32":-1}}]}})|";
TExpr exprx = apache::thrift::from_json_string<TExpr>(expr_json);
doris::vectorized::VExprContextSPtr context;
static_cast<void>(doris::vectorized::VExpr::create_expr_tree(exprx, context));
doris::RuntimeState runtime_stat;
DescriptorTbl desc_tbl;
desc_tbl._slot_desc_map[0] = tuple_desc->slots()[0];
runtime_stat.set_desc_tbl(&desc_tbl);
auto state = Status::OK();
state = context->prepare(&runtime_stat, row_desc);
ASSERT_TRUE(state.ok());
state = context->open(&runtime_stat);
ASSERT_TRUE(state.ok());
}
namespace doris {
template <PrimitiveType T>
struct literal_traits {};
template <>
struct literal_traits<TYPE_BOOLEAN> {
const static TPrimitiveType::type ttype = TPrimitiveType::BOOLEAN;
const static TExprNodeType::type tnode_type = TExprNodeType::BOOL_LITERAL;
using CXXType = bool;
};
template <>
struct literal_traits<TYPE_SMALLINT> {
const static TPrimitiveType::type ttype = TPrimitiveType::SMALLINT;
const static TExprNodeType::type tnode_type = TExprNodeType::INT_LITERAL;
using CXXType = int16_t;
};
template <>
struct literal_traits<TYPE_INT> {
const static TPrimitiveType::type ttype = TPrimitiveType::INT;
const static TExprNodeType::type tnode_type = TExprNodeType::INT_LITERAL;
using CXXType = int32_t;
};
template <>
struct literal_traits<TYPE_BIGINT> {
const static TPrimitiveType::type ttype = TPrimitiveType::BIGINT;
const static TExprNodeType::type tnode_type = TExprNodeType::INT_LITERAL;
using CXXType = int64_t;
};
template <>
struct literal_traits<TYPE_LARGEINT> {
const static TPrimitiveType::type ttype = TPrimitiveType::LARGEINT;
const static TExprNodeType::type tnode_type = TExprNodeType::LARGE_INT_LITERAL;
using CXXType = __int128_t;
};
template <>
struct literal_traits<TYPE_FLOAT> {
const static TPrimitiveType::type ttype = TPrimitiveType::FLOAT;
const static TExprNodeType::type tnode_type = TExprNodeType::FLOAT_LITERAL;
using CXXType = float;
};
template <>
struct literal_traits<TYPE_DOUBLE> {
const static TPrimitiveType::type ttype = TPrimitiveType::FLOAT;
const static TExprNodeType::type tnode_type = TExprNodeType::FLOAT_LITERAL;
using CXXType = float;
};
template <>
struct literal_traits<TYPE_DATETIME> {
const static TPrimitiveType::type ttype = TPrimitiveType::DATETIME;
const static TExprNodeType::type tnode_type = TExprNodeType::DATE_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_DATETIMEV2> {
const static TPrimitiveType::type ttype = TPrimitiveType::DATETIMEV2;
const static TExprNodeType::type tnode_type = TExprNodeType::DATE_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_TIMESTAMPTZ> {
const static TPrimitiveType::type ttype = TPrimitiveType::TIMESTAMPTZ;
const static TExprNodeType::type tnode_type = TExprNodeType::DATE_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_DATE> {
const static TPrimitiveType::type ttype = TPrimitiveType::DATE;
const static TExprNodeType::type tnode_type = TExprNodeType::DATE_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_DATEV2> {
const static TPrimitiveType::type ttype = TPrimitiveType::DATEV2;
const static TExprNodeType::type tnode_type = TExprNodeType::DATE_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_JSONB> {
const static TPrimitiveType::type ttype = TPrimitiveType::JSONB;
const static TExprNodeType::type tnode_type = TExprNodeType::JSON_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_STRING> {
const static TPrimitiveType::type ttype = TPrimitiveType::STRING;
const static TExprNodeType::type tnode_type = TExprNodeType::STRING_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_DECIMALV2> {
const static TPrimitiveType::type ttype = TPrimitiveType::DECIMALV2;
const static TExprNodeType::type tnode_type = TExprNodeType::DECIMAL_LITERAL;
using CXXType = std::string;
};
template <>
struct literal_traits<TYPE_TIMEV2> {
const static TPrimitiveType::type ttype = TPrimitiveType::TIMEV2;
const static TExprNodeType::type tnode_type = TExprNodeType::TIMEV2_LITERAL;
using CXXType = double;
};
//======================== set literal ===================================
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires std::is_integral_v<U>
void set_literal(TExprNode& node, const U& value) {
TIntLiteral int_literal;
int_literal.__set_value(value);
node.__set_int_literal(int_literal);
}
template <>
void set_literal<TYPE_BOOLEAN, bool>(TExprNode& node, const bool& value) {
TBoolLiteral bool_literal;
bool_literal.__set_value(value);
node.__set_bool_literal(bool_literal);
}
template <>
void set_literal<TYPE_LARGEINT, __int128_t>(TExprNode& node, const __int128_t& value) {
TLargeIntLiteral largeIntLiteral;
largeIntLiteral.__set_value(LargeIntValue::to_string(value));
node.__set_large_int_literal(largeIntLiteral);
}
// std::is_same<U, std::string>::value
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_DATETIME)
void set_literal(TExprNode& node, const U& value) {
TDateLiteral date_literal;
date_literal.__set_value(value);
node.__set_date_literal(date_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_DATETIMEV2)
void set_literal(TExprNode& node, const U& value) {
TDateLiteral date_literal;
date_literal.__set_value(value);
node.__set_date_literal(date_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_TIMESTAMPTZ)
void set_literal(TExprNode& node, const U& value) {
TDateLiteral date_literal;
date_literal.__set_value(value);
node.__set_date_literal(date_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_DATE)
void set_literal(TExprNode& node, const U& value) {
TDateLiteral date_literal;
date_literal.__set_value(value);
node.__set_date_literal(date_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_DATEV2)
void set_literal(TExprNode& node, const U& value) {
TDateLiteral date_literal;
date_literal.__set_value(value);
node.__set_date_literal(date_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_JSONB)
void set_literal(TExprNode& node, const U& value) {
TJsonLiteral jsonb_literal;
jsonb_literal.__set_value(value);
node.__set_json_literal(jsonb_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_STRING)
void set_literal(TExprNode& node, const U& value) {
TStringLiteral string_literal;
string_literal.__set_value(value);
node.__set_string_literal(string_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(std::numeric_limits<U>::is_iec559 && T != TYPE_TIMEV2)
void set_literal(TExprNode& node, const U& value) {
TFloatLiteral floatLiteral;
floatLiteral.__set_value(value);
node.__set_float_literal(floatLiteral);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_DECIMALV2)
void set_literal(TExprNode& node, const U& value) {
TDecimalLiteral decimal_literal;
decimal_literal.__set_value(value);
node.__set_decimal_literal(decimal_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
requires(T == TYPE_TIMEV2)
void set_literal(TExprNode& node, const U& value) {
TTimeV2Literal timev2_literal;
timev2_literal.__set_value(value);
node.__set_timev2_literal(timev2_literal);
}
template <PrimitiveType T, class U = typename literal_traits<T>::CXXType>
doris::TExprNode create_literal(const U& value, int scale = 9) {
TExprNode node;
TTypeDesc type_desc;
TTypeNode type_node;
std::vector<TTypeNode> type_nodes;
type_nodes.emplace_back();
TScalarType scalar_type;
scalar_type.__set_precision(27);
scalar_type.__set_scale(scale);
scalar_type.__set_len(20);
scalar_type.__set_type(literal_traits<T>::ttype);
type_nodes[0].__set_scalar_type(scalar_type);
type_desc.__set_types(type_nodes);
node.__set_type(type_desc);
node.__set_node_type(literal_traits<T>::tnode_type);
set_literal<T, U>(node, value);
return node;
}
} // namespace doris
TEST(TEST_VEXPR, LITERALTEST) {
using namespace doris;
using namespace doris::vectorized;
// bool
{
VLiteral literal(create_literal<TYPE_BOOLEAN>(true));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<uint8_t>();
EXPECT_EQ(v, true);
EXPECT_EQ("1", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_BOOLEAN, 0, 0), true);
EXPECT_EQ("1", node->value());
}
// smallint
{
VLiteral literal(create_literal<TYPE_SMALLINT>(1024));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<int16_t>();
EXPECT_EQ(v, 1024);
EXPECT_EQ("1024", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_SMALLINT, 0, 0), true);
EXPECT_EQ("1024", node->value());
}
// int
{
VLiteral literal(create_literal<TYPE_INT>(1024));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<int32_t>();
EXPECT_EQ(v, 1024);
EXPECT_EQ("1024", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_INT, 0, 0), true);
EXPECT_EQ("1024", node->value());
}
// bigint
{
VLiteral literal(create_literal<TYPE_BIGINT>(1024));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<int64_t>();
EXPECT_EQ(v, 1024);
EXPECT_EQ("1024", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_BIGINT, 0, 0), true);
EXPECT_EQ("1024", node->value());
}
// large int
{
VLiteral literal(create_literal<TYPE_LARGEINT, __int128_t>(1024));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<__int128_t>();
EXPECT_EQ(v, 1024);
EXPECT_EQ("1024", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_LARGEINT, 0, 0), true);
EXPECT_EQ("1024", node->value());
}
// float
{
VLiteral literal(create_literal<TYPE_FLOAT, float>(1024.0f));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<double>();
EXPECT_FLOAT_EQ(v, 1024.0f);
EXPECT_EQ("1024", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_FLOAT, 0, 0), true);
EXPECT_EQ("1024", node->value());
}
// double
{
VLiteral literal(create_literal<TYPE_DOUBLE, double>(1024.0));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<double>();
EXPECT_FLOAT_EQ(v, 1024.0);
EXPECT_EQ("1024", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_DOUBLE, 0, 0), true);
EXPECT_EQ("1024", node->value());
}
// datetime
{
VecDateTimeValue data_time_value;
const char* date = "20210407000000";
data_time_value.from_date_str(date, strlen(date));
std::cout << data_time_value.type() << std::endl;
__int64_t dt;
memcpy(&dt, &data_time_value, sizeof(__int64_t));
VLiteral literal(create_literal<TYPE_DATETIME, std::string>(std::string(date)));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<__int64_t>();
EXPECT_EQ(v, dt);
EXPECT_EQ("2021-04-07 00:00:00", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_DATETIME, 0, 0), true);
EXPECT_EQ("2021-04-07 00:00:00", node->value());
}
// datetimev2
{
uint16_t year = 1997;
uint8_t month = 11;
uint8_t day = 18;
uint8_t hour = 9;
uint8_t minute = 12;
uint8_t second = 46;
uint32_t microsecond = 999999; // target scale is 4, so the microsecond will be rounded up
DateV2Value<DateTimeV2ValueType> datetime_v2;
datetime_v2.unchecked_set_time(year, month, day, hour, minute, second, microsecond);
std::string date = datetime_v2.to_string();
VLiteral literal(create_literal<TYPE_DATETIMEV2, std::string>(date, 4));
Block block;
int ret = -1;
EXPECT_TRUE(literal.execute(nullptr, &block, &ret).ok());
EXPECT_EQ("1997-11-18 09:12:47.0000", literal.value());
auto ctn = block.safe_get_by_position(ret);
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_DATETIMEV2, 0, 4), true);
EXPECT_EQ("1997-11-18 09:12:47.0000", node->value());
}
// timestamptz
{
TimezoneUtils::load_timezones_to_cache();
uint16_t year = 1997;
uint8_t month = 11;
uint8_t day = 18;
uint8_t hour = 9;
uint8_t minute = 12;
uint8_t second = 46;
uint32_t microsecond = 999999; // target scale is 4, so the microsecond will be rounded up
int scale = 6;
std::string tz_str = "+08:00";
cctz::time_zone tz;
TimezoneUtils::find_cctz_time_zone(tz_str, tz);
DateV2Value<DateTimeV2ValueType> datetime_v2;
datetime_v2.unchecked_set_time(year, month, day, hour, minute, second, microsecond);
TimestampTzValue tz_value;
tz_value.from_datetime(datetime_v2, tz, scale, scale);
std::string tz_value_str = tz_value.to_string(tz, scale);
VLiteral literal(create_literal<TYPE_TIMESTAMPTZ, std::string>(tz_value_str, scale));
Block block;
int ret = -1;
EXPECT_TRUE(literal.execute(nullptr, &block, &ret).ok());
EXPECT_EQ("1997-11-18 01:12:46.999999+00:00", literal.value());
auto ctn = block.safe_get_by_position(ret);
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_TIMESTAMPTZ, 0, scale), true);
EXPECT_EQ("1997-11-18 01:12:46.999999+00:00", node->value());
node = std::make_shared<VLiteral>(
create_texpr_node_from(&tz_value, TYPE_TIMESTAMPTZ, 0, scale), true);
EXPECT_EQ("1997-11-18 01:12:46.999999+00:00", node->value());
}
// date
{
VecDateTimeValue date_time_value;
date_time_value.set_type(TIME_DATE);
const char* date = "20210407";
date_time_value.from_date_str(date, strlen(date));
__int64_t dt;
memcpy(&dt, &date_time_value, sizeof(__int64_t));
VLiteral literal(create_literal<TYPE_DATE, std::string>(std::string(date)));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<__int64_t>();
EXPECT_EQ(v, dt);
EXPECT_EQ("2021-04-07", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_DATE, 0, 0), true);
EXPECT_EQ("2021-04-07", node->value());
}
// datev2
{
DateV2Value<DateV2ValueType> data_time_value;
const char* date = "20210407";
data_time_value.from_date_str(date, strlen(date));
uint32_t dt;
memcpy(&dt, &data_time_value, sizeof(uint32_t));
VLiteral literal(create_literal<TYPE_DATEV2, std::string>(std::string(date)));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<uint32_t>();
EXPECT_EQ(v, dt);
EXPECT_EQ("2021-04-07", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_DATEV2, 0, 0), true);
EXPECT_EQ("2021-04-07", node->value());
}
config::allow_zero_date = true;
{
DateV2Value<DateV2ValueType> data_time_value;
const char* date = "00000000";
EXPECT_EQ(data_time_value.from_date_str(date, strlen(date), -1, true), true);
DateV2Value<DateV2ValueType> data_time_value1;
const char* date1 = "00000101";
EXPECT_EQ(data_time_value1.from_date_str(date1, strlen(date1), -1, true), true);
EXPECT_EQ(data_time_value.to_int64(), data_time_value1.to_int64());
EXPECT_EQ(data_time_value.from_date_str(date, strlen(date)), true);
}
{
DateV2Value<DateTimeV2ValueType> data_time_value;
const char* date = "00000000111111";
EXPECT_EQ(data_time_value.from_date_str(date, strlen(date), -1, true), true);
DateV2Value<DateTimeV2ValueType> data_time_value1;
const char* date1 = "00000101111111";
EXPECT_EQ(data_time_value1.from_date_str(date1, strlen(date1), -1, true), true);
EXPECT_EQ(data_time_value.to_int64(), data_time_value1.to_int64());
EXPECT_EQ(data_time_value.from_date_str(date, strlen(date)), true);
}
// jsonb
{
std::string j = R"([null,true,false,100,6.18,"abc"])";
VLiteral literal(create_literal<TYPE_JSONB, std::string>(j));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
EXPECT_EQ(j, literal.value());
}
// string
{
std::string s = "I am Amory, 24";
VLiteral literal(create_literal<TYPE_STRING, std::string>(std::string("I am Amory, 24")));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<String>();
EXPECT_EQ(v, s);
EXPECT_EQ(s, literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_STRING, 0, 0), true);
EXPECT_EQ(s, node->value());
}
// decimalv2
{
VLiteral literal(create_literal<TYPE_DECIMALV2, std::string>(std::string("1234.56")));
Block block;
int ret = -1;
static_cast<void>(literal.execute(nullptr, &block, &ret));
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<DecimalField<Decimal128V2>>();
EXPECT_FLOAT_EQ(((double)v.get_value()) / (std::pow(10, v.get_scale())), 1234.56);
EXPECT_EQ("1234.560000000", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_DECIMALV2, 27, 9), true);
EXPECT_EQ("1234.560000000", node->value());
}
// timev2
{
VLiteral literal(create_literal<TYPE_TIMEV2, double>(12123400, 4));
Block block;
int ret = -1;
EXPECT_TRUE(literal.execute(nullptr, &block, &ret).ok());
auto ctn = block.safe_get_by_position(ret);
auto v = (*ctn.column)[0].get<Float64>();
EXPECT_FLOAT_EQ(v / 1000000, 12.1234);
EXPECT_EQ("00:00:12.1234", literal.value());
auto node = std::make_shared<VLiteral>(
create_texpr_node_from((*ctn.column)[0], TYPE_TIMEV2, 0, 0), true);
EXPECT_EQ("00:00:12", node->value());
}
// deciaml32
{
auto src_col = ColumnDecimal32::create(9, 2);
auto& src_data = src_col->get_data();
src_data.resize(0);
src_data.push_back(Decimal32(12345)); // 123.45
src_data.push_back(Decimal32(-12345)); // -123.45
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](0), TYPE_DECIMAL32, 9, 2), true);
EXPECT_EQ("123.45", node->value());
}
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](1), TYPE_DECIMAL32, 9, 2), true);
EXPECT_EQ("-123.45", node->value());
}
}
// decimal64
{
auto src_col = ColumnDecimal64::create(18, 4);
auto& src_data = src_col->get_data();
src_data.resize(0);
src_data.push_back(Decimal64(123456789)); // 12345.6789
src_data.push_back(Decimal64(-123456789)); // -12345.6789
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](0), TYPE_DECIMAL64, 18, 4), true);
EXPECT_EQ("12345.6789", node->value());
}
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](1), TYPE_DECIMAL64, 18, 4), true);
EXPECT_EQ("-12345.6789", node->value());
}
}
// decimal128
{
auto src_col = ColumnDecimal128V3::create(38, 6);
auto& src_data = src_col->get_data();
src_data.resize(0);
src_data.push_back(Decimal128V3(123456789012345));
src_data.push_back(Decimal128V3(-123456789012345));
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](0), TYPE_DECIMAL128I, 38, 6), true);
EXPECT_EQ("123456789.012345", node->value());
}
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](1), TYPE_DECIMAL128I, 38, 6), true);
EXPECT_EQ("-123456789.012345", node->value());
}
}
// decimal256
{
auto src_col = ColumnDecimal256::create(76, 8);
auto& src_data = src_col->get_data();
src_data.resize(0);
src_data.push_back(Decimal256(1));
src_data.push_back(Decimal256(123456789));
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](0), TYPE_DECIMAL256, 76, 8), true);
EXPECT_EQ("0.00000001", node->value());
}
{
auto node = std::make_shared<VLiteral>(
create_texpr_node_from(src_col->operator[](1), TYPE_DECIMAL256, 76, 8), true);
EXPECT_EQ("1.23456789", node->value());
}
}
config::allow_zero_date = false;
}