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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 8 additions & 0 deletions be/src/vec/columns/column_const.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,14 @@ std::pair<const ColumnPtr&, bool> unpack_if_const(const ColumnPtr& ptr) noexcept
void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const,
const std::initializer_list<size_t>& parameters,
Block& block, const ColumnNumbers& arg_indexes) {
default_preprocess_parameter_columns(
columns, col_const, std::span<const size_t>(parameters.begin(), parameters.end()),
block, arg_indexes);
}

void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const,
const std::span<const size_t>& parameters, Block& block,
const ColumnNumbers& arg_indexes) {
if (std::all_of(parameters.begin(), parameters.end(),
[&](size_t const_index) -> bool { return col_const[const_index]; })) {
// only need to avoid expanding when all parameters are const
Expand Down
4 changes: 4 additions & 0 deletions be/src/vec/columns/column_const.h
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,10 @@ void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_co
const std::initializer_list<size_t>& parameters,
Block& block, const ColumnNumbers& arg_indexes);

void default_preprocess_parameter_columns(ColumnPtr* columns, const bool* col_const,
const std::span<const size_t>& parameters, Block& block,
const ColumnNumbers& arg_indexes);

/** ColumnConst contains another column with single element,
* but looks like a column with arbitrary amount of same elements.
*/
Expand Down
145 changes: 145 additions & 0 deletions be/src/vec/functions/function_interval.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,145 @@
// 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 <algorithm>
#include <cstddef>
#include <cstdint>
#include <limits>
#include <memory>
#include <vector>

#include "common/status.h"
#include "runtime/define_primitive_type.h"
#include "runtime/primitive_type.h"
#include "vec/columns/column.h"
#include "vec/columns/column_const.h"
#include "vec/columns/column_nullable.h"
#include "vec/columns/column_vector.h"
#include "vec/common/assert_cast.h"
#include "vec/core/block.h"
#include "vec/core/column_numbers.h"
#include "vec/core/column_with_type_and_name.h"
#include "vec/core/types.h"
#include "vec/data_types/data_type.h"
#include "vec/data_types/data_type_number.h"
#include "vec/functions/function.h"
#include "vec/functions/function_needs_to_handle_null.h"
#include "vec/functions/simple_function_factory.h"

namespace doris::vectorized {
class FunctionIntervalImpl {
public:
static constexpr auto name = "interval";
static size_t get_number_of_arguments() { return 0; }
static bool is_variadic() { return true; }
static DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) {
return std::make_shared<DataTypeInt32>();
}

static bool is_return_nullable(bool has_nullable,
const std::vector<ColumnWithConstAndNullMap>& cols_info) {
return false;
}

static bool execute_const_null(ColumnInt32::MutablePtr& res_col,
PaddedPODArray<UInt8>& res_null_map_data,
size_t input_rows_count, size_t null_index) {
if (null_index == 0) {
res_col->insert_many_vals(-1, input_rows_count);
return true;
}
return false;
}

static void collect_columns_info(std::vector<ColumnWithConstAndNullMap>& columns_info,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

利用这一套似乎并没有怎么简化代码。就自己手工处理下const和nullable就行了吧?

Block& block, const ColumnNumbers& arguments,
bool& has_nullable) {
const size_t arg_size = arguments.size();
std::vector<ColumnPtr> arg_col(arg_size);
std::vector<uint8_t> is_const(arg_size);
std::vector<size_t> param_idx(arg_size - 1);
bool all_const = true;
for (size_t i = 0; i < arg_size; ++i) {
is_const[i] = is_column_const(*block.get_by_position(arguments[i]).column);
if (i != 0) {
param_idx[i - 1] = i;
all_const = all_const & is_const[i];
}
}
arg_col[0] = is_const[0] ? assert_cast<const ColumnConst&>(
*block.get_by_position(arguments[0]).column)
.get_data_column_ptr()
: block.get_by_position(arguments[0]).column;
default_preprocess_parameter_columns(arg_col.data(),
reinterpret_cast<const bool*>(is_const.data()),
param_idx, block, arguments);

// The second parameter's `is_const` is used to indicate whether all the parameters are constants.
columns_info[0].is_const = is_const[0];
columns_info[1].is_const = all_const;
for (size_t i = 0; i < arguments.size(); ++i) {
ColumnPtr col_ptr = arg_col[i];
columns_info[i].type = block.get_by_position(arguments[i]).type;
columns_info[i].column = col_ptr;

if (const auto* nullable = check_and_get_column<ColumnNullable>(col_ptr.get())) {
has_nullable = true;
columns_info[i].nested_col = &nullable->get_nested_column();
columns_info[i].null_map = &nullable->get_null_map_data();
} else {
columns_info[i].nested_col = col_ptr.get();
}
}
}

static void execute(const std::vector<ColumnWithConstAndNullMap>& columns_info,
ColumnInt32::MutablePtr& res_col, PaddedPODArray<UInt8>& res_null_map_data,
size_t input_rows_count) {
res_col->resize(input_rows_count);
const auto& compare_data =
assert_cast<const ColumnInt64&>(*columns_info[0].nested_col).get_data();
const size_t num_thresholds = columns_info.size() - 1;

for (size_t row = 0; row < input_rows_count; ++row) {
if (columns_info[0].is_null_at(row)) {
res_col->get_data()[row] = -1;
continue;
}

// Determine whether all the threshold columns are constant columns.
size_t row_idx = columns_info[1].is_const ? 0 : row;
auto compare_val = compare_data[index_check_const(row, columns_info[0].is_const)];
std::vector<int64_t> thresholds;
thresholds.reserve(num_thresholds);

for (size_t i = 1; i < columns_info.size(); ++i) {
const auto& th_col = assert_cast<const ColumnInt64&>(*columns_info[i].nested_col);
thresholds.push_back(
columns_info[i].is_null_at(row_idx) ? 0 : th_col.get_data()[row_idx]);
}

auto it = std::upper_bound(thresholds.begin(), thresholds.end(), compare_val);
res_col->get_data()[row] = static_cast<Int32>(it - thresholds.begin());
}
}
};

void register_function_interval(SimpleFunctionFactory& factory) {
factory.register_function<FunctionNeedsToHandleNull<FunctionIntervalImpl, TYPE_INT>>();
}

} // namespace doris::vectorized
27 changes: 20 additions & 7 deletions be/src/vec/functions/function_needs_to_handle_null.h
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,17 @@
#pragma once
#include <boost/mpl/aux_/na_fwd.hpp>

