Merge pull request #46790 from avoiderboi/add-width-bucket

Add `WIDTH_BUCKET` function
This commit is contained in:
Nikolai Kochetov 2023-03-01 18:35:36 +01:00 committed by GitHub
commit d469f138dc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 538 additions and 0 deletions

View File

@ -579,3 +579,33 @@ Result:
│ 3628800 │
└───────────────┘
```
## width_bucket(operand, low, high, count)
Returns the number of the bucket in which `operand` falls in a histogram having `count` equal-width buckets spanning the range `low` to `high`. Returns `0` if `operand < low`, and returns `count+1` if `operand >= high`.
`operand`, `low`, `high` can be any native number type. `count` can only be unsigned native integer and its value cannot be zero.
**Syntax**
```sql
widthBucket(operand, low, high, count)
```
There is also a case insensitive alias called `WIDTH_BUCKET` to provide compatibility with other databases.
**Example**
Query:
``` sql
SELECT widthBucket(10.15, -8.6, 23, 18);
```
Result:
``` text
┌─widthBucket(10.15, -8.6, 23, 18)─┐
│ 11 │
└──────────────────────────────────┘
```

View File

@ -0,0 +1,290 @@
#include <Columns/ColumnConst.h>
#include <Columns/ColumnVector.h>
#include <Core/ColumnWithTypeAndName.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Core/Types.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/NumberTraits.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
#include <Common/Exception.h>
#include <Common/NaNUtils.h>
#include <Common/register_objects.h>
#include <algorithm>
#include <iterator>
#include <memory>
#include <optional>
#include <string>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
class FunctionWidthBucket : public IFunction
{
template <typename TDataType>
void throwIfInvalid(
const size_t argument_index,
const ColumnConst * col_const,
const typename ColumnVector<TDataType>::Container * col_vec,
const size_t expected_size) const
{
if ((nullptr == col_const) ^ (nullptr != col_vec && col_vec->size() == expected_size))
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Logical error in function {}: argument {} has unexpected type or size!",
getName(),
argument_index);
}
}
template <typename TDataType>
const typename ColumnVector<TDataType>::Container * getDataIfNotNull(const ColumnVector<TDataType> * col_vec) const
{
if (nullptr == col_vec)
{
return nullptr;
}
return &col_vec->getData();
}
template <typename TDataType>
static TDataType
getValue(const ColumnConst * col_const, const typename ColumnVector<TDataType>::Container * col_vec, const size_t index)
{
if (nullptr != col_const)
{
return col_const->getValue<TDataType>();
}
return col_vec->data()[index];
}
static Float64 calculateRelativeBucket(const Float64 operand, const Float64 low, const Float64 high)
{
return (operand - low) / (high - low);
}
template <typename TResultType, typename TCountType>
std::optional<TResultType> checkArguments(const Float64 operand, const Float64 low, const Float64 high, const TCountType count) const
{
if (count == 0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Last argument (count) for function {} cannot be 0.", getName());
}
if (isNaN(operand) || isNaN(low) || isNaN(high))
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "The first three arguments (operand, low, high) cannot be NaN in function {}", getName());
}
// operand can be infinity, the following conditions will take care of it
if (!isFinite(low) || !isFinite(high))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The second and third arguments (low, high) cannot be Inf function {}", getName());
}
if (operand < low || low >= high)
{
return 0;
}
else if (operand >= high)
{
return count + 1;
}
return std::nullopt;
}
template <typename TResultType, typename TCountType>
TResultType NO_SANITIZE_UNDEFINED calculate(const Float64 operand, const Float64 low, const Float64 high, const TCountType count) const
{
if (const auto maybe_early_return = checkArguments<TResultType>(operand, low, high, count); maybe_early_return.has_value())
{
return *maybe_early_return;
}
const auto relative_bucket = calculateRelativeBucket(operand, low, high);
if (isNaN(relative_bucket) || !isFinite(relative_bucket))
{
throw Exception(
ErrorCodes::LOGICAL_ERROR, "The calculation resulted in NaN or Inf which is unexpected in function {}.", getName());
}
return static_cast<TResultType>(count * relative_bucket + 1);
}
template <is_any_of<UInt8, UInt16, UInt32, UInt64> TCountType>
ColumnPtr executeForResultType(const ColumnsWithTypeAndName & arguments, size_t input_rows_count) const
{
using ResultType = typename NumberTraits::Construct<false, false, NumberTraits::nextSize(sizeof(TCountType))>::Type;
auto common_type = std::make_shared<DataTypeNumber<Float64>>();
std::vector<ColumnPtr> casted_columns;
casted_columns.reserve(3);
for (const auto argument_index : collections::range(0, 3))
{
casted_columns.push_back(castColumn(arguments[argument_index], common_type));
}
const auto * operands_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(casted_columns[0].get()));
const auto * lows_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(casted_columns[1].get()));
const auto * highs_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(casted_columns[2].get()));
const auto * counts_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<TCountType>>(arguments[3].column.get()));
const auto * operands_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(casted_columns[0].get());
const auto * lows_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(casted_columns[1].get());
const auto * highs_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(casted_columns[2].get());
const auto * counts_col_const = checkAndGetColumnConst<ColumnVector<TCountType>>(arguments[3].column.get());
throwIfInvalid<Float64>(0, operands_col_const, operands_vec, input_rows_count);
throwIfInvalid<Float64>(1, lows_col_const, lows_vec, input_rows_count);
throwIfInvalid<Float64>(2, highs_col_const, highs_vec, input_rows_count);
throwIfInvalid<TCountType>(4, counts_col_const, counts_vec, input_rows_count);
const auto are_all_const_cols
= nullptr != operands_col_const && nullptr != lows_col_const && nullptr != highs_col_const && nullptr != counts_col_const;
if (are_all_const_cols)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Logical error in function {}: unexpected combination of argument types!", getName());
}
auto result_column = ColumnVector<ResultType>::create();
result_column->reserve(1);
auto & result_data = result_column->getData();
for (const auto row_index : collections::range(0, input_rows_count))
{
const auto operand = getValue<Float64>(operands_col_const, operands_vec, row_index);
const auto low = getValue<Float64>(lows_col_const, lows_vec, row_index);
const auto high = getValue<Float64>(highs_col_const, highs_vec, row_index);
const auto count = getValue<TCountType>(counts_col_const, counts_vec, row_index);
result_data.push_back(calculate<ResultType>(operand, low, high, count));
}
return result_column;
}
public:
static inline const char * name = "widthBucket";
explicit FunctionWidthBucket() = default;
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionWidthBucket>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 4; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
for (const auto argument_index : collections::range(0, 3))
{
if (!isNativeNumber(arguments[argument_index]))
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The first three arguments of function {} must be a Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float32 "
"or Float64.",
getName());
}
}
if (!WhichDataType(arguments[3]).isNativeUInt())
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The last argument of function {} must be UInt8, UInt16, UInt32 or UInt64, found {}.",
getName(),
arguments[3]->getName());
}
switch (arguments[3]->getTypeId())
{
case TypeIndex::UInt8:
return std::make_shared<DataTypeUInt16>();
case TypeIndex::UInt16:
return std::make_shared<DataTypeUInt32>();
case TypeIndex::UInt32:
[[fallthrough]];
case TypeIndex::UInt64:
return std::make_shared<DataTypeUInt64>();
default:
break;
}
UNREACHABLE();
}
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
ColumnPtr
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
{
switch (arguments[3].type->getTypeId())
{
case TypeIndex::UInt8:
return executeForResultType<UInt8>(arguments, input_rows_count);
case TypeIndex::UInt16:
return executeForResultType<UInt16>(arguments, input_rows_count);
case TypeIndex::UInt32:
return executeForResultType<UInt32>(arguments, input_rows_count);
case TypeIndex::UInt64:
return executeForResultType<UInt64>(arguments, input_rows_count);
default:
break;
}
UNREACHABLE();
}
bool useDefaultImplementationForConstants() const override { return true; }
};
REGISTER_FUNCTION(WidthBucket)
{
factory.registerFunction<FunctionWidthBucket>({
R"(
Returns the number of the bucket in which `operand` falls in a histogram having `count` equal-width buckets spanning the range `low` to `high`. Returns `0` if `operand < low`, and returns `count+1` if `operand >= high`.
`operand`, `low`, `high` can be any native number type. `count` can only be unsigned native integer and its value cannot be zero.
**Syntax**
```sql
widthBucket(operand, low, high, count)
```
There is also a case insensitive alias called `WIDTH_BUCKET` to provide compatibility with other databases.
**Example**
Query:
[example:simple]
Result:
``` text
widthBucket(10.15, -8.6, 23, 18)
11
```
)",
Documentation::Examples{
{"simple", "SELECT widthBucket(10.15, -8.6, 23, 18)"},
},
Documentation::Categories{"Mathematical"},
});
factory.registerAlias("width_bucket", "widthBucket", FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,156 @@
-7.6 -10 0 4 1
-6 -5 -1 2 0
0 0 10 4 1
1 3 0 1 0
3 -100 200 10 4
3 0 10 3 1
4.333 1 11 3 1
4.34 1 11 3 2
----------
0
11
----------
-7 -10 0 4 2
-6 -5 -1 2 0
0 0 10 4 1
1 3 0 1 0
3 -100 200 10 4
3 0 10 3 1
4 1 11 3 1
4 1 11 3 1
----------
249 65526 0 4 0
250 65531 4294967295 2 0
0 0 10 4 1
1 3 0 1 0
3 65436 200 10 0
3 0 10 3 1
4 1 11 3 1
4 1 11 3 1
----------
----------
UInt16
UInt32
UInt64
UInt64
----------
5
3
1
0
4
1
1
1
0
0
0
0
1
1
1
1
0
0
1
0
1
1
1
1
1
0
1
0
10
3
4
4
5
3
5
2
11
4
4
4
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
2
0
1
0
3
3
3
3
10
10
9
0
4
9
8
8
0
0
0
0
0
0
0
0
0
0
0
0
1
0
0
0
2
0
1
0
12
4
4
4
14
14
14
14
14
14
14
14
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0
0

View File

@ -0,0 +1,62 @@
CREATE TABLE mytable
(
operand Float64,
low Float64,
high Float64,
count UInt64,
PRIMARY KEY (operand, low, high, count)
) ENGINE = MergeTree();
INSERT INTO mytable VALUES (3, -100, 200, 10), (0, 0, 10, 4), (3, 0, 10, 3), (4.333, 1, 11, 3), (4.34, 1, 11, 3), (-7.6, -10, 0, 4), (-6, -5, -1, 2), (1, 3, 0, 1), (3, 2, 5, 0);
SELECT operand, low, high, count, WIDTH_BUCKET(operand, low, high, count) FROM mytable WHERE count != 0;
SELECT '----------';
-- zero is not valid for count
SELECT operand, low, high, count, WIDTH_BUCKET(operand, low, high, count) FROM mytable WHERE count = 0; -- { serverError BAD_ARGUMENTS }
-- operand, low and high cannot be NaN
SELECT WIDTH_BUCKET(0, 10, NaN, 10); -- { serverError BAD_ARGUMENTS }
SELECT WIDTH_BUCKET(NaN, 0, 10, 10); -- { serverError BAD_ARGUMENTS }
SELECT WIDTH_BUCKET(0, NaN, 10, 10); -- { serverError BAD_ARGUMENTS }
-- low and high cannot be Inf
SELECT WIDTH_BUCKET(1, -Inf, 10, 10); -- { serverError BAD_ARGUMENTS }
-- low and high cannot be Inf
SELECT WIDTH_BUCKET(1, 0, Inf, 10); -- { serverError BAD_ARGUMENTS }
-- operand can be Inf
SELECT WIDTH_BUCKET(-Inf, 0, 10, 10);
SELECT WIDTH_BUCKET(Inf, 0, 10, 10);
SELECT '----------';
-- IntXX types
SELECT toInt64(operand) AS operand, toInt32(low) AS low, toInt16(high) AS high, count, WIDTH_BUCKET(operand, low, high, count) FROM mytable WHERE count != 0;
SELECT '----------';
-- UIntXX types
SELECT toUInt8(toInt8(operand)) AS operand, toUInt16(toInt16(low)) AS low, toUInt32(toInt32(high)) AS high, count, WIDTH_BUCKET(operand, low, high, count) FROM mytable WHERE count != 0;
SELECT '----------';
SELECT WIDTH_BUCKET(1, 2, 3, -1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT WIDTH_BUCKET(1, 2, 3, 1.3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT WIDTH_BUCKET('a', 1, 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT WIDTH_BUCKET(1, toUInt128(42), 2, 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT WIDTH_BUCKET(1, 2, toInt128(42), 3); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT WIDTH_BUCKET(1, 2, 3, toInt256(42)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT '----------';
-- Return type checks
SELECT toTypeName(WIDTH_BUCKET(1, 2, 3, toUInt8(1)));
SELECT toTypeName(WIDTH_BUCKET(1, 2, 3, toUInt16(1)));
SELECT toTypeName(WIDTH_BUCKET(1, 2, 3, toUInt32(1)));
SELECT toTypeName(WIDTH_BUCKET(1, 2, 3, toUInt64(1)));
SELECT '----------';
-- Test handling ColumnConst
SELECT WIDTH_BUCKET(1, low, high, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, 2, high, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(3, 3, high, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, low, 4, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(5, low, 5, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, 6, 6, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(7, 7, 7, count) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, low, high, 8) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(9, low, high, 9) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, 10, high, 10) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(11, 11, high, 11) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, low, 12, 12) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(13, low, 13, 13) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(operand, 14, 14, 14) FROM mytable WHERE count != 0;
SELECT WIDTH_BUCKET(15, 15, 15, 15) FROM mytable WHERE count != 0;