[refactor](remove unused code) remove many xxxVal structure (#16143)

remove many xxxVal structure
remove BetaRowsetWriter::_add_row
remove anyval_util.cpp
remove non-vectorized geo functions
remove non-vectorized like predicate
Co-authored-by: yiguolei <yiguolei@gmail.com>
This commit is contained in:
yiguolei
2023-01-28 14:17:43 +08:00
committed by GitHub
parent 4e64ff6329
commit e49766483e
43 changed files with 166 additions and 3140 deletions

View File

@ -23,10 +23,8 @@
#include "common/config.h"
#include "common/logging.h"
#include "exprs/like_predicate.h"
#include "exprs/math_functions.h"
#include "exprs/string_functions.h"
#include "geo/geo_functions.h"
#include "olap/options.h"
#include "runtime/block_spill_manager.h"
#include "runtime/exec_env.h"
@ -359,8 +357,6 @@ void Daemon::init(int argc, char** argv, const std::vector<StorePath>& paths) {
DiskInfo::init();
MemInfo::init();
UserFunctionCache::instance()->init(config::user_function_dir);
LikePredicate::init();
GeoFunctions::init();
LOG(INFO) << CpuInfo::debug_string();
LOG(INFO) << DiskInfo::debug_string();

View File

@ -29,55 +29,6 @@ namespace doris {
typedef bool (*CompareLargeFunc)(const void*, const void*);
template <class T>
inline bool compare_large(const void* lhs, const void* rhs) {
return *reinterpret_cast<const T*>(lhs) > *reinterpret_cast<const T*>(rhs);
}
inline CompareLargeFunc get_compare_func(PrimitiveType type) {
switch (type) {
case TYPE_BOOLEAN:
return compare_large<bool>;
case TYPE_TINYINT:
return compare_large<int8_t>;
case TYPE_SMALLINT:
return compare_large<int16_t>;
case TYPE_INT:
return compare_large<int32_t>;
case TYPE_BIGINT:
return compare_large<int64_t>;
case TYPE_LARGEINT:
return compare_large<__int128>;
case TYPE_FLOAT:
return compare_large<float>;
case TYPE_DOUBLE:
return compare_large<double>;
case TYPE_DATE:
case TYPE_DATETIME:
return compare_large<DateTimeValue>;
case TYPE_DECIMALV2:
return compare_large<DecimalV2Value>;
case TYPE_CHAR:
case TYPE_VARCHAR:
case TYPE_STRING:
return compare_large<StringRef>;
default:
DCHECK(false) << "Unsupported Compare type";
}
__builtin_unreachable();
}
static const char* NEGATIVE_INFINITY = "-oo";
static const char* POSITIVE_INFINITY = "+oo";
@ -145,45 +96,6 @@ enum SQLFilterOp {
FILTER_NOT_IN = 5
};
inline int get_olap_size(PrimitiveType type) {
switch (type) {
case TYPE_BOOLEAN:
case TYPE_TINYINT: {
return 1;
}
case TYPE_SMALLINT: {
return 2;
}
case TYPE_DATE: {
return 3;
}
case TYPE_INT:
case TYPE_FLOAT: {
return 4;
}
case TYPE_BIGINT:
case TYPE_LARGEINT:
case TYPE_DOUBLE:
case TYPE_DATETIME: {
return 8;
}
case TYPE_DECIMALV2: {
return 12;
}
default: {
DCHECK(false);
}
}
return 0;
}
template <PrimitiveType>
static constexpr bool always_false_v = false;

View File

@ -22,12 +22,10 @@ set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/exprs")
set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/exprs")
add_library(Exprs
anyval_util.cpp
block_bloom_filter_avx_impl.cc
block_bloom_filter_impl.cc
runtime_filter.cpp
runtime_filter_rpc.cpp
like_predicate.cpp
math_functions.cpp
rpc_fn_comm.cpp
string_functions.cpp

View File

@ -1,229 +0,0 @@
// 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.
// This file is copied from
// https://github.com/apache/impala/blob/branch-2.9.0/be/src/exprs/anyval-util.cc
// and modified by Doris
#include "exprs/anyval_util.h"
#include "common/object_pool.h"
#include "runtime/mem_pool.h"
#include "runtime/memory/mem_tracker.h"
namespace doris {
using doris_udf::BooleanVal;
using doris_udf::TinyIntVal;
using doris_udf::SmallIntVal;
using doris_udf::IntVal;
using doris_udf::BigIntVal;
using doris_udf::LargeIntVal;
using doris_udf::FloatVal;
using doris_udf::DoubleVal;
using doris_udf::DecimalV2Val;
using doris_udf::DateTimeVal;
using doris_udf::StringVal;
using doris_udf::AnyVal;
using doris_udf::DateV2Val;
using doris_udf::DateTimeV2Val;
Status allocate_any_val(RuntimeState* state, MemPool* pool, const TypeDescriptor& type,
const std::string& mem_limit_exceeded_msg, AnyVal** result) {
const int anyval_size = AnyValUtil::any_val_size(type);
const int anyval_alignment = AnyValUtil::any_val_alignment(type);
*result = reinterpret_cast<AnyVal*>(pool->try_allocate_aligned(anyval_size, anyval_alignment));
if (*result == nullptr) {
RETURN_LIMIT_EXCEEDED(state, mem_limit_exceeded_msg, anyval_size);
}
memset(static_cast<void*>(*result), 0, anyval_size);
return Status::OK();
}
AnyVal* create_any_val(ObjectPool* pool, const TypeDescriptor& type) {
switch (type.type) {
case TYPE_NULL:
return pool->add(new AnyVal);
case TYPE_BOOLEAN:
return pool->add(new BooleanVal);
case TYPE_TINYINT:
return pool->add(new TinyIntVal);
case TYPE_SMALLINT:
return pool->add(new SmallIntVal);
case TYPE_INT:
return pool->add(new IntVal);
case TYPE_BIGINT:
return pool->add(new BigIntVal);
case TYPE_LARGEINT:
return pool->add(new LargeIntVal);
case TYPE_FLOAT:
return pool->add(new FloatVal);
case TYPE_TIME:
case TYPE_TIMEV2:
case TYPE_DOUBLE:
return pool->add(new DoubleVal);
case TYPE_CHAR:
case TYPE_HLL:
case TYPE_VARCHAR:
case TYPE_OBJECT:
case TYPE_QUANTILE_STATE:
case TYPE_STRING:
return pool->add(new StringVal);
case TYPE_DECIMALV2:
return pool->add(new DecimalV2Val);
case TYPE_DECIMAL32:
return pool->add(new IntVal);
case TYPE_DECIMAL64:
return pool->add(new BigIntVal);
case TYPE_DECIMAL128I:
return pool->add(new LargeIntVal);
case TYPE_DATE:
return pool->add(new DateTimeVal);
case TYPE_DATEV2:
return pool->add(new DateV2Val);
case TYPE_DATETIMEV2:
return pool->add(new DateTimeV2Val);
case TYPE_DATETIME:
return pool->add(new DateTimeVal);
case TYPE_ARRAY:
return pool->add(new CollectionVal);
default:
DCHECK(false) << "Unsupported type: " << type.type;
return nullptr;
}
}
FunctionContext::TypeDesc AnyValUtil::column_type_to_type_desc(const TypeDescriptor& type) {
FunctionContext::TypeDesc out;
switch (type.type) {
case TYPE_BOOLEAN:
out.type = FunctionContext::TYPE_BOOLEAN;
break;
case TYPE_TINYINT:
out.type = FunctionContext::TYPE_TINYINT;
break;
case TYPE_SMALLINT:
out.type = FunctionContext::TYPE_SMALLINT;
break;
case TYPE_INT:
out.type = FunctionContext::TYPE_INT;
break;
case TYPE_BIGINT:
out.type = FunctionContext::TYPE_BIGINT;
break;
case TYPE_LARGEINT:
out.type = FunctionContext::TYPE_LARGEINT;
break;
case TYPE_FLOAT:
out.type = FunctionContext::TYPE_FLOAT;
break;
case TYPE_TIME:
case TYPE_TIMEV2:
case TYPE_DOUBLE:
out.type = FunctionContext::TYPE_DOUBLE;
break;
case TYPE_DATE:
out.type = FunctionContext::TYPE_DATE;
break;
case TYPE_DATETIME:
out.type = FunctionContext::TYPE_DATETIME;
break;
case TYPE_DATEV2:
out.type = FunctionContext::TYPE_DATEV2;
break;
case TYPE_DATETIMEV2:
out.type = FunctionContext::TYPE_DATETIMEV2;
break;
case TYPE_DECIMAL32:
out.type = FunctionContext::TYPE_DECIMAL32;
out.precision = type.precision;
out.scale = type.scale;
break;
case TYPE_DECIMAL64:
out.type = FunctionContext::TYPE_DECIMAL64;
out.precision = type.precision;
out.scale = type.scale;
break;
case TYPE_DECIMAL128I:
out.type = FunctionContext::TYPE_DECIMAL128I;
out.precision = type.precision;
out.scale = type.scale;
break;
case TYPE_VARCHAR:
out.type = FunctionContext::TYPE_VARCHAR;
out.len = type.len;
break;
case TYPE_HLL:
out.type = FunctionContext::TYPE_HLL;
out.len = type.len;
break;
case TYPE_OBJECT:
out.type = FunctionContext::TYPE_OBJECT;
// FIXME(cmy): is this fallthrough meaningful?
case TYPE_QUANTILE_STATE:
out.type = FunctionContext::TYPE_QUANTILE_STATE;
break;
case TYPE_CHAR:
out.type = FunctionContext::TYPE_CHAR;
out.len = type.len;
break;
case TYPE_DECIMALV2:
out.type = FunctionContext::TYPE_DECIMALV2;
// out.precision = type.precision;
// out.scale = type.scale;
break;
case TYPE_NULL:
out.type = FunctionContext::TYPE_NULL;
break;
case TYPE_ARRAY:
out.type = FunctionContext::TYPE_ARRAY;
for (const auto& t : type.children) {
out.children.push_back(column_type_to_type_desc(t));
}
break;
case TYPE_STRING:
out.type = FunctionContext::TYPE_STRING;
out.len = type.len;
break;
case TYPE_JSONB:
out.type = FunctionContext::TYPE_JSONB;
out.len = type.len;
break;
default:
DCHECK(false) << "Unknown type: " << type;
}
return out;
}
} // namespace doris

View File

@ -1,552 +0,0 @@
// 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.
// This file is copied from
// https://github.com/apache/impala/blob/branch-2.9.0/be/src/exprs/anyval-util.h
// and modified by Doris
#pragma once
#include "common/status.h"
#include "runtime/collection_value.h"
#include "runtime/primitive_type.h"
#include "runtime/type_limit.h"
#include "runtime/types.h"
#include "udf/udf.h"
#include "util/hash_util.hpp"
#include "util/types.h"
namespace doris {
using doris_udf::FunctionContext;
using doris_udf::BooleanVal;
using doris_udf::TinyIntVal;
using doris_udf::SmallIntVal;
using doris_udf::IntVal;
using doris_udf::BigIntVal;
using doris_udf::LargeIntVal;
using doris_udf::FloatVal;
using doris_udf::DoubleVal;
using doris_udf::DecimalV2Val;
using doris_udf::DateTimeVal;
using doris_udf::DateTimeV2Val;
using doris_udf::DateV2Val;
using doris_udf::StringVal;
using doris_udf::AnyVal;
class MemPool;
// Utilities for AnyVals
class AnyValUtil {
public:
static uint32_t hash(const doris_udf::BooleanVal& v, int seed) {
return HashUtil::hash(&v.val, 1, seed);
}
static uint32_t hash(const doris_udf::TinyIntVal& v, int seed) {
return HashUtil::hash(&v.val, 1, seed);
}
static uint32_t hash(const doris_udf::SmallIntVal& v, int seed) {
return HashUtil::hash(&v.val, 2, seed);
}
static uint32_t hash(const doris_udf::IntVal& v, int seed) {
return HashUtil::hash(&v.val, 4, seed);
}
static uint32_t hash(const doris_udf::BigIntVal& v, int seed) {
return HashUtil::hash(&v.val, 8, seed);
}
static uint32_t hash(const doris_udf::FloatVal& v, int seed) {
return HashUtil::hash(&v.val, 4, seed);
}
static uint32_t hash(const doris_udf::DoubleVal& v, int seed) {
return HashUtil::hash(&v.val, 8, seed);
}
static uint32_t hash(const doris_udf::StringVal& v, int seed) {
return HashUtil::hash(v.ptr, v.len, seed);
}
static uint32_t hash(const doris_udf::DateTimeVal& v, int seed) {
DateTimeValue tv = DateTimeValue::from_datetime_val(v);
return tv.hash(seed);
}
static uint32_t hash(const doris_udf::DecimalV2Val& v, int seed) {
return HashUtil::hash(&v.val, 16, seed);
}
static uint32_t hash(const doris_udf::LargeIntVal& v, int seed) {
return HashUtil::hash(&v.val, 8, seed);
}
static uint64_t hash64(const doris_udf::BooleanVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 1, seed);
}
static uint64_t hash64(const doris_udf::TinyIntVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 1, seed);
}
static uint64_t hash64(const doris_udf::SmallIntVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 2, seed);
}
static uint64_t hash64(const doris_udf::IntVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 4, seed);
}
static uint64_t hash64(const doris_udf::BigIntVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 8, seed);
}
static uint64_t hash64(const doris_udf::FloatVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 4, seed);
}
static uint64_t hash64(const doris_udf::DoubleVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 8, seed);
}
static uint64_t hash64(const doris_udf::StringVal& v, int64_t seed) {
return HashUtil::fnv_hash64(v.ptr, v.len, seed);
}
static uint64_t hash64(const doris_udf::DateTimeVal& v, int64_t seed) {
DateTimeValue tv = DateTimeValue::from_datetime_val(v);
return HashUtil::fnv_hash64(&tv, 12, seed);
}
static uint64_t hash64(const doris_udf::DecimalV2Val& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 16, seed);
}
static uint64_t hash64(const doris_udf::LargeIntVal& v, int64_t seed) {
return HashUtil::fnv_hash64(&v.val, 8, seed);
}
static uint64_t hash64_murmur(const doris_udf::BooleanVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 1, seed);
}
static uint64_t hash64_murmur(const doris_udf::TinyIntVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 1, seed);
}
static uint64_t hash64_murmur(const doris_udf::SmallIntVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 2, seed);
}
static uint64_t hash64_murmur(const doris_udf::IntVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 4, seed);
}
static uint64_t hash64_murmur(const doris_udf::BigIntVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 8, seed);
}
static uint64_t hash64_murmur(const doris_udf::FloatVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 4, seed);
}
static uint64_t hash64_murmur(const doris_udf::DoubleVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 8, seed);
}
static uint64_t hash64_murmur(const doris_udf::StringVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(v.ptr, v.len, seed);
}
static uint64_t hash64_murmur(const doris_udf::DateTimeVal& v, int64_t seed) {
DateTimeValue tv = DateTimeValue::from_datetime_val(v);
return HashUtil::murmur_hash64A(&tv, 12, seed);
}
static uint64_t hash64_murmur(const doris_udf::DecimalV2Val& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 16, seed);
}
static uint64_t hash64_murmur(const doris_udf::LargeIntVal& v, int64_t seed) {
return HashUtil::murmur_hash64A(&v.val, 8, seed);
}
template <typename Val>
static Val min_val(FunctionContext* ctx) {
if constexpr (std::is_same_v<Val, StringVal>) {
return StringVal();
} else if constexpr (std::is_same_v<Val, DateTimeVal>) {
DateTimeVal val;
type_limit<DateTimeValue>::min().to_datetime_val(&val);
return val;
} else if constexpr (std::is_same_v<Val, DecimalV2Val>) {
DecimalV2Val val;
type_limit<DecimalV2Value>::min().to_decimal_val(&val);
return val;
} else if constexpr (std::is_same_v<Val, DateV2Val>) {
DateV2Val val;
type_limit<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>::min()
.to_datev2_val(&val);
return val;
} else if constexpr (std::is_same_v<Val, DateTimeV2Val>) {
DateTimeV2Val val;
type_limit<
doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>::min()
.to_datetimev2_val(&val);
return val;
} else {
return Val(type_limit<decltype(std::declval<Val>().val)>::min());
}
}
template <typename Val>
static Val max_val(FunctionContext* ctx) {
if constexpr (std::is_same_v<Val, StringVal>) {
StringRef sv = type_limit<StringRef>::max();
StringVal max_val;
max_val.ptr = ctx->allocate(sv.size);
memcpy(max_val.ptr, sv.data, sv.size);
max_val.len = sv.size;
return max_val;
} else if constexpr (std::is_same_v<Val, DateTimeVal>) {
DateTimeVal val;
type_limit<DateTimeValue>::max().to_datetime_val(&val);
return val;
} else if constexpr (std::is_same_v<Val, DecimalV2Val>) {
DecimalV2Val val;
type_limit<DecimalV2Value>::max().to_decimal_val(&val);
return val;
} else if constexpr (std::is_same_v<Val, DateV2Val>) {
DateV2Val val;
type_limit<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>>::max()
.to_datev2_val(&val);
return val;
} else if constexpr (std::is_same_v<Val, DateTimeV2Val>) {
DateTimeV2Val val;
type_limit<
doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>>::max()
.to_datetimev2_val(&val);
return val;
} else {
return Val(type_limit<decltype(std::declval<Val>().val)>::max());
}
}
// Returns the byte size of *Val for type t.
static int any_val_size(const TypeDescriptor& t) {
switch (t.type) {
case TYPE_BOOLEAN:
return sizeof(doris_udf::BooleanVal);
case TYPE_TINYINT:
return sizeof(doris_udf::TinyIntVal);
case TYPE_SMALLINT:
return sizeof(doris_udf::SmallIntVal);
case TYPE_INT:
return sizeof(doris_udf::IntVal);
case TYPE_BIGINT:
return sizeof(doris_udf::BigIntVal);
case TYPE_LARGEINT:
return sizeof(doris_udf::LargeIntVal);
case TYPE_FLOAT:
return sizeof(doris_udf::FloatVal);
case TYPE_DOUBLE:
return sizeof(doris_udf::DoubleVal);
case TYPE_OBJECT:
case TYPE_QUANTILE_STATE:
case TYPE_HLL:
case TYPE_CHAR:
case TYPE_VARCHAR:
case TYPE_STRING:
return sizeof(doris_udf::StringVal);
case TYPE_DATEV2:
return sizeof(doris_udf::DateV2Val);
case TYPE_DATETIMEV2:
return sizeof(doris_udf::DateTimeV2Val);
case TYPE_DATE:
case TYPE_DATETIME:
return sizeof(doris_udf::DateTimeVal);
case TYPE_DECIMALV2:
return sizeof(doris_udf::DecimalV2Val);
case TYPE_ARRAY:
return sizeof(doris_udf::CollectionVal);
default:
DCHECK(false) << t;
return 0;
}
}
/// Returns the byte alignment of *Val for type t.
static int any_val_alignment(const TypeDescriptor& t) {
switch (t.type) {
case TYPE_BOOLEAN:
return alignof(BooleanVal);
case TYPE_TINYINT:
return alignof(TinyIntVal);
case TYPE_SMALLINT:
return alignof(SmallIntVal);
case TYPE_INT:
return alignof(IntVal);
case TYPE_BIGINT:
return alignof(BigIntVal);
case TYPE_LARGEINT:
return alignof(LargeIntVal);
case TYPE_FLOAT:
return alignof(FloatVal);
case TYPE_DOUBLE:
return alignof(DoubleVal);
case TYPE_OBJECT:
case TYPE_QUANTILE_STATE:
case TYPE_HLL:
case TYPE_VARCHAR:
case TYPE_CHAR:
case TYPE_STRING:
return alignof(StringVal);
case TYPE_DATETIME:
case TYPE_DATE:
return alignof(DateTimeVal);
case TYPE_DATEV2:
return alignof(DateV2Val);
case TYPE_DATETIMEV2:
return alignof(DateTimeV2Val);
case TYPE_DECIMALV2:
return alignof(DecimalV2Val);
case TYPE_ARRAY:
return alignof(doris_udf::CollectionVal);
default:
DCHECK(false) << t;
return 0;
}
}
static std::string to_string(const StringVal& v) {
return std::string(reinterpret_cast<char*>(v.ptr), v.len);
}
static StringVal from_string(FunctionContext* ctx, const std::string& s) {
StringVal val = from_buffer(ctx, s.c_str(), s.size());
return val;
}
static void TruncateIfNecessary(const FunctionContext::TypeDesc& type, StringVal* val) {
if (type.type == FunctionContext::TYPE_VARCHAR || type.type == FunctionContext::TYPE_CHAR) {
DCHECK(type.len >= 0);
val->len = std::min(val->len, (int64_t)type.len);
}
}
static StringVal from_buffer(FunctionContext* ctx, const char* ptr, int64_t len) {
StringVal result(ctx, len);
memcpy(result.ptr, ptr, len);
return result;
}
static StringVal from_string_temp(FunctionContext* ctx, const std::string& s) {
StringVal val = from_buffer_temp(ctx, s.c_str(), s.size());
return val;
}
static StringVal from_buffer_temp(FunctionContext* ctx, const char* ptr, int64_t len) {
StringVal result = StringVal::create_temp_string_val(ctx, len);
memcpy(result.ptr, ptr, len);
return result;
}
static FunctionContext::TypeDesc column_type_to_type_desc(const TypeDescriptor& type);
// Utility to put val into an AnyVal struct
static void set_any_val(const void* slot, const TypeDescriptor& type, doris_udf::AnyVal* dst) {
if (slot == nullptr) {
dst->is_null = true;
return;
}
dst->is_null = false;
switch (type.type) {
case TYPE_NULL:
return;
case TYPE_BOOLEAN:
reinterpret_cast<doris_udf::BooleanVal*>(dst)->val =
*reinterpret_cast<const bool*>(slot);
return;
case TYPE_TINYINT:
reinterpret_cast<doris_udf::TinyIntVal*>(dst)->val =
*reinterpret_cast<const int8_t*>(slot);
return;
case TYPE_SMALLINT:
reinterpret_cast<doris_udf::SmallIntVal*>(dst)->val =
*reinterpret_cast<const int16_t*>(slot);
return;
case TYPE_INT:
reinterpret_cast<doris_udf::IntVal*>(dst)->val =
*reinterpret_cast<const int32_t*>(slot);
return;
case TYPE_BIGINT:
reinterpret_cast<doris_udf::BigIntVal*>(dst)->val =
*reinterpret_cast<const int64_t*>(slot);
return;
case TYPE_LARGEINT:
memcpy(&reinterpret_cast<doris_udf::LargeIntVal*>(dst)->val, slot, sizeof(__int128));
return;
case TYPE_FLOAT:
reinterpret_cast<doris_udf::FloatVal*>(dst)->val =
*reinterpret_cast<const float*>(slot);
return;
case TYPE_TIME:
case TYPE_TIMEV2:
case TYPE_DOUBLE:
reinterpret_cast<doris_udf::DoubleVal*>(dst)->val =
*reinterpret_cast<const double*>(slot);
return;
case TYPE_CHAR:
case TYPE_VARCHAR:
case TYPE_HLL:
case TYPE_OBJECT:
case TYPE_QUANTILE_STATE:
case TYPE_STRING:
reinterpret_cast<const StringRef*>(slot)->to_string_val(
reinterpret_cast<StringVal*>(dst));
return;
case TYPE_DECIMALV2:
reinterpret_cast<doris_udf::DecimalV2Val*>(dst)->val =
reinterpret_cast<const PackedInt128*>(slot)->value;
return;
case TYPE_DECIMAL32:
reinterpret_cast<doris_udf::Decimal32Val*>(dst)->val =
*reinterpret_cast<const int32_t*>(slot);
return;
case TYPE_DECIMAL64:
reinterpret_cast<doris_udf::Decimal64Val*>(dst)->val =
*reinterpret_cast<const int64_t*>(slot);
return;
case TYPE_DECIMAL128I:
memcpy(&reinterpret_cast<doris_udf::Decimal128Val*>(dst)->val, slot, sizeof(__int128));
return;
case TYPE_DATE:
reinterpret_cast<const DateTimeValue*>(slot)->to_datetime_val(
reinterpret_cast<doris_udf::DateTimeVal*>(dst));
return;
case TYPE_DATETIME:
reinterpret_cast<const DateTimeValue*>(slot)->to_datetime_val(
reinterpret_cast<doris_udf::DateTimeVal*>(dst));
case TYPE_DATEV2:
reinterpret_cast<
const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>*>(slot)
->to_datev2_val(reinterpret_cast<doris_udf::DateV2Val*>(dst));
return;
case TYPE_DATETIMEV2:
reinterpret_cast<
const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>*>(
slot)
->to_datev2_val(reinterpret_cast<doris_udf::DateV2Val*>(dst));
return;
case TYPE_ARRAY:
reinterpret_cast<const CollectionValue*>(slot)->to_collection_val(
reinterpret_cast<CollectionVal*>(dst));
return;
default:
DCHECK(false) << "NYI";
}
}
/// Templated equality functions. These assume the input values are not nullptr.
template <typename T>
static bool equals(const PrimitiveType& type, const T& x, const T& y) {
return equals_internal(x, y);
}
/// Templated equality functions. These assume the input values are not nullptr.
template <typename T>
static bool equals(const T& x, const T& y) {
return equals_internal(x, y);
}
template <typename T>
static bool equals(const TypeDescriptor& type, const T& x, const T& y) {
return equals_internal(x, y);
}
template <typename T>
static bool equals(const FunctionContext::TypeDesc& type, const T& x, const T& y) {
return equals_internal(x, y);
}
private:
/// Implementations of Equals().
template <typename T>
static bool equals_internal(const T& x, const T& y);
};
template <typename T>
inline bool AnyValUtil::equals_internal(const T& x, const T& y) {
DCHECK(!x.is_null);
DCHECK(!y.is_null);
return x.val == y.val;
}
template <>
inline bool AnyValUtil::equals_internal(const StringVal& x, const StringVal& y) {
DCHECK(!x.is_null);
DCHECK(!y.is_null);
StringRef x_sv = StringRef(x);
StringRef y_sv = StringRef(y);
return x_sv == y_sv;
}
template <>
inline bool AnyValUtil::equals_internal(const DateTimeVal& x, const DateTimeVal& y) {
DCHECK(!x.is_null);
DCHECK(!y.is_null);
DateTimeValue x_tv = DateTimeValue::from_datetime_val(x);
DateTimeValue y_tv = DateTimeValue::from_datetime_val(y);
return x_tv == y_tv;
}
template <>
inline bool AnyValUtil::equals_internal(const DecimalV2Val& x, const DecimalV2Val& y) {
DCHECK(!x.is_null);
DCHECK(!y.is_null);
return x == y;
}
// Creates the corresponding AnyVal subclass for type. The object is added to the pool.
doris_udf::AnyVal* create_any_val(ObjectPool* pool, const TypeDescriptor& type);
/// Allocates an AnyVal subclass of 'type' from 'pool'. The AnyVal's memory is
/// initialized to all 0's. Returns a MemLimitExceeded() error with message
/// 'mem_limit_exceeded_msg' if the allocation cannot be made because of a memory
/// limit.
Status allocate_any_val(RuntimeState* state, MemPool* pool, const TypeDescriptor& type,
const std::string& mem_limit_exceeded_msg, AnyVal** result);
} // namespace doris