#include "udf/udf.h"
#include "vec/functions/function.h"

namespace doris::vectorized {
#include "common/compile_check_begin.h"

// Helper struct to store information about const+nullable columns
struct ColumnWithConstAndNullMap {
ColumnPtr column = nullptr;
const IColumn* nested_col = nullptr;
DataTypePtr type = nullptr;
const NullMap* null_map = nullptr;
bool is_const = false;

Expand Down Expand Up @@ -80,8 +83,12 @@ class FunctionNeedsToHandleNull : public IFunction {
if (columns_info[i].is_const && columns_info[i].null_map &&
(*columns_info[i].null_map)[0] &&
execute_const_null(res_col, null_map_data, input_rows_count, i)) {
block.replace_by_position(
result, ColumnNullable::create(std::move(res_col), std::move(null_map)));
if (is_return_nullable(has_nullable, columns_info)) {
block.replace_by_position(result, ColumnNullable::create(std::move(res_col),
std::move(null_map)));
} else {
block.replace_by_position(result, std::move(res_col));
}
return Status::OK();
}
}
Expand Down Expand Up @@ -121,17 +128,23 @@ class FunctionNeedsToHandleNull : public IFunction {

// Collect the required information for each column into columns_info
// Including whether it is a constant column, nested column and null map(if exists).
void collect_columns_info(std::vector<ColumnWithConstAndNullMap>& columns_info,
const Block& block, const ColumnNumbers& arguments,
bool& has_nullable) const {
void collect_columns_info(std::vector<ColumnWithConstAndNullMap>& columns_info, Block& block,
const ColumnNumbers& arguments, bool& has_nullable) const {
if constexpr (requires {
Impl::collect_columns_info(columns_info, block, arguments, has_nullable);
}) {
Impl::collect_columns_info(columns_info, block, arguments, has_nullable);
return;
}
for (size_t i = 0; i < arguments.size(); ++i) {
ColumnPtr col_ptr;
const auto& col_with_type = block.get_by_position(arguments[i]);
std::tie(col_ptr, columns_info[i].is_const) = unpack_if_const(col_with_type.column);
columns_info[i].type = col_with_type.type;
columns_info[i].column = col_ptr;

if (is_column_nullable(*col_ptr)) {
if (const auto* nullable = check_and_get_column<ColumnNullable>(col_ptr.get())) {
has_nullable = true;
const auto* nullable = check_and_get_column<ColumnNullable>(col_ptr.get());
columns_info[i].nested_col = &nullable->get_nested_column();
columns_info[i].null_map = &nullable->get_null_map_data();
} else {
Expand Down
2 changes: 1 addition & 1 deletion be/src/vec/functions/function_string.h
Original file line number Diff line number Diff line change
Expand Up @@ -5096,7 +5096,7 @@ class FunctionExportSet : public IFunction {
bool col_const[5];
ColumnPtr arg_cols[5];
bool all_const = true;
for (int i = 0; i < arg_size; ++i) {
for (int i = 1; i < arg_size; ++i) {
col_const[i] = is_column_const(*block.get_by_position(arguments[i]).column);
all_const = all_const && col_const[i];
}
Expand Down
2 changes: 2 additions & 0 deletions be/src/vec/functions/simple_function_factory.h
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@ void register_function_geo(SimpleFunctionFactory& factory);
void register_function_multi_string_position(SimpleFunctionFactory& factory);
void register_function_multi_string_search(SimpleFunctionFactory& factory);
void register_function_width_bucket(SimpleFunctionFactory& factory);
void register_function_interval(SimpleFunctionFactory& factory);
void register_function_ignore(SimpleFunctionFactory& factory);
void register_function_encryption(SimpleFunctionFactory& factory);
void register_function_regexp_extract(SimpleFunctionFactory& factory);
Expand Down Expand Up @@ -325,6 +326,7 @@ class SimpleFunctionFactory {
register_function_multi_string_position(instance);
register_function_multi_string_search(instance);
register_function_width_bucket(instance);
register_function_interval(instance);
register_function_match(instance);
register_function_ip(instance);
register_function_tokenize(instance);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1690,6 +1690,7 @@ functionNameIdentifier
| CURRENT_USER
| DATABASE
| IF
| INTERVAL
| LEFT
| LIKE
| PASSWORD
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProductApproximate;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Interval;
import org.apache.doris.nereids.trees.expressions.functions.scalar.InttoUuid;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4CIDRToRange;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString;
Expand Down Expand Up @@ -783,6 +784,7 @@ public class BuiltinScalarFunctions implements FunctionHelper {
scalar(InnerProductApproximate.class, "inner_product_approximate"),
scalar(InnerProduct.class, "inner_product"),
scalar(Instr.class, "instr"),
scalar(Interval.class, "interval"),
scalar(InttoUuid.class, "int_to_uuid"),
scalar(Ipv4NumToString.class, "ipv4_num_to_string", "inet_ntoa"),
scalar(Ipv4StringToNum.class, "ipv4_string_to_num"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1102,4 +1102,34 @@ public static Expression boolor(BooleanLiteral first) {
public static Expression boolxor(BooleanLiteral first) {
return first;
}

/**
* interval
*/
@ExecFunction(name = "interval")
public static Expression interval(NullLiteral compareValue, Literal... thresholds) {
return new IntegerLiteral(-1);
}

/**
* interval
*/
@ExecFunction(name = "interval")
public static Expression interval(BigIntLiteral compareValue, BigIntLiteral... thresholds) {
long value = compareValue.getValue();
if (value < thresholds[0].getValue()) {
return new IntegerLiteral(0);
}
int l = 0;
int r = thresholds.length;
while (l < r) {
int mid = (l + r) >> 1;
if (thresholds[mid].getValue() <= value) {
l = mid + 1;
} else {
r = mid;
}
}
return new IntegerLiteral(l);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

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

import org.apache.doris.catalog.FunctionSignature;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.BigIntType;
import org.apache.doris.nereids.types.IntegerType;
import org.apache.doris.nereids.util.ExpressionUtils;

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

import java.util.List;

/**
* ScalarFunction 'interval'.
*/
public class Interval extends ScalarFunction implements ExplicitlyCastableSignature, AlwaysNotNullable {

public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.ret(IntegerType.INSTANCE).varArgs(BigIntType.INSTANCE, BigIntType.INSTANCE));

public Interval(Expression arg, Expression... varArgs) {
super("interval", ExpressionUtils.mergeArguments(arg, varArgs));
}

@Override
public Interval withChildren(List<Expression> children) {
Preconditions.checkArgument(children.size() >= 2);
return new Interval(children.get(0), children.subList(1, children.size()).toArray(new Expression[0]));
}

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

@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitInterval(this, context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -255,6 +255,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
import org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProductApproximate;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Interval;
import org.apache.doris.nereids.trees.expressions.functions.scalar.InttoUuid;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4CIDRToRange;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Ipv4NumToString;
Expand Down Expand Up @@ -1462,6 +1463,10 @@ default R visitInstr(Instr instr, C context) {
return visitScalarFunction(instr, context);
}

default R visitInterval(Interval interval, C context) {
return visitScalarFunction(interval, context);
}

default R visitIpv4NumToString(Ipv4NumToString ipv4NumToString, C context) {
return visitScalarFunction(ipv4NumToString, context);
}
Expand Down
Loading