View File

@ -31,10 +31,9 @@
#include <string_view>
#include <vector>
#include "common/compiler_util.h"
#include "common/logging.h"
#include "exprs/anyval_util.h"
#include "gutil/strings/stringpiece.h"
#include "rapidjson/error/en.h"
#include "udf/udf.h"
#include "util/string_util.h"

View File

@ -88,6 +88,8 @@ struct JsonPath {
}
};
using namespace doris_udf;
class JsonFunctions {
public:
/**

View File

@ -1,438 +0,0 @@
// 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.
// This file is copied from
// https://github.com/apache/impala/blob/branch-2.9.0/be/src/exprs/like-predicate.cc
// and modified by Doris
#include "exprs/like_predicate.h"
#include <string.h>
#include <sstream>
#include "exprs/string_functions.h"
namespace doris {
// A regex to match any regex pattern is equivalent to a substring search.
static const RE2 SUBSTRING_RE(
"(?:\\.\\*)*([^\\.\\^\\{\\[\\(\\|\\)\\]\\}\\+\\*\\?\\$\\\\]*)(?:\\.\\*)*");
// A regex to match any regex pattern which is equivalent to matching a constant string
// at the end of the string values.
static const RE2 ENDS_WITH_RE("(?:\\.\\*)*([^\\.\\^\\{\\[\\(\\|\\)\\]\\}\\+\\*\\?\\$\\\\]*)\\$");
// A regex to match any regex pattern which is equivalent to matching a constant string
// at the end of the string values.
static const RE2 STARTS_WITH_RE("\\^([^\\.\\^\\{\\[\\(\\|\\)\\]\\}\\+\\*\\?\\$\\\\]*)(?:\\.\\*)*");
// A regex to match any regex pattern which is equivalent to a constant string match.
static const RE2 EQUALS_RE("\\^([^\\.\\^\\{\\[\\(\\|\\)\\]\\}\\+\\*\\?\\$\\\\]*)\\$");
static const re2::RE2 LIKE_SUBSTRING_RE("(?:%+)(((\\\\%)|(\\\\_)|([^%_]))+)(?:%+)");
static const re2::RE2 LIKE_ENDS_WITH_RE("(?:%+)(((\\\\%)|(\\\\_)|([^%_]))+)");
static const re2::RE2 LIKE_STARTS_WITH_RE("(((\\\\%)|(\\\\_)|([^%_]))+)(?:%+)");
static const re2::RE2 LIKE_EQUALS_RE("(((\\\\%)|(\\\\_)|([^%_]))+)");
void LikePredicate::init() {}
void LikePredicate::like_prepare(FunctionContext* context,
FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::THREAD_LOCAL) {
return;
}
LikePredicateState* state = new LikePredicateState();
state->function = like_fn;
context->set_function_state(scope, state);
if (context->is_arg_constant(1)) {
StringVal pattern_val = *reinterpret_cast<StringVal*>(context->get_constant_arg(1));
if (pattern_val.is_null) {
return;
}
StringRef pattern = StringRef(pattern_val);
std::string pattern_str(pattern.data, pattern.size);
std::string search_string;
if (RE2::FullMatch(pattern_str, LIKE_ENDS_WITH_RE, &search_string)) {
remove_escape_character(&search_string);
state->set_search_string(search_string);
state->function = constant_ends_with_fn;
} else if (RE2::FullMatch(pattern_str, LIKE_SUBSTRING_RE, &search_string)) {
remove_escape_character(&search_string);
state->set_search_string(search_string);
state->function = constant_substring_fn;
} else if (RE2::FullMatch(pattern_str, LIKE_EQUALS_RE, &search_string)) {
remove_escape_character(&search_string);
state->set_search_string(search_string);
state->function = constant_equals_fn;
} else if (RE2::FullMatch(pattern_str, LIKE_STARTS_WITH_RE, &search_string)) {
remove_escape_character(&search_string);
state->set_search_string(search_string);
state->function = constant_starts_with_fn;
} else {
std::string re_pattern;
convert_like_pattern(context,
*reinterpret_cast<StringVal*>(context->get_constant_arg(1)),
&re_pattern);
RE2::Options opts;
opts.set_never_nl(false);
opts.set_dot_nl(true);
state->regex.reset(new RE2(re_pattern, opts));
if (!state->regex->ok()) {
context->set_error("Invalid regex: $0");
}
}
}
}
BooleanVal LikePredicate::like(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
return (state->function)(context, val, pattern);
}
void LikePredicate::like_close(FunctionContext* context,
FunctionContext::FunctionStateScope scope) {
if (scope == FunctionContext::THREAD_LOCAL) {
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
delete state;
}
}
void LikePredicate::regex_prepare(FunctionContext* context,
FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::THREAD_LOCAL) {
return;
}
LikePredicateState* state = new LikePredicateState();
context->set_function_state(scope, state);
state->function = regex_fn;
if (context->is_arg_constant(1)) {
StringVal* pattern = reinterpret_cast<StringVal*>(context->get_constant_arg(1));
if (pattern->is_null) {
return;
}
std::string pattern_str(reinterpret_cast<const char*>(pattern->ptr), pattern->len);
std::string search_string;
// The following four conditionals check if the pattern is a constant string,
// starts with a constant string and is followed by any number of wildcard characters,
// ends with a constant string and is preceded by any number of wildcard characters or
// has a constant substring surrounded on both sides by any number of wildcard
// characters. In any of these conditions, we can search for the pattern more
// efficiently by using our own string match functions rather than regex matching.
if (RE2::FullMatch(pattern_str, EQUALS_RE, &search_string)) {
state->set_search_string(search_string);
state->function = constant_equals_fn;
} else if (RE2::FullMatch(pattern_str, STARTS_WITH_RE, &search_string)) {
state->set_search_string(search_string);
state->function = constant_starts_with_fn;
} else if (RE2::FullMatch(pattern_str, ENDS_WITH_RE, &search_string)) {
state->set_search_string(search_string);
state->function = constant_ends_with_fn;
} else if (RE2::FullMatch(pattern_str, SUBSTRING_RE, &search_string)) {
state->set_search_string(search_string);
state->function = constant_substring_fn;
} else {
RE2::Options opts;
opts.set_never_nl(false);
opts.set_dot_nl(true);
state->regex.reset(new RE2(pattern_str, opts));
if (!state->regex->ok()) {
std::stringstream error;
error << "Invalid regex expression" << pattern->ptr;
context->set_error(error.str().c_str());
}
state->function = constant_regex_fn_partial;
}
}
}
BooleanVal LikePredicate::regex(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
return (state->function)(context, val, pattern);
}
// This prepare function is used only when 3 parameters are passed to the regexp_like()
// function. For the 2 parameter version, the RegexPrepare() function is used to prepare.
void LikePredicate::regexp_like_prepare(FunctionContext* context,
FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::THREAD_LOCAL) {
return;
}
LikePredicateState* state = new LikePredicateState();
context->set_function_state(scope, state);
// If both the pattern and the match parameter are constant, we pre-compile the
// regular expression once here. Otherwise, the RE is compiled per row in RegexpLike()
if (context->is_arg_constant(1) && context->is_arg_constant(2)) {
StringVal* pattern = nullptr;
pattern = reinterpret_cast<StringVal*>(context->get_constant_arg(1));
if (pattern->is_null) {
return;
}
StringVal* match_parameter = reinterpret_cast<StringVal*>(context->get_constant_arg(2));
std::stringstream error;
if (match_parameter->is_null) {
error << "match parameter is null";
context->set_error(error.str().c_str());
return;
}
RE2::Options opts;
opts.set_never_nl(false);
opts.set_dot_nl(true);
std::string error_str;
if (!StringFunctions::set_re2_options(*match_parameter, &error_str, &opts)) {
context->set_error(error_str.c_str());
return;
}
std::string pattern_str(reinterpret_cast<const char*>(pattern->ptr), pattern->len);
state->regex.reset(new RE2(pattern_str, opts));
if (!state->regex->ok()) {
error << "Invalid regex expression" << pattern->ptr;
context->set_error(error.str().c_str());
}
}
}
// This is used only for the 3 parameter version of regexp_like(). The 2 parameter
// version calls Regex() directly.
BooleanVal LikePredicate::regexp_like(FunctionContext* context, const StringVal& val,
const StringVal& pattern, const StringVal& match_parameter) {
if (val.is_null || pattern.is_null) {
return BooleanVal::null();
}
// If either the pattern or the third optional match parameter are not constant, we
// have to recompile the RE for every row.
if (!context->is_arg_constant(2) || !context->is_arg_constant(1)) {
if (match_parameter.is_null) {
return BooleanVal::null();
}
RE2::Options opts;
std::string error_str;
if (!StringFunctions::set_re2_options(match_parameter, &error_str, &opts)) {
context->set_error(error_str.c_str());
return BooleanVal(false);
}
std::string re_pattern(reinterpret_cast<const char*>(pattern.ptr), pattern.len);
re2::RE2 re(re_pattern, opts);
if (re.ok()) {
return RE2::PartialMatch(
re2::StringPiece(reinterpret_cast<const char*>(val.ptr), val.len), re);
} else {
context->set_error("Invalid regex: $0");
return BooleanVal(false);
}
}
return constant_regex_fn_partial(context, val, pattern);
}
void LikePredicate::regex_close(FunctionContext* context,
FunctionContext::FunctionStateScope scope) {
if (scope == FunctionContext::THREAD_LOCAL) {
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
delete state;
}
}
BooleanVal LikePredicate::regex_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
return regex_match(context, val, pattern, false);
}
BooleanVal LikePredicate::like_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
return regex_match(context, val, pattern, true);
}
BooleanVal LikePredicate::constant_substring_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
if (val.is_null) {
return BooleanVal::null();
}
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
if (state->search_string_sv.size == 0) {
return BooleanVal(true);
}
StringRef pattern_value = StringRef(val);
return BooleanVal(state->substring_pattern.search(&pattern_value) != -1);
}
BooleanVal LikePredicate::constant_starts_with_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
if (val.is_null) {
return BooleanVal::null();
}
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
if (val.len < state->search_string_sv.size) {
return BooleanVal(false);
} else {
StringRef v = StringRef(reinterpret_cast<char*>(val.ptr), state->search_string_sv.size);
return BooleanVal(state->search_string_sv.eq((v)));
}
}
BooleanVal LikePredicate::constant_ends_with_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
if (val.is_null) {
return BooleanVal::null();
}
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
if (val.len < state->search_string_sv.size) {
return BooleanVal(false);
} else {
char* ptr = reinterpret_cast<char*>(val.ptr) + val.len - state->search_string_sv.size;
int len = state->search_string_sv.size;
StringRef v = StringRef(ptr, len);
return BooleanVal(state->search_string_sv.eq(v));
}
}
BooleanVal LikePredicate::constant_equals_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
if (val.is_null) {
return BooleanVal::null();
}
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
return BooleanVal(state->search_string_sv.eq(StringRef(val)));
}
BooleanVal LikePredicate::constant_regex_fn_partial(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
if (val.is_null) {
return BooleanVal::null();
}
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
re2::StringPiece operand_sp(reinterpret_cast<const char*>(val.ptr), val.len);
return RE2::PartialMatch(operand_sp, *state->regex);
}
BooleanVal LikePredicate::constant_regex_fn(FunctionContext* context, const StringVal& val,
const StringVal& pattern) {
if (val.is_null) {
return BooleanVal::null();
}
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
re2::StringPiece operand_sp(reinterpret_cast<const char*>(val.ptr), val.len);
return RE2::FullMatch(operand_sp, *state->regex);
}
BooleanVal LikePredicate::regex_match(FunctionContext* context, const StringVal& operand_value,
const StringVal& pattern_value, bool is_like_pattern) {
if (operand_value.is_null || pattern_value.is_null) {
return BooleanVal::null();
}
if (context->is_arg_constant(1)) {
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
if (is_like_pattern) {
return RE2::FullMatch(re2::StringPiece(reinterpret_cast<const char*>(operand_value.ptr),
operand_value.len),
*state->regex.get());
} else {
return RE2::PartialMatch(
re2::StringPiece(reinterpret_cast<const char*>(operand_value.ptr),
operand_value.len),
*state->regex.get());
}
} else {
std::string re_pattern;
RE2::Options opts;
opts.set_never_nl(false);
opts.set_dot_nl(true);
if (is_like_pattern) {
convert_like_pattern(context, pattern_value, &re_pattern);
} else {
re_pattern = std::string(reinterpret_cast<const char*>(pattern_value.ptr),
pattern_value.len);
}
re2::RE2 re(re_pattern, opts);
if (re.ok()) {
if (is_like_pattern) {
return RE2::FullMatch(
re2::StringPiece(reinterpret_cast<const char*>(operand_value.ptr),
operand_value.len),
re);
} else {
return RE2::PartialMatch(
re2::StringPiece(reinterpret_cast<const char*>(operand_value.ptr),
operand_value.len),
re);
}
} else {
context->set_error("Invalid regex: $0");
return BooleanVal(false);
}
}
}
void LikePredicate::convert_like_pattern(FunctionContext* context, const StringVal& pattern,
std::string* re_pattern) {
re_pattern->clear();
LikePredicateState* state = reinterpret_cast<LikePredicateState*>(
context->get_function_state(FunctionContext::THREAD_LOCAL));
bool is_escaped = false;
for (int i = 0; i < pattern.len; ++i) {
if (!is_escaped && pattern.ptr[i] == '%') {
re_pattern->append(".*");
} else if (!is_escaped && pattern.ptr[i] == '_') {
re_pattern->append(".");
// check for escape char before checking for regex special chars, they might overlap
} else if (!is_escaped && pattern.ptr[i] == state->escape_char) {
is_escaped = true;
} else if (pattern.ptr[i] == '.' || pattern.ptr[i] == '[' || pattern.ptr[i] == ']' ||
pattern.ptr[i] == '{' || pattern.ptr[i] == '}' || pattern.ptr[i] == '(' ||
pattern.ptr[i] == ')' || pattern.ptr[i] == '\\' || pattern.ptr[i] == '*' ||
pattern.ptr[i] == '+' || pattern.ptr[i] == '?' || pattern.ptr[i] == '|' ||
pattern.ptr[i] == '^' || pattern.ptr[i] == '$') {
// escape all regex special characters; see list at
re_pattern->append("\\");
re_pattern->append(1, pattern.ptr[i]);
is_escaped = false;
} else {
// regular character or escaped special character
re_pattern->append(1, pattern.ptr[i]);
is_escaped = false;
}
}
}
void LikePredicate::remove_escape_character(std::string* search_string) {
std::string tmp_search_string;
tmp_search_string.swap(*search_string);
int len = tmp_search_string.length();
for (int i = 0; i < len;) {
if (tmp_search_string[i] == '\\' && i + 1 < len &&
(tmp_search_string[i + 1] == '%' || tmp_search_string[i + 1] == '_')) {
search_string->append(1, tmp_search_string[i + 1]);
i += 2;
} else {
search_string->append(1, tmp_search_string[i]);
i++;
}
}
}
} // namespace doris

View File

@ -1,160 +0,0 @@
// 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.
// This file is copied from
// https://github.com/apache/impala/blob/branch-2.9.0/be/src/exprs/like-predicate.h
// and modified by Doris
#pragma once
#include <re2/re2.h>
#include <memory>
#include <string>
#include "runtime/string_search.hpp"
namespace doris {
typedef doris_udf::BooleanVal (*LikePredicateFunction)(doris_udf::FunctionContext*,
const doris_udf::StringVal&,
const doris_udf::StringVal&);
struct LikePredicateState {
char escape_char;
/// This is the function, set in the prepare function, that will be used to determine
/// the value of the predicate. It will be set depending on whether the expression is
/// a LIKE, RLIKE or REGEXP predicate, whether the pattern is a constant argument
/// and whether the pattern has any constant substrings. If the pattern is not a
/// constant argument, none of the following fields can be set because we cannot know
/// the format of the pattern in the prepare function and must deal with each pattern
/// separately.
LikePredicateFunction function;
/// Holds the string the StringRef points to and is set any time StringRef is
/// used.
std::string search_string;
/// Used for LIKE predicates if the pattern is a constant argument, and is either a
/// constant string or has a constant string at the beginning or end of the pattern.
/// This will be set in order to check for that pattern in the corresponding part of
/// the string.
StringRef search_string_sv;
/// Used for LIKE predicates if the pattern is a constant argument and has a constant
/// string in the middle of it. This will be use in order to check for the substring
/// in the value.
StringSearch substring_pattern;
/// Used for RLIKE and REGEXP predicates if the pattern is a constant argument.
std::unique_ptr<re2::RE2> regex;
LikePredicateState() : escape_char('\\') {}
void set_search_string(const std::string& search_string_arg) {
search_string = search_string_arg;
search_string_sv = StringRef(search_string);
substring_pattern.set_pattern(&search_string_sv);
}
};
class LikePredicate {
public:
static void init();
private:
friend class OpcodeRegistry;
static void like_prepare(doris_udf::FunctionContext* context,
doris_udf::FunctionContext::FunctionStateScope scope);
static doris_udf::BooleanVal like(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
static void like_close(doris_udf::FunctionContext* context,
doris_udf::FunctionContext::FunctionStateScope scope);
static void regex_prepare(doris_udf::FunctionContext* context,
doris_udf::FunctionContext::FunctionStateScope scope);
static doris_udf::BooleanVal regex(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
/// Prepare function for regexp_like() when a third optional parameter is used
static void regexp_like_prepare(doris_udf::FunctionContext* context,
doris_udf::FunctionContext::FunctionStateScope scope);
/// Handles regexp_like() when 3 parameters are passed to it
static doris_udf::BooleanVal regexp_like(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern,
const doris_udf::StringVal& match_parameter);
static void regex_close(doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope scope);
static doris_udf::BooleanVal regex_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
static doris_udf::BooleanVal like_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
/// Handling of like predicates that map to strstr
static doris_udf::BooleanVal constant_substring_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
/// Handling of like predicates that can be implemented using strncmp
static doris_udf::BooleanVal constant_starts_with_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
/// Handling of like predicates that can be implemented using strncmp
static doris_udf::BooleanVal constant_ends_with_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
/// Handling of like predicates that can be implemented using strcmp
static doris_udf::BooleanVal constant_equals_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
static doris_udf::BooleanVal constant_regex_fn_partial(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
static doris_udf::BooleanVal constant_regex_fn(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern);
static doris_udf::BooleanVal regex_match(doris_udf::FunctionContext* context,
const doris_udf::StringVal& val,
const doris_udf::StringVal& pattern,
bool is_like_pattern);
/// Convert a LIKE pattern (with embedded % and _) into the corresponding
/// regular expression pattern. Escaped chars are copied verbatim.
static void convert_like_pattern(doris_udf::FunctionContext* context,
const doris_udf::StringVal& pattern, std::string* re_pattern);
static void remove_escape_character(std::string* search_string);
};
} // namespace doris

View File

@ -28,7 +28,6 @@
#include <sstream>
#include "common/compiler_util.h"
#include "exprs/anyval_util.h"
#include "runtime/decimalv2_value.h"
#include "runtime/large_int_value.h"
#include "util/simd/vstring_function.h"
@ -127,7 +126,9 @@ StringVal MathFunctions::decimal_to_base(FunctionContext* ctx, int64_t src_num,
buf[buf_index] = '-';
++result_len;
}
return AnyValUtil::from_buffer_temp(ctx, buf + max_digits - result_len, result_len);
StringVal result = StringVal::create_temp_string_val(ctx, result_len);
memcpy(result.ptr, buf + max_digits - result_len, result_len);
return result;
}
bool MathFunctions::decimal_in_base_to_decimal(int64_t src_num, int8_t src_base, int64_t* result) {

View File

@ -24,7 +24,6 @@
#include <algorithm>
#include "exprs/anyval_util.h"
#include "math_functions.h"
#include "util/simd/vstring_function.h"
#include "util/url_parser.h"
@ -80,7 +79,8 @@ re2::RE2* StringFunctions::compile_regex(const StringVal& pattern, std::string*
re2::RE2* re = new re2::RE2(pattern_sp, options);
if (!re->ok()) {
std::stringstream ss;
ss << "Could not compile regexp pattern: " << AnyValUtil::to_string(pattern) << std::endl
ss << "Could not compile regexp pattern: "
<< std::string(reinterpret_cast<char*>(pattern.ptr), pattern.len) << std::endl
<< "Error: " << re->error();
*error_str = ss.str();
delete re;

View File

@ -23,7 +23,6 @@ set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/geo")
add_library(Geo STATIC
geo_common.cpp
geo_functions.cpp
geo_types.cpp
wkt_parse.cpp
${GENSRC_DIR}/geo/wkt_lex.l.cpp

View File

@ -1,287 +0,0 @@
// 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 "geo/geo_functions.h"
#include <s2/s2debug.h>
#include <s2/s2earth.h>
#include "geo/geo_types.h"
namespace doris {
void GeoFunctions::init() {
// set s2debug to false to avoid crash
FLAGS_s2debug = false;
}
DoubleVal GeoFunctions::st_distance_sphere(FunctionContext* ctx, const DoubleVal& x_lng,
const DoubleVal& x_lat, const DoubleVal& y_lng,
const DoubleVal& y_lat) {
if (x_lng.is_null || x_lat.is_null || y_lng.is_null || y_lat.is_null) {
return DoubleVal::null();
}
S2LatLng x = S2LatLng::FromDegrees(x_lat.val, x_lng.val);
if (!x.is_valid()) {
return DoubleVal::null();
}
S2LatLng y = S2LatLng::FromDegrees(y_lat.val, y_lng.val);
if (!y.is_valid()) {
return DoubleVal::null();
}
return DoubleVal(S2Earth::ToMeters(x.GetDistance(y)));
}
doris_udf::StringVal GeoFunctions::st_point(doris_udf::FunctionContext* ctx,
const doris_udf::DoubleVal& x,
const doris_udf::DoubleVal& y) {
if (x.is_null || y.is_null) {
return StringVal::null();
}
GeoPoint point;
auto res = point.from_coord(x.val, y.val);
if (res != GEO_PARSE_OK) {
return StringVal::null();
}
std::string buf;
point.encode_to(&buf);
StringVal result(ctx, buf.size());
memcpy(result.ptr, buf.data(), buf.size());
return result;
}
DoubleVal GeoFunctions::st_x(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& point_encoded) {
if (point_encoded.is_null) {
return DoubleVal::null();
}
GeoPoint point;
auto res = point.decode_from(point_encoded.ptr, point_encoded.len);
if (!res) {
return DoubleVal::null();
}
return DoubleVal(point.x());
}
DoubleVal GeoFunctions::st_y(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& point_encoded) {
if (point_encoded.is_null) {
return DoubleVal::null();
}
GeoPoint point;
auto res = point.decode_from(point_encoded.ptr, point_encoded.len);
if (!res) {
return DoubleVal::null();
}
return DoubleVal(point.y());
}
StringVal GeoFunctions::st_as_wkt(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& shape_encoded) {
if (shape_encoded.is_null) {
return StringVal::null();
}
std::unique_ptr<GeoShape> shape(GeoShape::from_encoded(shape_encoded.ptr, shape_encoded.len));
if (shape == nullptr) {
return StringVal::null();
}
auto wkt = shape->as_wkt();
StringVal result(ctx, wkt.size());
memcpy(result.ptr, wkt.data(), wkt.size());
return result;
}
void GeoFunctions::st_from_wkt_close(FunctionContext* ctx,
FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::FRAGMENT_LOCAL) {
return;
}
StConstructState* state = reinterpret_cast<StConstructState*>(ctx->get_function_state(scope));
delete state;
}
void GeoFunctions::st_from_wkt_prepare_common(FunctionContext* ctx,
FunctionContext::FunctionStateScope scope,
GeoShapeType shape_type) {
if (scope != FunctionContext::FRAGMENT_LOCAL) {
return;
}
if (!ctx->is_arg_constant(0)) {
return;
}
std::unique_ptr<StConstructState> state(new StConstructState());
StringVal* str = reinterpret_cast<StringVal*>(ctx->get_constant_arg(0));
if (str->is_null) {
str->is_null = true;
} else {
GeoParseStatus status;
std::unique_ptr<GeoShape> shape(
GeoShape::from_wkt((const char*)str->ptr, str->len, &status));
if (shape == nullptr || (shape_type != GEO_SHAPE_ANY && shape->type() != shape_type)) {
state->is_null = true;
} else {
shape->encode_to(&state->encoded_buf);
}
}
ctx->set_function_state(scope, state.release());
}
StringVal GeoFunctions::st_from_wkt_common(FunctionContext* ctx, const StringVal& wkt,
GeoShapeType shape_type) {
if (wkt.is_null) {
return StringVal::null();
}
StConstructState* state =
(StConstructState*)ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL);
if (state == nullptr) {
GeoParseStatus status;
std::unique_ptr<GeoShape> shape(GeoShape::from_wkt((const char*)wkt.ptr, wkt.len, &status));
if (shape == nullptr || (shape_type != GEO_SHAPE_ANY && shape->type() != shape_type)) {
return StringVal::null();
}
std::string buf;
shape->encode_to(&buf);
StringVal result(ctx, buf.size());
memcpy(result.ptr, buf.data(), buf.size());
return result;
} else {
if (state->is_null) {
return StringVal::null();
}
StringVal result((uint8_t*)state->encoded_buf.data(), state->encoded_buf.size());
return result;
}
}
void GeoFunctions::st_circle_prepare(doris_udf::FunctionContext* ctx,
doris_udf::FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::FRAGMENT_LOCAL) {
return;
}
if (!ctx->is_arg_constant(0) || !ctx->is_arg_constant(1) || !ctx->is_arg_constant(2)) {
return;
}
std::unique_ptr<StConstructState> state(new StConstructState());
DoubleVal* lng = reinterpret_cast<DoubleVal*>(ctx->get_constant_arg(0));
DoubleVal* lat = reinterpret_cast<DoubleVal*>(ctx->get_constant_arg(1));
DoubleVal* radius = reinterpret_cast<DoubleVal*>(ctx->get_constant_arg(2));
if (lng->is_null || lat->is_null || radius->is_null) {
state->is_null = true;
} else {
std::unique_ptr<GeoCircle> circle(new GeoCircle());
auto res = circle->init(lng->val, lat->val, radius->val);
if (res != GEO_PARSE_OK) {
state->is_null = true;
} else {
circle->encode_to(&state->encoded_buf);
}
}
ctx->set_function_state(scope, state.release());
}
doris_udf::StringVal GeoFunctions::st_circle(FunctionContext* ctx, const DoubleVal& lng,
const DoubleVal& lat, const DoubleVal& radius) {
if (lng.is_null || lat.is_null || radius.is_null) {
return StringVal::null();
}
StConstructState* state =
(StConstructState*)ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL);
if (state == nullptr) {
std::unique_ptr<GeoCircle> circle(new GeoCircle());
auto res = circle->init(lng.val, lat.val, radius.val);
if (res != GEO_PARSE_OK) {
return StringVal::null();
}
std::string buf;
circle->encode_to(&buf);
StringVal result(ctx, buf.size());
memcpy(result.ptr, buf.data(), buf.size());
return result;
} else {
if (state->is_null) {
return StringVal::null();
}
StringVal result((uint8_t*)state->encoded_buf.data(), state->encoded_buf.size());
return result;
}
}
void GeoFunctions::st_contains_prepare(doris_udf::FunctionContext* ctx,
doris_udf::FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::FRAGMENT_LOCAL) {
return;
}
if (!ctx->is_arg_constant(0) && !ctx->is_arg_constant(1)) {
return;
}
std::unique_ptr<StContainsState> contains_ctx(new StContainsState());
for (int i = 0; !contains_ctx->is_null && i < 2; ++i) {
if (ctx->is_arg_constant(i)) {
StringVal* str = reinterpret_cast<StringVal*>(ctx->get_constant_arg(i));
if (str->is_null) {
contains_ctx->is_null = true;
} else {
contains_ctx->shapes[i] =
std::shared_ptr<GeoShape>(GeoShape::from_encoded(str->ptr, str->len));
if (contains_ctx->shapes[i] == nullptr) {
contains_ctx->is_null = true;
}
}
}
}
ctx->set_function_state(scope, contains_ctx.release());
}
void GeoFunctions::st_contains_close(doris_udf::FunctionContext* ctx,
doris_udf::FunctionContext::FunctionStateScope scope) {
if (scope != FunctionContext::FRAGMENT_LOCAL) {
return;
}
StContainsState* contains_ctx =
reinterpret_cast<StContainsState*>(ctx->get_function_state(scope));
delete contains_ctx;
}
doris_udf::BooleanVal GeoFunctions::st_contains(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& lhs,
const doris_udf::StringVal& rhs) {
if (lhs.is_null || rhs.is_null) {
return BooleanVal::null();
}
const StContainsState* state = reinterpret_cast<StContainsState*>(
ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
if (state != nullptr && state->is_null) {
return BooleanVal::null();
}
std::vector<std::shared_ptr<GeoShape>> shapes = {nullptr, nullptr};
const StringVal* strs[2] = {&lhs, &rhs};
for (int i = 0; i < 2; ++i) {
if (state != nullptr && state->shapes[i] != nullptr) {
shapes[i] = state->shapes[i];
} else {
shapes[i] =
std::shared_ptr<GeoShape>(GeoShape::from_encoded(strs[i]->ptr, strs[i]->len));
if (shapes[i] == nullptr) {
return BooleanVal::null();
}
}
}
return shapes[0]->contains(shapes[1].get());
}
} // namespace doris

View File

@ -1,126 +0,0 @@
// 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.
#pragma once
#include "geo/geo_common.h"
#include "geo/geo_types.h"
#include "udf/udf.h"
namespace doris {
class GeoFunctions {
public:
static void init();
// compute distance between two points in earth sphere
static DoubleVal st_distance_sphere(FunctionContext* ctx, const DoubleVal& x_lng,
const DoubleVal& x_lat, const DoubleVal& y_lng,
const DoubleVal& y_lat);
// point
static doris_udf::StringVal st_point(doris_udf::FunctionContext* ctx,
const doris_udf::DoubleVal& x,
const doris_udf::DoubleVal& y);
static doris_udf::DoubleVal st_x(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& point);
static doris_udf::DoubleVal st_y(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& point);
// to wkt
static doris_udf::StringVal st_as_wkt(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& shape);
// from wkt
static void st_from_wkt_prepare_common(doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope,
GeoShapeType shape_type);
static void st_from_wkt_close(doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
static doris_udf::StringVal st_from_wkt_common(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& wkt,
GeoShapeType shape_type);
static void st_from_wkt_prepare(doris_udf::FunctionContext* ctx,
doris_udf::FunctionContext::FunctionStateScope scope)
__attribute__((used)) {
st_from_wkt_prepare_common(ctx, scope, GEO_SHAPE_ANY);
}
static doris_udf::StringVal st_from_wkt(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& wkt) __attribute__((used)) {
return st_from_wkt_common(ctx, wkt, GEO_SHAPE_ANY);
}
// for line
static void st_line_prepare(doris_udf::FunctionContext* ctx,
doris_udf::FunctionContext::FunctionStateScope scope)
__attribute__((used)) {
st_from_wkt_prepare_common(ctx, scope, GEO_SHAPE_LINE_STRING);
}
static doris_udf::StringVal st_line(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& wkt) __attribute__((used)) {
return st_from_wkt_common(ctx, wkt, GEO_SHAPE_LINE_STRING);
}
// for polygon
static void st_polygon_prepare(doris_udf::FunctionContext* ctx,
doris_udf::FunctionContext::FunctionStateScope scope)
__attribute__((used)) {
st_from_wkt_prepare_common(ctx, scope, GEO_SHAPE_POLYGON);
}
static doris_udf::StringVal st_polygon(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& wkt) __attribute__((used)) {
return st_from_wkt_common(ctx, wkt, GEO_SHAPE_POLYGON);
}
// for circle
static doris_udf::StringVal st_circle(doris_udf::FunctionContext* ctx,
const doris_udf::DoubleVal& center_lng,
const doris_udf::DoubleVal& center_lat,
const doris_udf::DoubleVal& radius_meter);
static void st_circle_prepare(doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
// Returns true if and only if no points of the second geometry
// lie in the exterior of the first geometry, and at least one
// point of the interior of the first geometry lies in the
// interior of the second geometry.
static doris_udf::BooleanVal st_contains(doris_udf::FunctionContext* ctx,
const doris_udf::StringVal& lhs,
const doris_udf::StringVal& rhs);
static void st_contains_prepare(doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
static void st_contains_close(doris_udf::FunctionContext*,
doris_udf::FunctionContext::FunctionStateScope);
};
struct StConstructState {
StConstructState() : is_null(false) {}
~StConstructState() {}
bool is_null;
std::string encoded_buf;
};
struct StContainsState {
StContainsState() : is_null(false), shapes {nullptr, nullptr} {}
~StContainsState() {}
bool is_null;
std::vector<std::shared_ptr<GeoShape>> shapes;
};
} // namespace doris

View File

@ -60,7 +60,6 @@ add_library(Olap STATIC
data_dir.cpp
short_key_index.cpp
snapshot_manager.cpp
stream_name.cpp
tablet.cpp
tablet_manager.cpp
tablet_meta.cpp

View File

@ -16,7 +16,6 @@
// under the License.
#pragma once
#include "exprs/like_predicate.h"
#include "olap/column_predicate.h"
#include "udf/udf.h"
#include "vec/columns/column_dictionary.h"
@ -146,7 +145,7 @@ private:
// lifetime controlled by scan node
doris_udf::FunctionContext* _fn_ctx;
using PatternType = std::conditional_t<is_vectorized, StringRef, StringVal>;
using StateType = std::conditional_t<is_vectorized, vectorized::LikeState, LikePredicateState>;
using StateType = vectorized::LikeState;
PatternType pattern;
StateType* _state;

View File

@ -618,27 +618,6 @@ Status BetaRowsetWriter::_add_block_for_segcompaction(
return Status::OK();
}
template <typename RowType>
Status BetaRowsetWriter::_add_row(const RowType& row) {
if (PREDICT_FALSE(_segment_writer == nullptr)) {
RETURN_NOT_OK(_create_segment_writer(&_segment_writer));
}
// TODO update rowset zonemap
auto s = _segment_writer->append_row(row);
if (PREDICT_FALSE(!s.ok())) {
LOG(WARNING) << "failed to append row: " << s.to_string();
return Status::Error<WRITER_DATA_WRITE_ERROR>();
}
if (PREDICT_FALSE(_segment_writer->estimate_segment_size() >= MAX_SEGMENT_SIZE ||
_segment_writer->num_rows_written() >= _context.max_rows_per_segment)) {
RETURN_NOT_OK(_flush_segment_writer(&_segment_writer));
}
++_raw_num_rows_written;
return Status::OK();
}
template Status BetaRowsetWriter::_add_row(const RowCursor& row);
Status BetaRowsetWriter::add_rowset(RowsetSharedPtr rowset) {
assert(rowset->rowset_meta()->rowset_type() == BETA_ROWSET);
RETURN_NOT_OK(rowset->link_files_to(_context.rowset_dir, _context.rowset_id));

View File

@ -41,8 +41,6 @@ public:
Status init(const RowsetWriterContext& rowset_writer_context) override;
Status add_row(const RowCursor& row) override { return _add_row(row); }
Status add_block(const vectorized::Block* block) override;
// add rowset by create hard link
@ -83,8 +81,6 @@ public:
int32_t get_atomic_num_segment() const override { return _num_segment.load(); }
private:
template <typename RowType>
Status _add_row(const RowType& row);
Status _add_block(const vectorized::Block* block,
std::unique_ptr<segment_v2::SegmentWriter>* writer);
Status _add_block_for_segcompaction(const vectorized::Block* block,

View File

@ -36,10 +36,6 @@ public:
virtual Status init(const RowsetWriterContext& rowset_writer_context) = 0;
// Memory note: input `row` is guaranteed to be copied into writer's internal buffer, including all slice data
// referenced by `row`. That means callers are free to de-allocate memory for `row` after this method returns.
virtual Status add_row(const RowCursor& row) = 0;
virtual Status add_block(const vectorized::Block* block) {
return Status::Error<ErrorCode::NOT_IMPLEMENTED_ERROR>();
}

View File

@ -1,46 +0,0 @@
// 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 "olap/stream_name.h"
namespace doris {
StreamName::StreamName(uint32_t unique_column_id, StreamInfoMessage::Kind kind)
: _unique_column_id(unique_column_id), _kind(kind) {}
bool StreamName::operator<(const StreamName& another) const {
if (_kind == StreamInfoMessage::ROW_INDEX || another._kind == StreamInfoMessage::ROW_INDEX) {
// if both are indexes
if (_kind == another._kind) {
return _unique_column_id < another._unique_column_id;
} else {
return _kind < another._kind;
}
} else {
if (_unique_column_id != another._unique_column_id) {
return _unique_column_id < another._unique_column_id;
} else {
return _kind < another._kind;
}
}
}
bool StreamName::operator==(const StreamName& another) const {
return _unique_column_id == another._unique_column_id && _kind == another._kind;
}
} // namespace doris

View File

@ -1,43 +0,0 @@
// 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.
#pragma once
#include <gen_cpp/column_data_file.pb.h>
namespace doris {
// Define the name of the stream, which is a unique identifier for the stream.
// Implement the comparison function to agree on the order of streams in the file:
// 1. First arrange the index stream: the Index stream is sorted by column unique id.
// 2. Rearrange non-index streams: first by column unique id, then by kind.
class StreamName {
public:
StreamName(uint32_t unique_column_id, StreamInfoMessage::Kind kind);
uint32_t unique_column_id() const { return _unique_column_id; }
StreamInfoMessage::Kind kind() const { return _kind; }
bool operator<(const StreamName& another) const;
bool operator==(const StreamName& another) const;
private:
uint32_t _unique_column_id;
StreamInfoMessage::Kind _kind;
};
} // namespace doris

View File

@ -29,13 +29,6 @@
namespace doris {
void CollectionValue::to_collection_val(CollectionVal* val) const {
val->length = _length;
val->data = _data;
val->null_signs = _null_signs;
val->has_null = _has_null;
}
void CollectionValue::shallow_copy(const CollectionValue* value) {
_length = value->_length;
_null_signs = value->_null_signs;

View File

@ -23,13 +23,11 @@
namespace doris_udf {
class FunctionContext;
struct AnyVal;
} // namespace doris_udf
namespace doris {
using doris_udf::FunctionContext;
using doris_udf::AnyVal;
using MemFootprint = std::pair<int64_t, uint8_t*>;
using GenMemFootprintFunc = std::function<MemFootprint(int64_t size)>;
@ -78,8 +76,6 @@ public:
bool is_null_at(uint64_t index) const { return this->_has_null && this->_null_signs[index]; }
void to_collection_val(CollectionVal* val) const;
uint64_t size() const { return _length; }
uint64_t length() const { return _length; }

View File

@ -47,6 +47,8 @@ enum DecimalError {
enum DecimalRoundMode { HALF_UP = 1, HALF_EVEN = 2, CEILING = 3, FLOOR = 4, TRUNCATE = 5 };
using namespace doris_udf;
class DecimalV2Value {
public:
friend DecimalV2Value operator+(const DecimalV2Value& v1, const DecimalV2Value& v2);

View File

@ -57,18 +57,6 @@ public:
return get_hash_value(value, type.type, 0);
}
// Get the hash value using the fvn hash function. Using different seeds with FVN
// results in different hash functions. get_hash_value() does not have this property
// and cannot be safely used as the first step in data repartitioning.
// However, get_hash_value() can be significantly faster.
// TODO: fix get_hash_value
static uint32_t get_hash_value_fvn(const void* value, const PrimitiveType& type, uint32_t seed);
static uint32_t get_hash_value_fvn(const void* value, const TypeDescriptor& type,
uint32_t seed) {
return get_hash_value_fvn(value, type.type, seed);
}
// Get the hash value using the fvn hash function. Using different seeds with FVN
// results in different hash functions. get_hash_value() does not have this property
// and cannot be safely used as the first step in data repartitioning.
@ -309,75 +297,6 @@ inline uint32_t RawValue::get_hash_value(const void* v, const PrimitiveType& typ
}
}
inline uint32_t RawValue::get_hash_value_fvn(const void* v, const PrimitiveType& type,
uint32_t seed) {
// Hash_combine with v = 0
if (v == nullptr) {
uint32_t value = 0x9e3779b9;
return seed ^ (value + (seed << 6) + (seed >> 2));
}
switch (type) {
case TYPE_VARCHAR:
case TYPE_CHAR:
case TYPE_HLL:
case TYPE_OBJECT:
case TYPE_STRING: {
const StringRef* string_value = reinterpret_cast<const StringRef*>(v);
return HashUtil::fnv_hash(string_value->data, string_value->size, seed);
}
case TYPE_BOOLEAN: {
uint32_t value = *reinterpret_cast<const bool*>(v) + 0x9e3779b9;
return seed ^ (value + (seed << 6) + (seed >> 2));
}
case TYPE_TINYINT:
return HashUtil::fnv_hash(v, 1, seed);
case TYPE_SMALLINT:
return HashUtil::fnv_hash(v, 2, seed);
case TYPE_INT:
return HashUtil::fnv_hash(v, 4, seed);
case TYPE_BIGINT:
return HashUtil::fnv_hash(v, 8, seed);
case TYPE_FLOAT:
return HashUtil::fnv_hash(v, 4, seed);
case TYPE_DOUBLE:
return HashUtil::fnv_hash(v, 8, seed);
case TYPE_DATE:
case TYPE_DATETIME:
return HashUtil::fnv_hash(v, 16, seed);
case TYPE_DATEV2:
return HashUtil::fnv_hash(v, 4, seed);
case TYPE_DATETIMEV2:
return HashUtil::fnv_hash(v, 8, seed);
case TYPE_DECIMALV2:
return HashUtil::fnv_hash(v, 16, seed);
case TYPE_DECIMAL32:
return HashUtil::fnv_hash(v, 4, seed);
case TYPE_DECIMAL64:
return HashUtil::fnv_hash(v, 8, seed);
case TYPE_DECIMAL128I:
return HashUtil::fnv_hash(v, 16, seed);
case TYPE_LARGEINT:
return HashUtil::fnv_hash(v, 16, seed);
default:
DCHECK(false) << "invalid type: " << type;
return 0;
}
}
// NOTE: this is just for split data, decimal use old doris hash function
// Because crc32 hardware is not equal with zlib crc32
inline uint32_t RawValue::zlib_crc32(const void* v, const TypeDescriptor& type, uint32_t seed) {

View File

@ -134,10 +134,6 @@ void FunctionContextImpl::free_local_allocations() {
_local_allocations.clear();
}
void FunctionContextImpl::set_constant_args(const std::vector<doris_udf::AnyVal*>& constant_args) {
_constant_args = constant_args;
}
void FunctionContextImpl::set_constant_cols(
const std::vector<doris::ColumnPtrWrapper*>& constant_cols) {
_constant_cols = constant_cols;
@ -447,70 +443,6 @@ const FunctionContext::TypeDesc* FunctionContext::get_arg_type(int arg_idx) cons
return &_impl->_arg_types[arg_idx];
}
void HllVal::init(FunctionContext* ctx) {
len = doris::HLL_COLUMN_DEFAULT_LEN;
ptr = ctx->allocate(len);
memset(ptr, 0, len);
// the HLL type is HLL_DATA_FULL in UDF or UDAF
ptr[0] = doris::HllDataType::HLL_DATA_FULL;
is_null = false;
}
void HllVal::agg_parse_and_cal(FunctionContext* ctx, const HllVal& other) {
doris::HllSetResolver resolver;
// zero size means the src input is a HyperLogLog object
if (other.len == 0) {
auto* hll = reinterpret_cast<doris::HyperLogLog*>(other.ptr);
uint8_t* other_ptr = ctx->allocate(doris::HLL_COLUMN_DEFAULT_LEN);
int other_len = hll->serialize(ptr);
resolver.init((char*)other_ptr, other_len);
} else {
resolver.init((char*)other.ptr, other.len);
}
resolver.parse();
if (resolver.get_hll_data_type() == doris::HLL_DATA_EMPTY) {
return;
}
uint8_t* pdata = ptr + 1;
int data_len = doris::HLL_REGISTERS_COUNT;
if (resolver.get_hll_data_type() == doris::HLL_DATA_EXPLICIT) {
for (int i = 0; i < resolver.get_explicit_count(); i++) {
uint64_t hash_value = resolver.get_explicit_value(i);
int idx = hash_value % data_len;
uint8_t first_one_bit = __builtin_ctzl(hash_value >> doris::HLL_COLUMN_PRECISION) + 1;
pdata[idx] = std::max(pdata[idx], first_one_bit);
}
} else if (resolver.get_hll_data_type() == doris::HLL_DATA_SPARSE) {
std::map<doris::HllSetResolver::SparseIndexType, doris::HllSetResolver::SparseValueType>&
sparse_map = resolver.get_sparse_map();
for (std::map<doris::HllSetResolver::SparseIndexType,
doris::HllSetResolver::SparseValueType>::iterator iter = sparse_map.begin();
iter != sparse_map.end(); ++iter) {
pdata[iter->first] = std::max(pdata[iter->first], (uint8_t)iter->second);
}
} else if (resolver.get_hll_data_type() == doris::HLL_DATA_FULL) {
char* full_value = resolver.get_full_value();
for (int i = 0; i < doris::HLL_REGISTERS_COUNT; i++) {
pdata[i] = std::max(pdata[i], (uint8_t)full_value[i]);
}
}
}
void HllVal::agg_merge(const HllVal& other) {
uint8_t* pdata = ptr + 1;
uint8_t* pdata_other = other.ptr + 1;
for (int i = 0; i < doris::HLL_REGISTERS_COUNT; ++i) {
pdata[i] = std::max(pdata[i], pdata_other[i]);
}
}
bool FunctionContext::is_arg_constant(int i) const {
if (i < 0 || i >= _impl->_constant_args.size()) {
return false;
@ -543,10 +475,6 @@ int FunctionContext::get_num_args() const {
return _impl->_arg_types.size();
}
int FunctionContext::get_num_constant_args() const {
return _impl->_constant_args.size();
}
const FunctionContext::TypeDesc& FunctionContext::get_return_type() const {
return _impl->_return_type;
}

View File

@ -23,6 +23,7 @@
#include <string.h>
#include <cstdint>
#include <functional>
#include <iostream>
#include <vector>
@ -46,17 +47,9 @@ namespace doris_udf {
// object containing a boolean to store if the value is nullptr and the value itself. The
// value is unspecified if the nullptr boolean is set.
struct AnyVal;
struct BooleanVal;
struct TinyIntVal;
struct SmallIntVal;
struct IntVal;
struct BigIntVal;
struct StringVal;
struct DateTimeVal;
struct DateV2Val;
struct DecimalV2Val;
struct HllVal;
struct CollectionVal;
// The FunctionContext is passed to every UDF/UDA and is the interface for the UDF to the
// rest of the system. It contains APIs to examine the system state, report errors
@ -200,6 +193,7 @@ public:
// byte sizes. For each call to TrackAllocation(), the UDF/UDA must call
// the corresponding Free().
void track_allocation(int64_t byte_size);
void free(int64_t byte_size);
// TODO: Do we need to add arbitrary key/value metadata. This would be plumbed
@ -220,6 +214,7 @@ public:
/// nullptr. SetFunctionState() does not take ownership of 'ptr'; it is up to the UDF/UDA
/// to clean up any function state if necessary.
void set_function_state(FunctionStateScope scope, void* ptr);
void* get_function_state(FunctionStateScope scope) const;
// Returns the return type information of this function. For UDAs, this is the final
@ -234,9 +229,6 @@ public:
// argument).
int get_num_args() const;
// Returns _constant_args size
int get_num_constant_args() const;
// Returns the type information for the arg_idx-th argument (0-indexed, not including
// the FunctionContext* argument). Returns nullptr if arg_idx is invalid.
const TypeDesc* get_arg_type(int arg_idx) const;
@ -264,259 +256,33 @@ public:
private:
friend class doris::FunctionContextImpl;
FunctionContext();
// Disable copy ctor and assignment operator
FunctionContext(const FunctionContext& other);
FunctionContext& operator=(const FunctionContext& other);
doris::FunctionContextImpl* _impl; // Owned by this object.
};
//----------------------------------------------------------------------------
//------------------------------- UDFs ---------------------------------------
//----------------------------------------------------------------------------
// The UDF function must implement this function prototype. This is not
// a typedef as the actual UDF's signature varies from UDF to UDF.
// typedef <*Val> Evaluate(FunctionContext* context, <const Val& arg>);
//
// The UDF must return one of the *Val structs. The UDF must accept a pointer
// to a FunctionContext object and then a const reference for each of the input arguments.
// nullptr input arguments will have nullptr passed in.
// Examples of valid Udf signatures are:
// 1) DoubleVal Example1(FunctionContext* context);
// 2) IntVal Example2(FunctionContext* context, const IntVal& a1, const DoubleVal& a2);
//
// UDFs can be variadic. The variable arguments must all come at the end and must be
// the same type. A example signature is:
// StringVal Concat(FunctionContext* context, const StringVal& separator,
// int num_var_args, const StringVal* args);
// In this case args[0] is the first variable argument and args[num_var_args - 1] is
// the last.
//
// The UDF should not maintain any state across calls since there is no guarantee
// on how the execution is multithreaded or distributed. Conceptually, the UDF
// should only read the input arguments and return the result, using only the
// FunctionContext as an external object.
//
// Memory Management: the UDF can assume that memory from input arguments will have
// the same lifetime as results for the UDF. In other words, the UDF can return
// memory from input arguments without making copies. For example, a function like
// substring will not need to allocate and copy the smaller string. For cases where
// the UDF needs a buffer, it should use the StringRef(FunctionContext, len) c'tor.
// TODO: things above is not right. StringRef shouldn't use here.
//
// The UDF can optionally specify a Prepare function. The prepare function is called
// once before any calls to the Udf to evaluate values. This is the appropriate time for
// the Udf to validate versions and things like that.
// If there is an error, this function should call FunctionContext::set_error()/
// FunctionContext::add_warning().
typedef void (*UdfPrepareFn)(FunctionContext* context);
/// --- Prepare / Close Functions ---
/// ---------------------------------
/// The UDF can optionally include a prepare function, specified in the "CREATE FUNCTION"
/// statement using "prepare_fn=<prepare function symbol>". The prepare function is called
/// before any calls to the UDF to evaluate values. This is the appropriate time for the
/// UDF to initialize any shared data structures, validate versions, etc. If there is an
/// error, this function should call FunctionContext::SetError()/
/// FunctionContext::AddWarning().
//
/// The prepare function is called multiple times with different FunctionStateScopes. It
/// will be called once per fragment with 'scope' set to FRAGMENT_LOCAL, and once per
/// execution thread with 'scope' set to THREAD_LOCAL.
typedef void (*UdfPrepare)(FunctionContext* context, FunctionContext::FunctionStateScope scope);
/// The UDF can also optionally include a close function, specified in the "CREATE
/// FUNCTION" statement using "close_fn=<close function symbol>". The close function is
/// called after all calls to the UDF have completed. This is the appropriate time for the
/// UDF to deallocate any shared data structures that are not needed to maintain the
/// results. If there is an error, this function should call FunctionContext::SetError()/
/// FunctionContext::AddWarning().
//
/// The close function is called multiple times with different FunctionStateScopes. It
/// will be called once per fragment with 'scope' set to FRAGMENT_LOCAL, and once per
/// execution thread with 'scope' set to THREAD_LOCAL.
typedef void (*UdfClose)(FunctionContext* context, FunctionContext::FunctionStateScope scope);
//----------------------------------------------------------------------------
//------------------------------- UDAs ---------------------------------------
//----------------------------------------------------------------------------
// The UDA execution is broken up into a few steps. The general calling pattern
// is one of these:
// 1) Init(), Evaluate() (repeatedly), Serialize()
// 2) Init(), Merge() (repeatedly), Serialize()
// 3) Init(), Finalize()
// The UDA is registered with three types: the result type, the input type and
// the intermediate type.
//
// If the UDA needs a fixed byte width intermediate buffer, the type should be
// TYPE_FIXED_BUFFER and Doris will allocate the buffer. If the UDA needs an unknown
// sized buffer, it should use TYPE_STRING and allocate it from the FunctionContext
// manually.
// For UDAs that need a complex data structure as the intermediate state, the
// intermediate type should be string and the UDA can cast the ptr to the structure
// it is using.
//
// Memory Management: For allocations that are not returned to Doris, the UDA
// should use the FunctionContext::Allocate()/Free() methods. For StringVal allocations
// returned to Doris (e.g. UdaSerialize()), the UDA should allocate the result
// via StringVal(FunctionContext*, int) ctor and Doris will automatically handle
// freeing it.
//
// For clarity in documenting the UDA interface, the various types will be typedefed
// here. The actual execution resolves all the types at runtime and none of these types
// should actually be used.
typedef AnyVal InputType;
typedef AnyVal InputType2;
typedef AnyVal ResultType;
typedef AnyVal IntermediateType;
// UdaInit is called once for each aggregate group before calls to any of the
// other functions below.
typedef void (*UdaInit)(FunctionContext* context, IntermediateType* result);
// This is called for each input value. The UDA should update result based on the
// input value. The update function can take any number of input arguments. Here
// are some examples:
typedef void (*UdaUpdate)(FunctionContext* context, const InputType& input,
IntermediateType* result);
typedef void (*UdaUpdate2)(FunctionContext* context, const InputType& input,
const InputType2& input2, IntermediateType* result);
// Merge an intermediate result 'src' into 'dst'.
typedef void (*UdaMerge)(FunctionContext* context, const IntermediateType& src,
IntermediateType* dst);
// Serialize the intermediate type. The serialized data is then sent across the
// wire. This is not called unless the intermediate type is String.
// No additional functions will be called with this FunctionContext object and the
// UDA should do final clean (e.g. Free()) here.
typedef const IntermediateType (*UdaSerialize)(FunctionContext* context,
const IntermediateType& type);
// Called once at the end to return the final value for this UDA.
// No additional functions will be called with this FunctionContext object and the
// UDA should do final clean (e.g. Free()) here.
typedef ResultType (*UdaFinalize)(FunctionContext* context, const IntermediateType& v);
//----------------------------------------------------------------------------
//-------------Implementation of the *Val structs ----------------------------
//----------------------------------------------------------------------------
struct AnyVal {
bool is_null;
AnyVal() : is_null(false) {}
AnyVal(bool is_null) : is_null(is_null) {}
};
struct BooleanVal : public AnyVal {
bool val;
BooleanVal() : val(false) {}
BooleanVal(bool val) : val(val) {}
static BooleanVal null() {
BooleanVal result;
result.is_null = true;
return result;
}
bool operator==(const BooleanVal& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const BooleanVal& other) const { return !(*this == other); }
};
struct TinyIntVal : public AnyVal {
int8_t val;
TinyIntVal() : val(0) {}
TinyIntVal(int8_t val) : val(val) {}
static TinyIntVal null() {
TinyIntVal result;
result.is_null = true;
return result;
}
bool operator==(const TinyIntVal& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const TinyIntVal& other) const { return !(*this == other); }
};
struct SmallIntVal : public AnyVal {
int16_t val;
SmallIntVal() : val(0) {}
SmallIntVal(int16_t val) : val(val) {}
static SmallIntVal null() {
SmallIntVal result;
result.is_null = true;
return result;
}
bool operator==(const SmallIntVal& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const SmallIntVal& other) const { return !(*this == other); }
};
struct IntVal : public AnyVal {
int32_t val;
IntVal() : val(0) {}
IntVal(int32_t val) : val(val) {}
static IntVal null() {
IntVal result;
result.is_null = true;
return result;
}
bool operator==(const IntVal& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const IntVal& other) const { return !(*this == other); }
};
struct BigIntVal : public AnyVal {
int64_t val;
BigIntVal() : val(0) {}
BigIntVal(int64_t val) : val(val) {}
static BigIntVal null() {
@ -536,110 +302,15 @@ struct BigIntVal : public AnyVal {
return val == other.val;
}
bool operator!=(const BigIntVal& other) const { return !(*this == other); }
};
struct Decimal32Val : public AnyVal {
int32_t val;
Decimal32Val() : val(0) {}
Decimal32Val(int32_t val) : val(val) {}
static Decimal32Val null() {
Decimal32Val result;
result.is_null = true;
return result;
}
bool operator==(const Decimal32Val& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const Decimal32Val& other) const { return !(*this == other); }
};
struct Decimal64Val : public AnyVal {
int64_t val;
Decimal64Val() : val(0) {}
Decimal64Val(int64_t val) : val(val) {}
static Decimal64Val null() {
Decimal64Val result;
result.is_null = true;
return result;
}
bool operator==(const Decimal64Val& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const Decimal64Val& other) const { return !(*this == other); }
};
struct Decimal128Val : public AnyVal {
__int128 val;
Decimal128Val() : val(0) {}
Decimal128Val(__int128 large_value) : val(large_value) {}
static Decimal128Val null() {
Decimal128Val result;
result.is_null = true;
return result;
}
bool operator==(const Decimal128Val& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const Decimal128Val& other) const { return !(*this == other); }
};
struct FloatVal : public AnyVal {
float val;
FloatVal() : val(0.0) {}
FloatVal(float val) : val(val) {}
static FloatVal null() {
FloatVal result;
result.is_null = true;
return result;
}
bool operator==(const FloatVal& other) const {
return is_null == other.is_null && val == other.val;
}
bool operator!=(const FloatVal& other) const { return !(*this == other); }
};
struct DoubleVal : public AnyVal {
double val;
DoubleVal() : val(0.0) {}
DoubleVal(double val) : val(val) {}
static DoubleVal null() {
@ -659,6 +330,7 @@ struct DoubleVal : public AnyVal {
return val == other.val;
}
bool operator!=(const DoubleVal& other) const { return !(*this == other); }
};
@ -689,39 +361,15 @@ struct DateTimeVal : public AnyVal {
return packed_time == other.packed_time;
}
bool operator!=(const DateTimeVal& other) const { return !(*this == other); }
};
struct DateV2Val : public AnyVal {
uint32_t datev2_value;
DateV2Val() : datev2_value(0) {}
DateV2Val(uint32_t val) : datev2_value(val) {}
static DateV2Val null() {
DateV2Val result;
result.is_null = true;
return result;
}
bool operator==(const DateV2Val& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return datev2_value == other.datev2_value;
}
bool operator!=(const DateV2Val& other) const { return !(*this == other); }
};
struct DateTimeV2Val : public AnyVal {
uint64_t datetimev2_value;
DateTimeV2Val() : datetimev2_value(0) {}
DateTimeV2Val(uint64_t val) : datetimev2_value(val) {}
static DateTimeV2Val null() {
@ -741,6 +389,7 @@ struct DateTimeV2Val : public AnyVal {
return datetimev2_value == other.datetimev2_value;
}
bool operator!=(const DateTimeV2Val& other) const { return !(*this == other); }
};
@ -810,10 +459,13 @@ struct StringVal : public AnyVal {
/// the memory allocation becomes too large, will set an error on FunctionContext and
/// return a nullptr string.
void append(FunctionContext* ctx, const uint8_t* buf, int64_t len);
void append(FunctionContext* ctx, const uint8_t* buf, int64_t len, const uint8_t* buf2,
int64_t buf2_len);
std::string to_string() const { return std::string((char*)ptr, len); }
};
std::ostream& operator<<(std::ostream& os, const StringVal& string_val);
struct DecimalV2Val : public AnyVal {
@ -853,82 +505,10 @@ struct DecimalV2Val : public AnyVal {
bool operator!=(const DecimalV2Val& other) const { return !(*this == other); }
};
struct LargeIntVal : public AnyVal {
__int128 val;
LargeIntVal() : val(0) {}
LargeIntVal(__int128 large_value) : val(large_value) {}
static LargeIntVal null() {
LargeIntVal result;
result.is_null = true;
return result;
}
bool operator==(const LargeIntVal& other) const {
if (is_null && other.is_null) {
return true;
}
if (is_null || other.is_null) {
return false;
}
return val == other.val;
}
bool operator!=(const LargeIntVal& other) const { return !(*this == other); }
};
// todo(kks): keep HllVal struct only for backward compatibility, we should remove it
// when doris 0.12 release
struct HllVal : public StringVal {
HllVal() : StringVal() {}
void init(FunctionContext* ctx);
void agg_parse_and_cal(FunctionContext* ctx, const HllVal& other);
void agg_merge(const HllVal& other);
};
struct CollectionVal : public AnyVal {
void* data;
uint64_t length;
// item has no null value if has_null is false.
// item ```may``` has null value if has_null is true.
bool has_null;
// null bitmap
bool* null_signs;
CollectionVal() = default;
CollectionVal(void* data, uint64_t length, bool has_null, bool* null_signs)
: data(data), length(length), has_null(has_null), null_signs(null_signs) {};
static CollectionVal null() {
CollectionVal val;
val.is_null = true;
return val;
}
};
typedef uint8_t* BufferVal;
} // namespace doris_udf
using doris_udf::BooleanVal;
using doris_udf::TinyIntVal;
using doris_udf::SmallIntVal;
using doris_udf::IntVal;
using doris_udf::BigIntVal;
using doris_udf::LargeIntVal;
using doris_udf::FloatVal;
using doris_udf::DoubleVal;
using doris_udf::StringVal;
using doris_udf::DecimalV2Val;
using doris_udf::DateTimeVal;
using doris_udf::HllVal;
using doris_udf::FunctionContext;
using doris_udf::CollectionVal;
using doris_udf::Decimal32Val;
using doris_udf::Decimal64Val;
using doris_udf::Decimal128Val;
} // namespace doris_udf

View File

@ -68,14 +68,10 @@ public:
/// it.
doris_udf::FunctionContext* clone(MemPool* pool);
void set_constant_args(const std::vector<doris_udf::AnyVal*>& constant_args);
void set_constant_cols(const std::vector<doris::ColumnPtrWrapper*>& cols);
uint8_t* varargs_buffer() { return _varargs_buffer; }
std::vector<doris_udf::AnyVal*>* staging_input_vals() { return &_staging_input_vals; }
bool closed() const { return _closed; }
int64_t num_updates() const { return _num_updates; }
@ -181,11 +177,6 @@ private:
std::vector<doris::ColumnPtrWrapper*> _constant_cols;
// Used by ScalarFnCall to store the arguments when running without codegen. Allows us
// to pass AnyVal* arguments to the scalar function directly, rather than codegening a
// call that passes the correct AnyVal subclass pointer type.
std::vector<doris_udf::AnyVal*> _staging_input_vals;
bool _check_overflow_for_decimal = false;
// Indicates whether this context has been closed. Used for verification/debugging.

View File

@ -30,13 +30,6 @@ public:
static const int DATETIME_TYPE_BYTE_SIZE = 4;
static const int DECIMAL_BYTE_SIZE = 16;
// get_val start
template <typename ValType, typename T>
static T get_val(const ValType& x) {
DCHECK(!x.is_null);
return x.val;
}
// serialize_size start
template <typename T>
static int32_t serialize_size(const T& v) {
@ -61,23 +54,6 @@ public:
}
};
template <>
inline StringRef Helper::get_val<StringVal>(const StringVal& x) {
DCHECK(!x.is_null);
return StringRef(x);
}
template <>
inline DateTimeValue Helper::get_val<DateTimeVal>(const DateTimeVal& x) {
return DateTimeValue::from_datetime_val(x);
}
template <>
inline DecimalV2Value Helper::get_val<DecimalV2Val>(const DecimalV2Val& x) {
return DecimalV2Value::from_decimal_val(x);
}
// get_val end
template <>
inline char* Helper::write_to<DateTimeValue>(const DateTimeValue& v, char* dest) {
DateTimeVal value;

View File

@ -17,7 +17,6 @@
#pragma once
#include "exprs/anyval_util.h"
#include "olap/hll.h"
#include "udf/udf.h"
#include "vec/aggregate_functions/aggregate_function.h"
@ -87,7 +86,7 @@ public:
} else {
auto value = static_cast<const ColumnDataType*>(columns[0])->get_data_at(row_num);
StringVal sv = value.to_string_val();
uint64_t hash_value = AnyValUtil::hash64_murmur(sv, HashUtil::MURMUR_SEED);
uint64_t hash_value = HashUtil::murmur_hash64A(sv.ptr, sv.len, HashUtil::MURMUR_SEED);
this->data(place).add(hash_value);
}
}

View File

@ -192,6 +192,7 @@ inline int string_compare(const char* s1, int64_t n1, const char* s2, int64_t n2
} // unnamed namespace
using namespace doris_udf;
/// The thing to avoid creating strings to find substrings in the hash table.
/// User should make sure data source is const.
/// maybe considering rewrite it with std::span / std::basic_string_view is meaningful.

View File

@ -21,7 +21,6 @@
#include "common/consts.h"
#include "common/status.h"
#include "exprs/anyval_util.h"
#include "fmt/format.h"
#include "fmt/ranges.h"
#include "udf/udf_internal.h"

View File

@ -22,7 +22,6 @@
#include <memory>
#include "exprs/anyval_util.h"
#include "gen_cpp/Exprs_types.h"
#include "vec/data_types/data_type_factory.hpp"
#include "vec/exprs/varray_literal.h"
@ -270,6 +269,109 @@ Status VExpr::clone_if_not_exists(const std::vector<VExprContext*>& ctxs, Runtim
}
return Status::OK();
}
FunctionContext::TypeDesc VExpr::column_type_to_type_desc(const TypeDescriptor& type) {
FunctionContext::TypeDesc out;
switch (type.type) {
case TYPE_BOOLEAN:
out.type = FunctionContext::TYPE_BOOLEAN;
break;
case TYPE_TINYINT:
out.type = FunctionContext::TYPE_TINYINT;
break;
case TYPE_SMALLINT:
out.type = FunctionContext::TYPE_SMALLINT;
break;
case TYPE_INT:
out.type = FunctionContext::TYPE_INT;
break;
case TYPE_BIGINT:
out.type = FunctionContext::TYPE_BIGINT;
break;
case TYPE_LARGEINT:
out.type = FunctionContext::TYPE_LARGEINT;
break;
case TYPE_FLOAT:
out.type = FunctionContext::TYPE_FLOAT;
break;
case TYPE_TIME:
case TYPE_TIMEV2:
case TYPE_DOUBLE:
out.type = FunctionContext::TYPE_DOUBLE;
break;
case TYPE_DATE:
out.type = FunctionContext::TYPE_DATE;
break;
case TYPE_DATETIME:
out.type = FunctionContext::TYPE_DATETIME;
break;
case TYPE_DATEV2:
out.type = FunctionContext::TYPE_DATEV2;
break;
case TYPE_DATETIMEV2:
out.type = FunctionContext::TYPE_DATETIMEV2;
break;
case TYPE_DECIMAL32:
out.type = FunctionContext::TYPE_DECIMAL32;
out.precision = type.precision;
out.scale = type.scale;
break;
case TYPE_DECIMAL64:
out.type = FunctionContext::TYPE_DECIMAL64;
out.precision = type.precision;
out.scale = type.scale;
break;
case TYPE_DECIMAL128I:
out.type = FunctionContext::TYPE_DECIMAL128I;
out.precision = type.precision;
out.scale = type.scale;
break;
case TYPE_VARCHAR:
out.type = FunctionContext::TYPE_VARCHAR;
out.len = type.len;
break;
case TYPE_HLL:
out.type = FunctionContext::TYPE_HLL;
out.len = type.len;
break;
case TYPE_OBJECT:
out.type = FunctionContext::TYPE_OBJECT;
// FIXME(cmy): is this fallthrough meaningful?
case TYPE_QUANTILE_STATE:
out.type = FunctionContext::TYPE_QUANTILE_STATE;
break;
case TYPE_CHAR:
out.type = FunctionContext::TYPE_CHAR;
out.len = type.len;
break;
case TYPE_DECIMALV2:
out.type = FunctionContext::TYPE_DECIMALV2;
// out.precision = type.precision;
// out.scale = type.scale;
break;
case TYPE_NULL:
out.type = FunctionContext::TYPE_NULL;
break;
case TYPE_ARRAY:
out.type = FunctionContext::TYPE_ARRAY;
for (const auto& t : type.children) {
out.children.push_back(VExpr::column_type_to_type_desc(t));
}
break;
case TYPE_STRING:
out.type = FunctionContext::TYPE_STRING;
out.len = type.len;
break;
case TYPE_JSONB:
out.type = FunctionContext::TYPE_JSONB;
out.len = type.len;
break;
default:
DCHECK(false) << "Unknown type: " << type;
}
return out;
}
std::string VExpr::debug_string() const {
// TODO: implement partial debug string for member vars
std::stringstream out;
@ -339,10 +441,10 @@ Status VExpr::get_const_col(VExprContext* context, ColumnPtrWrapper** output) {
}
void VExpr::register_function_context(doris::RuntimeState* state, VExprContext* context) {
FunctionContext::TypeDesc return_type = AnyValUtil::column_type_to_type_desc(_type);
FunctionContext::TypeDesc return_type = VExpr::column_type_to_type_desc(_type);
std::vector<FunctionContext::TypeDesc> arg_types;
for (int i = 0; i < _children.size(); ++i) {
arg_types.push_back(AnyValUtil::column_type_to_type_desc(_children[i]->type()));
arg_types.push_back(VExpr::column_type_to_type_desc(_children[i]->type()));
}
_fn_context_index = context->register_func(state, return_type, arg_types, 0);

View File

@ -180,6 +180,7 @@ public:
}
protected:
static FunctionContext::TypeDesc column_type_to_type_desc(const TypeDescriptor& type);
/// Simple debug string that provides no expr subclass-specific information
std::string debug_string(const std::string& expr_name) const {
std::stringstream out;

View File

@ -17,7 +17,6 @@
#include "vec/functions/functions_geo.h"
#include "geo/geo_functions.h"
#include "geo/geo_types.h"
#include "gutil/strings/substitute.h"
#include "vec/columns/column_const.h"

View File

@ -17,6 +17,7 @@
#pragma once
#include "geo/geo_types.h"
#include "udf/udf.h"
#include "vec/core/column_numbers.h"
#include "vec/data_types/data_type_number.h"
@ -25,6 +26,21 @@
namespace doris::vectorized {
struct StConstructState {
StConstructState() : is_null(false) {}
~StConstructState() {}
bool is_null;
std::string encoded_buf;
};
struct StContainsState {
StContainsState() : is_null(false), shapes {nullptr, nullptr} {}
~StContainsState() {}
bool is_null;
std::vector<std::shared_ptr<GeoShape>> shapes;
};
template <typename Impl, typename ReturnType = DataTypeString>
class GeoFunction : public IFunction {
public:

View File

@ -581,7 +581,7 @@ public:
return value;
}
uint32_t hash(int seed) const { return HashUtil::hash(this, sizeof(*this), seed); }
uint32_t hash(int seed) const { return ::doris::HashUtil::hash(this, sizeof(*this), seed); }
int day_of_year() const { return daynr() - calc_daynr(_year, 1, 1) + 1; }
@ -1040,7 +1040,7 @@ public:
DateV2Value<T>& operator--() { return *this += -1; }
uint32_t hash(int seed) const { return HashUtil::hash(this, sizeof(*this), seed); }
uint32_t hash(int seed) const { return ::doris::HashUtil::hash(this, sizeof(*this), seed); }
int day_of_year() const { return daynr() - calc_daynr(this->year(), 1, 1) + 1; }
@ -1094,23 +1094,6 @@ public:
bool get_date_from_daynr(uint64_t);
void to_datev2_val(doris_udf::DateV2Val* tv) const {
DCHECK(!is_datetime);
tv->datev2_value = this->to_date_int_val();
}
static DateV2Value<DateV2ValueType> from_datev2_val(const doris_udf::DateV2Val& tv) {
DCHECK(!is_datetime);
DateV2Value<DateV2ValueType> value;
value.from_date(tv.datev2_value);
return value;
}
void to_datetimev2_val(doris_udf::DateTimeV2Val* tv) const {
DCHECK(is_datetime);
tv->datetimev2_value = this->to_date_int_val();
}
static DateV2Value<DateTimeV2ValueType> from_datetimev2_val(
const doris_udf::DateTimeV2Val& tv) {
DCHECK(is_datetime);
@ -1505,24 +1488,25 @@ struct DateTraits<uint64_t> {
} // namespace doris
template <>
struct std::hash<doris::vectorized::VecDateTimeValue> {
size_t operator()(const doris::vectorized::VecDateTimeValue& v) const {
return doris::vectorized::hash_value(v);
struct std::hash<::doris::vectorized::VecDateTimeValue> {
size_t operator()(const ::doris::vectorized::VecDateTimeValue& v) const {
return ::doris::vectorized::hash_value(v);
}
};
template <>
struct std::hash<doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>> {
struct std::hash<::doris::vectorized::DateV2Value<::doris::vectorized::DateV2ValueType>> {
size_t operator()(
const doris::vectorized::DateV2Value<doris::vectorized::DateV2ValueType>& v) const {
return doris::vectorized::hash_value(v);
const ::doris::vectorized::DateV2Value<::doris::vectorized::DateV2ValueType>& v) const {
return ::doris::vectorized::hash_value(v);
}
};
template <>
struct std::hash<doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>> {
struct std::hash<::doris::vectorized::DateV2Value<::doris::vectorized::DateTimeV2ValueType>> {
size_t operator()(
const doris::vectorized::DateV2Value<doris::vectorized::DateTimeV2ValueType>& v) const {
return doris::vectorized::hash_value(v);
const ::doris::vectorized::DateV2Value<::doris::vectorized::DateTimeV2ValueType>& v)
const {
return ::doris::vectorized::hash_value(v);
}
};

View File

@ -258,7 +258,7 @@ Status VOrcWriterWrapper::write(const Block& block) {
// Buffer used by date type
char* ptr = (char*)malloc(BUFFER_UNIT_SIZE);
::StringVal buffer(ptr, BUFFER_UNIT_SIZE);
StringVal buffer(ptr, BUFFER_UNIT_SIZE);
size_t sz = block.rows();
auto row_batch = _create_row_batch(sz);

View File

@ -56,7 +56,6 @@ set(EXPRS_TEST_FILES
)
set(GEO_TEST_FILES
geo/wkt_parse_test.cpp
geo/geo_functions_test.cpp
geo/geo_types_test.cpp
)
set(GUTIL_TEST_FILES
@ -169,7 +168,6 @@ set(RUNTIME_TEST_FILES
)
set(TESTUTIL_TEST_FILES
testutil/test_util.cpp
testutil/array_utils.cpp
testutil/desc_tbl_builder.cc
testutil/function_utils.cpp
testutil/run_all_tests.cpp

View File

@ -26,7 +26,6 @@
#include <string>
#include "common/object_pool.h"
#include "exprs/anyval_util.h"
#include "exprs/json_functions.h"
#include "runtime/runtime_state.h"
#include "util/stopwatch.hpp"

View File

@ -1,327 +0,0 @@
// 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 "geo/geo_functions.h"
#include <gtest/gtest.h>
#include <s2/s2polygon.h>
#include <vector>
#include "geo/geo_types.h"
#include "geo/wkt_parse.h"
#include "testutil/function_utils.h"
#include "udf/udf.h"
#include "udf/udf_internal.h"
namespace doris {
class GeoFunctionsTest : public testing::Test {
public:
GeoFunctionsTest() {}
virtual ~GeoFunctionsTest() {}
};
TEST_F(GeoFunctionsTest, st_dist_sphere) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
{
DoubleVal x_lng(0.0);
DoubleVal x_lat(0.0);
DoubleVal y_lng(0.0);
DoubleVal y_lat(0.0);
auto dist = GeoFunctions::st_distance_sphere(ctx, x_lng, x_lat, y_lng, y_lat);
EXPECT_EQ(0, dist.val);
}
{
DoubleVal x_lng(0.0);
DoubleVal x_lat(0.0);
DoubleVal y_lng(0.0);
DoubleVal y_lat(1.0);
auto dist = GeoFunctions::st_distance_sphere(ctx, x_lng, x_lat, y_lng, y_lat);
LOG(INFO) << dist.val;
}
}
TEST_F(GeoFunctionsTest, st_point) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
DoubleVal lng(113);
DoubleVal lat(64);
auto str = GeoFunctions::st_point(ctx, lng, lat);
EXPECT_FALSE(str.is_null);
GeoPoint point;
auto res = point.decode_from(str.ptr, str.len);
EXPECT_TRUE(res);
EXPECT_EQ(113, point.x());
EXPECT_EQ(64, point.y());
}
TEST_F(GeoFunctionsTest, st_x_y) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoPoint point;
point.from_coord(134, 63);
std::string buf;
point.encode_to(&buf);
auto x = GeoFunctions::st_x(ctx, StringVal((uint8_t*)buf.data(), buf.size()));
auto y = GeoFunctions::st_y(ctx, StringVal((uint8_t*)buf.data(), buf.size()));
EXPECT_EQ(134, x.val);
EXPECT_EQ(63, y.val);
}
TEST_F(GeoFunctionsTest, as_wkt) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoPoint point;
point.from_coord(134, 63);
std::string buf;
point.encode_to(&buf);
auto wkt = GeoFunctions::st_as_wkt(ctx, StringVal((uint8_t*)buf.data(), buf.size()));
EXPECT_STREQ("POINT (134 63)", std::string((char*)wkt.ptr, wkt.len).c_str());
}
TEST_F(GeoFunctionsTest, st_from_wkt) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoFunctions::st_from_wkt_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_EQ(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
std::string wkt = "POINT (10.1 20.2)";
auto str = GeoFunctions::st_from_wkt(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_FALSE(str.is_null);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
// second time
{
StringVal wkt_val((uint8_t*)wkt.data(), wkt.size());
// push const value
std::vector<doris_udf::AnyVal*> const_vals;
const_vals.push_back(&wkt_val);
ctx->impl()->set_constant_args(const_vals);
// prepare
GeoFunctions::st_from_wkt_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_NE(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
// convert
auto str2 = GeoFunctions::st_from_wkt(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_FALSE(str2.is_null);
// close
GeoPoint point;
auto res = point.decode_from(str2.ptr, str2.len);
EXPECT_TRUE(res);
EXPECT_DOUBLE_EQ(10.1, point.x());
EXPECT_DOUBLE_EQ(20.2, point.y());
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
}
TEST_F(GeoFunctionsTest, st_line) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoFunctions::st_from_wkt_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
std::string wkt = "LINESTRING (10.1 20.2, 21.1 30.1)";
auto str = GeoFunctions::st_from_wkt(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_FALSE(str.is_null);
GeoLine line;
auto res = line.decode_from(str.ptr, str.len);
EXPECT_TRUE(res);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
// second time
{
StringVal wkt_val((uint8_t*)wkt.data(), wkt.size());
// push const value
std::vector<doris_udf::AnyVal*> const_vals;
const_vals.push_back(&wkt_val);
ctx->impl()->set_constant_args(const_vals);
// prepare
GeoFunctions::st_line_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_NE(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
// convert
auto str2 = GeoFunctions::st_line(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_FALSE(str2.is_null);
// close
GeoLine line;
auto res = line.decode_from(str2.ptr, str2.len);
EXPECT_TRUE(res);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
}
TEST_F(GeoFunctionsTest, st_polygon) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoFunctions::st_from_wkt_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
std::string wkt = "POLYGON ((10 10, 50 10, 50 50, 10 50, 10 10))";
auto str = GeoFunctions::st_from_wkt(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_FALSE(str.is_null);
// second time
{
StringVal wkt_val((uint8_t*)wkt.data(), wkt.size());
// push const value
std::vector<doris_udf::AnyVal*> const_vals;
const_vals.push_back(&wkt_val);
ctx->impl()->set_constant_args(const_vals);
// prepare
GeoFunctions::st_polygon_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_NE(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
// convert
auto str2 = GeoFunctions::st_polygon(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_FALSE(str2.is_null);
// close
GeoPolygon polygon;
auto res = polygon.decode_from(str2.ptr, str2.len);
EXPECT_TRUE(res);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
}
TEST_F(GeoFunctionsTest, st_circle) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoFunctions::st_from_wkt_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
DoubleVal lng(111);
DoubleVal lat(64);
DoubleVal radius_meter(10 * 100);
auto str = GeoFunctions::st_circle(ctx, lng, lat, radius_meter);
EXPECT_FALSE(str.is_null);
// second time
{
// push const value
std::vector<doris_udf::AnyVal*> const_vals;
const_vals.push_back(&lng);
const_vals.push_back(&lat);
const_vals.push_back(&radius_meter);
ctx->impl()->set_constant_args(const_vals);
// prepare
GeoFunctions::st_circle_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_NE(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
// convert
auto str2 = GeoFunctions::st_circle(ctx, lng, lat, radius_meter);
EXPECT_FALSE(str2.is_null);
// close
GeoCircle circle;
auto res = circle.decode_from(str2.ptr, str2.len);
EXPECT_TRUE(res);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
}
TEST_F(GeoFunctionsTest, st_poly_line_fail) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
{
GeoFunctions::st_polygon_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
std::string wkt = "POINT (10.1 20.2)";
auto str = GeoFunctions::st_polygon(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_TRUE(str.is_null);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
{
GeoFunctions::st_line_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
std::string wkt = "POINT (10.1 20.2)";
auto str = GeoFunctions::st_line(ctx, StringVal((uint8_t*)wkt.data(), wkt.size()));
EXPECT_TRUE(str.is_null);
GeoFunctions::st_from_wkt_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
}
TEST_F(GeoFunctionsTest, st_contains) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
EXPECT_EQ(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
std::string polygon_wkt = "POLYGON ((10 10, 50 10, 50 50, 10 50, 10 10))";
auto polygon = GeoFunctions::st_from_wkt(
ctx, StringVal((uint8_t*)polygon_wkt.data(), polygon_wkt.size()));
EXPECT_EQ(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
std::string point_wkt = "POINT (25 25)";
auto point =
GeoFunctions::st_from_wkt(ctx, StringVal((uint8_t*)point_wkt.data(), point_wkt.size()));
EXPECT_EQ(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
GeoFunctions::st_contains_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_EQ(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
auto res = GeoFunctions::st_contains(ctx, polygon, point);
EXPECT_TRUE(res.val);
GeoFunctions::st_contains_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
TEST_F(GeoFunctionsTest, st_contains_cached) {
FunctionUtils utils;
FunctionContext* ctx = utils.get_fn_ctx();
GeoFunctions::st_from_wkt_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
std::string polygon_wkt = "POLYGON ((10 10, 50 10, 50 50, 10 50, 10 10))";
auto polygon = GeoFunctions::st_from_wkt(
ctx, StringVal((uint8_t*)polygon_wkt.data(), polygon_wkt.size()));
std::string point_wkt = "POINT (25 25)";
auto point =
GeoFunctions::st_from_wkt(ctx, StringVal((uint8_t*)point_wkt.data(), point_wkt.size()));
// push const value
std::vector<doris_udf::AnyVal*> const_vals;
const_vals.push_back(&polygon);
const_vals.push_back(&point);
ctx->impl()->set_constant_args(const_vals);
// prepare
GeoFunctions::st_contains_prepare(ctx, FunctionContext::FRAGMENT_LOCAL);
EXPECT_NE(nullptr, ctx->get_function_state(FunctionContext::FRAGMENT_LOCAL));
auto res = GeoFunctions::st_contains(ctx, polygon, point);
EXPECT_TRUE(res.val);
GeoFunctions::st_contains_close(ctx, FunctionContext::FRAGMENT_LOCAL);
}
} // namespace doris

View File

@ -1,84 +0,0 @@
// 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 "testutil/array_utils.h"
#include "common/status.h"
#include "exprs/anyval_util.h"
#include "gen_cpp/olap_file.pb.h"
#include "runtime/collection_value.h"
#include "runtime/free_pool.hpp"
#include "runtime/mem_pool.h"
#include "runtime/memory/mem_tracker.h"
#include "udf/udf_internal.h"
namespace doris {
using TypeDesc = FunctionContext::TypeDesc;
void ArrayUtils::prepare_context(FunctionContext& context, MemPool& mem_pool,
const ColumnPB& column_pb) {
auto function_type_desc = create_function_type_desc(column_pb);
context.impl()->_return_type = function_type_desc;
context.impl()->_pool = new FreePool(&mem_pool);
}
TypeDesc ArrayUtils::create_function_type_desc(const ColumnPB& column_pb) {
TypeDesc type_desc;
type_desc.len = column_pb.length();
type_desc.precision = column_pb.precision();
type_desc.scale = column_pb.frac();
if (column_pb.type() == "ARRAY") {
type_desc.type = FunctionContext::TYPE_ARRAY;
} else if (column_pb.type() == "BOOLEAN") {
type_desc.type = FunctionContext::TYPE_BOOLEAN;
} else if (column_pb.type() == "TINYINT") {
type_desc.type = FunctionContext::TYPE_TINYINT;
} else if (column_pb.type() == "SMALLINT") {
type_desc.type = FunctionContext::TYPE_SMALLINT;
} else if (column_pb.type() == "INT") {
type_desc.type = FunctionContext::TYPE_INT;
} else if (column_pb.type() == "BIGINT") {
type_desc.type = FunctionContext::TYPE_BIGINT;
} else if (column_pb.type() == "LARGEINT") {
type_desc.type = FunctionContext::TYPE_LARGEINT;
} else if (column_pb.type() == "FLOAT") {
type_desc.type = FunctionContext::TYPE_FLOAT;
} else if (column_pb.type() == "DOUBLE") {
type_desc.type = FunctionContext::TYPE_DOUBLE;
} else if (column_pb.type() == "CHAR") {
type_desc.type = FunctionContext::TYPE_CHAR;
} else if (column_pb.type() == "VARCHAR") {
type_desc.type = FunctionContext::TYPE_VARCHAR;
} else if (column_pb.type() == "STRING") {
type_desc.type = FunctionContext::TYPE_STRING;
} else if (column_pb.type() == "DATE") {
type_desc.type = FunctionContext::TYPE_DATE;
} else if (column_pb.type() == "DATETIME") {
type_desc.type = FunctionContext::TYPE_DATETIME;
} else if (column_pb.type().compare(0, 7, "DECIMAL") == 0) {
type_desc.type = FunctionContext::TYPE_DECIMALV2;
} else {
DCHECK(false) << "Failed to create function type descriptor.";
}
for (const auto& sub_column_pb : column_pb.children_columns()) {
type_desc.children.push_back(create_function_type_desc(sub_column_pb));
}
return type_desc;
}
} // namespace doris

View File

@ -1,41 +0,0 @@
// 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.
#pragma once
#include <string>
#include "udf/udf.h"
namespace doris {
class ColumnPB;
class MemPool;
class Status;
class CollectionValue;
class ArrayUtils {
public:
using TypeDesc = FunctionContext::TypeDesc;
static void prepare_context(FunctionContext& context, MemPool& mem_pool,
const ColumnPB& column_pb);
private:
static TypeDesc create_function_type_desc(const ColumnPB& column_pb);
};
} // namespace doris