Merge pull request #72950 from emmanuelsdias/add-performance-test-and-alias-to-pr-auc

Remove needless code duplication between `arrayROCAUC` and `arrrayAUCPR`
This commit is contained in:
Robert Schulze 2024-12-16 09:19:20 +00:00 committed by GitHub
commit 11ab8fe460
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
18 changed files with 344 additions and 471 deletions

View File

@ -1161,6 +1161,7 @@ argMin
argmax
argmin
arrayAUC
arrayAUCPr
arrayAll
arrayAvg
arrayCompact

View File

@ -2142,16 +2142,19 @@ Result:
```
## arrayAUC
## arrayROCAUC
Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: <https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve>).
Calculates the Area Under the Curve (AUC), which is a concept in machine learning.
For more details, please see [here](https://developers.google.com/machine-learning/glossary#pr-auc-area-under-the-pr-curve), [here](https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc#expandable-1) and [here](https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve).
**Syntax**
``` sql
arrayAUC(arr_scores, arr_labels[, scale])
arrayROCAUC(arr_scores, arr_labels[, scale])
```
Alias: `arrayAUC`
**Arguments**
- `arr_scores` — scores prediction model gives.
@ -2167,27 +2170,33 @@ Returns AUC value with type Float64.
Query:
``` sql
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
```
Result:
``` text
┌─arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐
┌─arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐
│ 0.75 │
└───────────────────────────────────────────────┘
└──────────────────────────────────────────────────
```
## arrayPrAUC
## arrayAUCPR
Calculate AUC (Area Under the Curve) for the Precision Recall curve.
Calculate the area under the precision-recall (PR) curve.
A precision-recall curve is created by plotting precision on the y-axis and recall on the x-axis across all thresholds.
The resulting value ranges from 0 to 1, with a higher value indicating better model performance.
PR AUC is particularly useful for imbalanced datasets, providing a clearer comparison of performance compared to ROC AUC on those cases.
For more details, please see [here](https://developers.google.com/machine-learning/glossary#pr-auc-area-under-the-pr-curve), [here](https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc#expandable-1) and [here](https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve).
**Syntax**
``` sql
arrayPrAUC(arr_scores, arr_labels)
arrayAUCPR(arr_scores, arr_labels)
```
Alias: `arrayPRAUC`
**Arguments**
- `arr_scores` — scores prediction model gives.
@ -2202,13 +2211,13 @@ Returns PR-AUC value with type Float64.
Query:
``` sql
select arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
select arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
```
Result:
``` text
┌─arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐
┌─arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐
│ 0.8333333333333333 │
└─────────────────────────────────────────────────┘
```

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
/** The function takes two arrays: scores and labels.
* Label can be one of two values: positive and negative.
* Label can be one of two values: positive (> 0) and negative (<= 0)
* Score can be arbitrary number.
*
* These values are considered as the output of classifier. We have some true labels for objects.
@ -33,6 +33,8 @@ namespace ErrorCodes
* or have false positive or false negative result.
* Verying the threshold we can get different probabilities of false positive or false negatives or true positives, etc...
*
* ---------------------------------------------------------------------------------------------------------------------
*
* We can also calculate the True Positive Rate and the False Positive Rate:
*
* TPR (also called "sensitivity", "recall" or "probability of detection")
@ -73,13 +75,53 @@ namespace ErrorCodes
* threshold = 0.8, TPR = 0, FPR = 0, TPR_raw = 0, FPR_raw = 0
*
* The "curve" will be present by a line that moves one step either towards right or top on each threshold change.
*
* ---------------------------------------------------------------------------------------------------------------------
*
* We can also calculate the Precision and the Recall ("PR"):
*
* Precision is the ratio `tp / (tp + fp)` where `tp` is the number of true positives and `fp` the number of false positives.
* It represents how often the classifier is correct when giving a positive result.
* Precision = P(label = positive | score > threshold)
*
* Recall is the ratio `tp / (tp + fn)` where `tp` is the number of true positives and `fn` the number of false negatives.
* It represents the probability of the classifier to give positive result if the object has positive label.
* Recall = P(score > threshold | label = positive)
*
* We can draw a curve of values of Precision and Recall with different threshold on [0..1] x [0..1] unit square.
* This curve is named "Precision Recall curve" (PR).
*
* For the curve we can calculate, literally, Area Under the Curve, that will be in the range of [0..1].
*
* Let's look at the example:
* arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
*
* 1. We have pairs: (-, 0.1), (-, 0.4), (+, 0.35), (+, 0.8)
*
* 2. Let's sort by score descending: (+, 0.8), (-, 0.4), (+, 0.35), (-, 0.1)
*
* 3. Let's draw the points:
*
* threshold = 0.8, TP = 0, FP = 0, FN = 2, Recall = 0.0, Precision = 1
* threshold = 0.4, TP = 1, FP = 0, FN = 1, Recall = 0.5, Precision = 1
* threshold = 0.35, TP = 1, FP = 1, FN = 1, Recall = 0.5, Precision = 0.5
* threshold = 0.1, TP = 2, FP = 1, FN = 0, Recall = 1.0, Precision = 0.666
* threshold = 0, TP = 2, FP = 2, FN = 0, Recall = 1.0, Precision = 0.5
*
* This implementation uses the right Riemann sum (see https://en.wikipedia.org/wiki/Riemann_sum) to calculate the AUC.
* That is, each increment in area is calculated using `(R_n - R_{n-1}) * P_n`,
* where `R_n` is the Recall at the `n`-th point and `P_n` is the Precision at the `n`-th point.
*
* This implementation is not interpolated and is different from computing the AUC with the trapezoidal rule,
* which uses linear interpolation and can be too optimistic for the Precision Recall AUC metric.
*/
template <bool PR>
class FunctionArrayAUC : public IFunction
{
public:
static constexpr auto name = "arrayAUC";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayAUC>(); }
static constexpr auto name = PR ? "arrayAUCPR" : "arrayROCAUC";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayAUC<PR>>(); }
private:
static Float64 apply(
@ -87,7 +129,7 @@ private:
const IColumn & labels,
ColumnArray::Offset current_offset,
ColumnArray::Offset next_offset,
bool scale)
[[maybe_unused]] bool scale)
{
struct ScoreLabel
{
@ -96,26 +138,32 @@ private:
};
size_t size = next_offset - current_offset;
if (PR && size == 0)
return 0.0;
PODArrayWithStackMemory<ScoreLabel, 1024> sorted_labels(size);
for (size_t i = 0; i < size; ++i)
{
bool label = labels.getFloat64(current_offset + i) > 0;
sorted_labels[i].label = labels.getFloat64(current_offset + i) > 0;
sorted_labels[i].score = scores.getFloat64(current_offset + i);
sorted_labels[i].label = label;
}
/// Sorting scores in descending order to traverse the ROC curve from left to right
/// Sorting scores in descending order to traverse the ROC / Precision-Recall curve from left to right
std::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; });
if constexpr (!PR)
{
/// We will first calculate non-normalized area.
Float64 area = 0.0;
Float64 prev_score = sorted_labels[0].score;
size_t prev_fp = 0;
size_t prev_tp = 0;
size_t curr_fp = 0;
size_t curr_tp = 0;
for (size_t i = 0; i < size; ++i)
{
/// Only increment the area when the score changes
@ -145,6 +193,55 @@ private:
}
return area;
}
else
{
Float64 area = 0.0;
Float64 prev_score = sorted_labels[0].score;
size_t prev_tp = 0;
size_t curr_tp = 0; /// True positives predictions (positive label and score > threshold)
size_t curr_p = 0; /// Total positive predictions (score > threshold)
Float64 curr_precision;
for (size_t i = 0; i < size; ++i)
{
if (sorted_labels[i].score != prev_score)
{
/* Precision = TP / (TP + FP)
* Recall = TP / (TP + FN)
*
* Instead of calculating
* d_Area = Precision_n * (Recall_n - Recall_{n-1}),
* we can just calculate
* d_Area = Precision_n * (TP_n - TP_{n-1})
* and later divide it by (TP + FN).
*
* This can be done because (TP + FN) is constant and equal to total positive labels.
*/
curr_precision = static_cast<Float64>(curr_tp) / curr_p; /// curr_p should never be 0 because this if statement isn't executed on the first iteration and the
/// following iterations will have already counted (curr_p += 1) at least one positive prediction
area += curr_precision * (curr_tp - prev_tp);
prev_tp = curr_tp;
prev_score = sorted_labels[i].score;
}
if (sorted_labels[i].label)
curr_tp += 1;
curr_p += 1;
}
/// If there were no positive labels, Recall did not change and the area is 0
if (curr_tp == 0)
return 0.0;
curr_precision = curr_p > 0 ? static_cast<Float64>(curr_tp) / curr_p : 1.0;
area += curr_precision * (curr_tp - prev_tp);
/// Finally, we divide by (TP + FN) to obtain the Recall
/// At this point we've traversed the whole curve and curr_tp = total positive labels (TP + FN)
return area / curr_tp;
}
}
static void vector(
const IColumn & scores,
@ -168,8 +265,8 @@ private:
public:
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool isVariadic() const override { return PR ? false : true; }
size_t getNumberOfArguments() const override { return PR ? 2 : 0; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; }
@ -177,10 +274,11 @@ public:
{
size_t number_of_arguments = arguments.size();
if (number_of_arguments < 2 || number_of_arguments > 3)
if ((!PR && (number_of_arguments < 2 || number_of_arguments > 3))
|| (PR && number_of_arguments != 2))
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
getName(), number_of_arguments);
"Number of arguments for function {} doesn't match: passed {}, should be {}",
getName(), number_of_arguments, PR ? "2" : "2 or 3");
for (size_t i = 0; i < 2; ++i)
{
@ -193,7 +291,7 @@ public:
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", getName(), nested_type->getName());
}
if (number_of_arguments == 3)
if (!PR && number_of_arguments == 3)
{
if (!isBool(arguments[2].type) || arguments[2].column.get() == nullptr || !isColumnConst(*arguments[2].column))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument (scale) for function {} must be of type const Bool.", getName());
@ -202,10 +300,7 @@ public:
return std::make_shared<DataTypeFloat64>();
}
DataTypePtr getReturnTypeForDefaultImplementationForDynamic() const override
{
return std::make_shared<DataTypeFloat64>();
}
DataTypePtr getReturnTypeForDefaultImplementationForDynamic() const override { return std::make_shared<DataTypeFloat64>(); }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
@ -249,7 +344,10 @@ public:
REGISTER_FUNCTION(ArrayAUC)
{
factory.registerFunction<FunctionArrayAUC>();
factory.registerFunction<FunctionArrayAUC<false>>();
factory.registerFunction<FunctionArrayAUC<true>>();
factory.registerAlias("arrayAUC", "arrayROCAUC"); /// Backward compatibility, also ROC AUC is often shorted to just AUC
factory.registerAlias("arrayPRAUC", "arrayAUCPR");
}
}

View File

@ -1,246 +0,0 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
/** The function takes two arrays: scores and labels.
* Label can be one of two values: positive (> 0) and negative (<= 0).
* Score can be arbitrary number.
*
* These values are considered as the output of classifier. We have some true labels for objects.
* And classifier assigns some scores to objects that predict these labels in the following way:
* - we can define arbitrary threshold on score and predict that the label is positive if the score is greater than the threshold:
*
* f(object) = score
* predicted_label = score > threshold
*
* This way classifier may predict positive or negative value correctly - true positive (tp) or true negative (tn)
* or have false positive (fp) or false negative (fn) result.
* Varying the threshold we can get different probabilities of false positive or false negatives or true positives, etc...
*
* We can also calculate the Precision and the Recall:
*
* Precision is the ratio `tp / (tp + fp)` where `tp` is the number of true positives and `fp` the number of false positives.
* It represents how often the classifier is correct when giving a positive result.
* Precision = P(label = positive | score > threshold)
*
* Recall is the ratio `tp / (tp + fn)` where `tp` is the number of true positives and `fn` the number of false negatives.
* It represents the probability of the classifier to give positive result if the object has positive label.
* Recall = P(score > threshold | label = positive)
*
* We can draw a curve of values of Precision and Recall with different threshold on [0..1] x [0..1] unit square.
* This curve is named "Precision Recall curve" (PR).
*
* For the curve we can calculate, literally, Area Under the Curve, that will be in the range of [0..1].
*
* Let's look at the example:
* arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
*
* 1. We have pairs: (-, 0.1), (-, 0.4), (+, 0.35), (+, 0.8)
*
* 2. Let's sort by score descending: (+, 0.8), (-, 0.4), (+, 0.35), (-, 0.1)
*
* 3. Let's draw the points:
*
* threshold = 0.8, TP = 0, FP = 0, FN = 2, Recall = 0.0, Precision = 1
* threshold = 0.4, TP = 1, FP = 0, FN = 1, Recall = 0.5, Precision = 1
* threshold = 0.35, TP = 1, FP = 1, FN = 1, Recall = 0.5, Precision = 0.5
* threshold = 0.1, TP = 2, FP = 1, FN = 0, Recall = 1.0, Precision = 0.666
* threshold = 0, TP = 2, FP = 2, FN = 0, Recall = 1.0, Precision = 0.5
*
* This implementation uses the right Riemann sum (see https://en.wikipedia.org/wiki/Riemann_sum) to calculate the AUC.
* That is, each increment in area is calculated using `(R_n - R_{n-1}) * P_n`,
* where `R_n` is the Recall at the `n`-th point and `P_n` is the Precision at the `n`-th point.
*
* This implementation is not interpolated and is different from computing the AUC with the trapezoidal rule,
* which uses linear interpolation and can be too optimistic for the Precision Recall AUC metric.
*/
class FunctionArrayPrAUC : public IFunction
{
public:
static constexpr auto name = "arrayPrAUC";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayPrAUC>(); }
private:
static Float64 apply(const IColumn & scores, const IColumn & labels, ColumnArray::Offset current_offset, ColumnArray::Offset next_offset)
{
size_t size = next_offset - current_offset;
if (size == 0)
return 0.0;
struct ScoreLabel
{
Float64 score;
bool label;
};
PODArrayWithStackMemory<ScoreLabel, 1024> sorted_labels(size);
for (size_t i = 0; i < size; ++i)
{
sorted_labels[i].label = labels.getFloat64(current_offset + i) > 0;
sorted_labels[i].score = scores.getFloat64(current_offset + i);
}
/// Sorting scores in descending order to traverse the Precision Recall curve from left to right
std::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; });
size_t prev_tp = 0;
size_t curr_tp = 0; /// True positives predictions (positive label and score > threshold)
size_t curr_p = 0; /// Total positive predictions (score > threshold)
Float64 prev_score = sorted_labels[0].score;
Float64 curr_precision;
Float64 area = 0.0;
for (size_t i = 0; i < size; ++i)
{
if (sorted_labels[i].score != prev_score)
{
/* Precision = TP / (TP + FP)
* Recall = TP / (TP + FN)
*
* Instead of calculating
* d_Area = Precision_n * (Recall_n - Recall_{n-1}),
* we can just calculate
* d_Area = Precision_n * (TP_n - TP_{n-1})
* and later divide it by (TP + FN).
*
* This can be done because (TP + FN) is constant and equal to total positive labels.
*/
curr_precision = static_cast<Float64>(curr_tp) / curr_p; /// curr_p should never be 0 because this if statement isn't executed on the first iteration and the
/// following iterations will have already counted (curr_p += 1) at least one positive prediction
area += curr_precision * (curr_tp - prev_tp);
prev_tp = curr_tp;
prev_score = sorted_labels[i].score;
}
if (sorted_labels[i].label)
curr_tp += 1;
curr_p += 1;
}
/// If there were no positive labels, Recall did not change and the area is 0
if (curr_tp == 0)
return 0.0;
curr_precision = curr_p > 0 ? static_cast<Float64>(curr_tp) / curr_p : 1.0;
area += curr_precision * (curr_tp - prev_tp);
/// Finally, we divide by (TP + FN) to obtain the Recall
/// At this point we've traversed the whole curve and curr_tp = total positive labels (TP + FN)
return area / curr_tp;
}
static void vector(
const IColumn & scores,
const IColumn & labels,
const ColumnArray::Offsets & offsets,
PaddedPODArray<Float64> & result,
size_t input_rows_count)
{
result.resize(input_rows_count);
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < input_rows_count; ++i)
{
auto next_offset = offsets[i];
result[i] = apply(scores, labels, current_offset, next_offset);
current_offset = next_offset;
}
}
public:
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 2; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 2.",
getName(),
arguments.size());
for (size_t i = 0; i < 2; ++i)
{
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[i].type.get());
if (!array_type)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Both arguments for function {} must be of type Array", getName());
const auto & nested_type = array_type->getNestedType();
/// The first argument (scores) must be an array of numbers
if (i == 0 && !isNativeNumber(nested_type))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} cannot process values of type {} in its first argument", getName(), nested_type->getName());
/// The second argument (labels) must be an array of numbers or enums
if (i == 1 && !isNativeNumber(nested_type) && !isEnum(nested_type))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} cannot process values of type {} in its second argument", getName(), nested_type->getName());
}
return std::make_shared<DataTypeFloat64>();
}
DataTypePtr getReturnTypeForDefaultImplementationForDynamic() const override { return std::make_shared<DataTypeFloat64>(); }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
{
ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst();
ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst();
const ColumnArray * col_array1 = checkAndGetColumn<ColumnArray>(col1.get());
if (!col_array1)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of first argument of function {}, should be an Array",
arguments[0].column->getName(),
getName());
const ColumnArray * col_array2 = checkAndGetColumn<ColumnArray>(col2.get());
if (!col_array2)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of second argument of function {}, should be an Array",
arguments[1].column->getName(),
getName());
if (!col_array1->hasEqualOffsets(*col_array2))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName());
auto col_res = ColumnVector<Float64>::create();
vector(col_array1->getData(), col_array2->getData(), col_array1->getOffsets(), col_res->getData(), input_rows_count);
return col_res;
}
};
REGISTER_FUNCTION(ArrayPrAUC)
{
factory.registerFunction<FunctionArrayPrAUC>();
}
}

View File

@ -0,0 +1,3 @@
<test>
<query>SELECT avg(ifNotFinite(arrayAUCPR(arrayMap(x -> rand(x) / 0x100000000, range(2 + rand() % 100)), arrayMap(x -> rand(x) % 2, range(2 + rand() % 100))), 0)) FROM numbers(100000)</query>
</test>

View File

@ -0,0 +1,4 @@
<test>
<query>SELECT avg(ifNotFinite(arrayROCAUC(arrayMap(x -> rand(x) / 0x100000000, range(2 + rand() % 100)), arrayMap(x -> rand(x) % 2, range(2 + rand() % 100))), 0)) FROM numbers(100000)</query>
</test>

View File

@ -1,4 +0,0 @@
<test>
<query>SELECT avg(ifNotFinite(arrayAUC(arrayMap(x -> rand(x) / 0x100000000, range(2 + rand() % 100)), arrayMap(x -> rand(x) % 2, range(2 + rand() % 100))), 0)) FROM numbers(100000)</query>
</test>

View File

@ -0,0 +1,59 @@
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)));
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)));
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))));
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))));
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]);
select arrayROCAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]);
select arrayROCAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]);
select arrayROCAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), true);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), true);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), true);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), true);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], true);
select arrayROCAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], true);
select arrayROCAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], true);
select arrayROCAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], true);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), false);
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), false);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], false);
select arrayROCAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], false);
select arrayROCAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], false);
select arrayROCAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], false);
-- negative tests
select arrayROCAUC([0, 0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
select arrayROCAUC([0.1, 0.35], [0, 0, 1, 1]); -- { serverError BAD_ARGUMENTS }
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], materialize(true)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select arrayROCAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true, true); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
-- alias
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false);

View File

@ -1,56 +0,0 @@
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)));
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)));
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))));
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))));
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]);
select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]);
select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]);
select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]);
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), true);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), true);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), true);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), true);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], true);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], true);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], true);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], true);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], true);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], true);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], true);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], true);
select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], true);
select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], true);
select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], true);
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), false);
select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), false);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], false);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], false);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], false);
select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], false);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], false);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], false);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], false);
select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], false);
select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], false);
select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], false);
select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], false);
-- negative tests
select arrayAUC([0, 0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
select arrayAUC([0.1, 0.35], [0, 0, 1, 1]); -- { serverError BAD_ARGUMENTS }
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], materialize(true)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true, true); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }

View File

@ -0,0 +1,46 @@
SELECT arrayROCAUC([], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([1], [1]);
SELECT arrayROCAUC([1], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([], [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([1, 2], [3]); -- { serverError BAD_ARGUMENTS }
SELECT arrayROCAUC([1], [2, 3]); -- { serverError BAD_ARGUMENTS }
SELECT arrayROCAUC([1, 1], [1, 1]);
SELECT arrayROCAUC([1, 1], [0, 0]);
SELECT arrayROCAUC([1, 1], [0, 1]);
SELECT arrayROCAUC([0, 1], [0, 1]);
SELECT arrayROCAUC([1, 0], [0, 1]);
SELECT arrayROCAUC([0, 0, 1], [0, 1, 1]);
SELECT arrayROCAUC([0, 1, 1], [0, 1, 1]);
SELECT arrayROCAUC([0, 1, 1], [0, 0, 1]);
SELECT arrayROCAUC([], [], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([1], [1], true);
SELECT arrayROCAUC([1], [], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([], [1], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([1, 2], [3], true); -- { serverError BAD_ARGUMENTS }
SELECT arrayROCAUC([1], [2, 3], true); -- { serverError BAD_ARGUMENTS }
SELECT arrayROCAUC([1, 1], [1, 1], true);
SELECT arrayROCAUC([1, 1], [0, 0], true);
SELECT arrayROCAUC([1, 1], [0, 1], true);
SELECT arrayROCAUC([0, 1], [0, 1], true);
SELECT arrayROCAUC([1, 0], [0, 1], true);
SELECT arrayROCAUC([0, 0, 1], [0, 1, 1], true);
SELECT arrayROCAUC([0, 1, 1], [0, 1, 1], true);
SELECT arrayROCAUC([0, 1, 1], [0, 0, 1], true);
SELECT arrayROCAUC([], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([1], [1], false);
SELECT arrayROCAUC([1], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([], [1], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([1, 2], [3], false); -- { serverError BAD_ARGUMENTS }
SELECT arrayROCAUC([1], [2, 3], false); -- { serverError BAD_ARGUMENTS }
SELECT arrayROCAUC([1, 1], [1, 1], false);
SELECT arrayROCAUC([1, 1], [0, 0], false);
SELECT arrayROCAUC([1, 1], [0, 1], false);
SELECT arrayROCAUC([0, 1], [0, 1], false);
SELECT arrayROCAUC([1, 0], [0, 1], false);
SELECT arrayROCAUC([0, 0, 1], [0, 1, 1], false);
SELECT arrayROCAUC([0, 1, 1], [0, 1, 1], false);
SELECT arrayROCAUC([0, 1, 1], [0, 0, 1], false);
SELECT arrayROCAUC([0, 1, 1], [0, 0, 1], false, true); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayROCAUC([0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayROCAUC([0, 1, 1], [0, 0, 1], 'false'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayROCAUC([0, 1, 1], [0, 0, 1], 4); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

View File

@ -1,46 +0,0 @@
SELECT arrayAUC([], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([1], [1]);
SELECT arrayAUC([1], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([], [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([1, 2], [3]); -- { serverError BAD_ARGUMENTS }
SELECT arrayAUC([1], [2, 3]); -- { serverError BAD_ARGUMENTS }
SELECT arrayAUC([1, 1], [1, 1]);
SELECT arrayAUC([1, 1], [0, 0]);
SELECT arrayAUC([1, 1], [0, 1]);
SELECT arrayAUC([0, 1], [0, 1]);
SELECT arrayAUC([1, 0], [0, 1]);
SELECT arrayAUC([0, 0, 1], [0, 1, 1]);
SELECT arrayAUC([0, 1, 1], [0, 1, 1]);
SELECT arrayAUC([0, 1, 1], [0, 0, 1]);
SELECT arrayAUC([], [], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([1], [1], true);
SELECT arrayAUC([1], [], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([], [1], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([1, 2], [3], true); -- { serverError BAD_ARGUMENTS }
SELECT arrayAUC([1], [2, 3], true); -- { serverError BAD_ARGUMENTS }
SELECT arrayAUC([1, 1], [1, 1], true);
SELECT arrayAUC([1, 1], [0, 0], true);
SELECT arrayAUC([1, 1], [0, 1], true);
SELECT arrayAUC([0, 1], [0, 1], true);
SELECT arrayAUC([1, 0], [0, 1], true);
SELECT arrayAUC([0, 0, 1], [0, 1, 1], true);
SELECT arrayAUC([0, 1, 1], [0, 1, 1], true);
SELECT arrayAUC([0, 1, 1], [0, 0, 1], true);
SELECT arrayAUC([], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([1], [1], false);
SELECT arrayAUC([1], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([], [1], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([1, 2], [3], false); -- { serverError BAD_ARGUMENTS }
SELECT arrayAUC([1], [2, 3], false); -- { serverError BAD_ARGUMENTS }
SELECT arrayAUC([1, 1], [1, 1], false);
SELECT arrayAUC([1, 1], [0, 0], false);
SELECT arrayAUC([1, 1], [0, 1], false);
SELECT arrayAUC([0, 1], [0, 1], false);
SELECT arrayAUC([1, 0], [0, 1], false);
SELECT arrayAUC([0, 0, 1], [0, 1, 1], false);
SELECT arrayAUC([0, 1, 1], [0, 1, 1], false);
SELECT arrayAUC([0, 1, 1], [0, 0, 1], false);
SELECT arrayAUC([0, 1, 1], [0, 0, 1], false, true); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayAUC([0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayAUC([0, 1, 1], [0, 0, 1], 'false'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayAUC([0, 1, 1], [0, 0, 1], 4); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

View File

@ -90,7 +90,7 @@ alphaTokens
and
appendTrailingCharIfAbsent
array
arrayAUC
arrayAUCPR
arrayAll
arrayAvg
arrayCompact
@ -124,10 +124,10 @@ arrayMax
arrayMin
arrayPopBack
arrayPopFront
arrayPrAUC
arrayProduct
arrayPushBack
arrayPushFront
arrayROCAUC
arrayRandomSample
arrayReduce
arrayReduceInRanges

View File

@ -0,0 +1,51 @@
-- type correctness tests
select floor(arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]), 10);
select floor(arrayAUCPR([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8))), 10);
select floor(arrayAUCPR([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8))), 10);
select floor(arrayAUCPR([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1)))), 10);
select floor(arrayAUCPR([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1)))), 10);
select floor(arrayAUCPR(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]), 10);
select floor(arrayAUCPR(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]), 10);
-- output value correctness test
select floor(arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]), 10);
select floor(arrayAUCPR([0.1, 0.4, 0.4, 0.35, 0.8], [0, 0, 1, 1, 1]), 10);
select floor(arrayAUCPR([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]), 10);
select floor(arrayAUCPR([0.1, 0.35, 0.4, 0.4, 0.8], [1, 0, 1, 0, 0]), 10);
select floor(arrayAUCPR([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]), 10);
select floor(arrayAUCPR([0, 1, 2, 3, 4, 5, 6, 7, 8, 9], [1, 0, 1, 0, 0, 0, 1, 0, 0, 1]), 10);
select floor(arrayAUCPR([0, 1, 1, 2, 2, 2, 3, 3, 3, 3], [1, 0, 1, 0, 0, 0, 1, 0, 0, 1]), 10);
-- edge cases
SELECT floor(arrayAUCPR([1], [1]), 10);
SELECT floor(arrayAUCPR([1], [0]), 10);
SELECT floor(arrayAUCPR([0], [0]), 10);
SELECT floor(arrayAUCPR([0], [1]), 10);
SELECT floor(arrayAUCPR([1, 1], [1, 1]), 10);
SELECT floor(arrayAUCPR([1, 1], [0, 0]), 10);
SELECT floor(arrayAUCPR([1, 1], [0, 1]), 10);
SELECT floor(arrayAUCPR([0, 1], [0, 1]), 10);
SELECT floor(arrayAUCPR([1, 0], [0, 1]), 10);
SELECT floor(arrayAUCPR([0, 0, 1], [0, 1, 1]), 10);
SELECT floor(arrayAUCPR([0, 1, 1], [0, 1, 1]), 10);
SELECT floor(arrayAUCPR([0, 1, 1], [0, 0, 1]), 10);
-- negative tests
select arrayAUCPR([], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select arrayAUCPR([0, 0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
select arrayAUCPR([0.1, 0.35], [0, 0, 1, 1]); -- { serverError BAD_ARGUMENTS }
select arrayAUCPR([0.1, 0.4, 0.35, 0.8], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], [1, 1, 0, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
select arrayAUCPR(['a', 'b', 'c', 'd'], [1, 0, 1, 1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select arrayAUCPR([0.1, 0.4, NULL, 0.8], [0, 0, 1, 1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
select arrayAUCPR([0.1, 0.4, 0.35, 0.8], [0, NULL, 1, 1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
--alias
SELECT floor(arrayPRAUC([1], [1]), 10);

View File

@ -1,49 +0,0 @@
-- type correctness tests
select floor(arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]), 10);
select floor(arrayPrAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8))), 10);
select floor(arrayPrAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8))), 10);
select floor(arrayPrAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1)))), 10);
select floor(arrayPrAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1)))), 10);
select floor(arrayPrAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]), 10);
select floor(arrayPrAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]), 10);
-- output value correctness test
select floor(arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]), 10);
select floor(arrayPrAUC([0.1, 0.4, 0.4, 0.35, 0.8], [0, 0, 1, 1, 1]), 10);
select floor(arrayPrAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]), 10);
select floor(arrayPrAUC([0.1, 0.35, 0.4, 0.4, 0.8], [1, 0, 1, 0, 0]), 10);
select floor(arrayPrAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]), 10);
select floor(arrayPrAUC([0, 1, 2, 3, 4, 5, 6, 7, 8, 9], [1, 0, 1, 0, 0, 0, 1, 0, 0, 1]), 10);
select floor(arrayPrAUC([0, 1, 1, 2, 2, 2, 3, 3, 3, 3], [1, 0, 1, 0, 0, 0, 1, 0, 0, 1]), 10);
-- edge cases
SELECT floor(arrayPrAUC([1], [1]), 10);
SELECT floor(arrayPrAUC([1], [0]), 10);
SELECT floor(arrayPrAUC([0], [0]), 10);
SELECT floor(arrayPrAUC([0], [1]), 10);
SELECT floor(arrayPrAUC([1, 1], [1, 1]), 10);
SELECT floor(arrayPrAUC([1, 1], [0, 0]), 10);
SELECT floor(arrayPrAUC([1, 1], [0, 1]), 10);
SELECT floor(arrayPrAUC([0, 1], [0, 1]), 10);
SELECT floor(arrayPrAUC([1, 0], [0, 1]), 10);
SELECT floor(arrayPrAUC([0, 0, 1], [0, 1, 1]), 10);
SELECT floor(arrayPrAUC([0, 1, 1], [0, 1, 1]), 10);
SELECT floor(arrayPrAUC([0, 1, 1], [0, 0, 1]), 10);
-- negative tests
select arrayPrAUC([], []); -- { serverError BAD_ARGUMENTS }
select arrayPrAUC([0, 0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
select arrayPrAUC([0.1, 0.35], [0, 0, 1, 1]); -- { serverError BAD_ARGUMENTS }
select arrayPrAUC([0.1, 0.4, 0.35, 0.8], []); -- { serverError BAD_ARGUMENTS }
select arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], [1, 1, 0, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
select arrayPrAUC(cast(['false', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), [1, 0]); -- { serverError BAD_ARGUMENTS }
select arrayPrAUC(['a', 'b', 'c', 'd'], [1, 0, 1, 1]); -- { serverError BAD_ARGUMENTS }
select arrayPrAUC([0.1, 0.4, NULL, 0.8], [0, 0, 1, 1]); -- { serverError BAD_ARGUMENTS }
select arrayPrAUC([0.1, 0.4, 0.35, 0.8], [0, NULL, 1, 1]); -- { serverError BAD_ARGUMENTS }

View File

@ -49,6 +49,7 @@ AutoML
Autocompletion
AvroConfluent
AzureQueue
BFloat
BIGINT
BIGSERIAL
BORO
@ -244,10 +245,8 @@ Deduplication
DefaultTableEngine
DelayedInserts
DeliveryTag
Deltalake
DeltaLake
deltalakeCluster
deltaLakeCluster
Deltalake
Denormalize
DestroyAggregatesThreads
DestroyAggregatesThreadsActive
@ -380,15 +379,11 @@ Homebrew's
HorizontalDivide
Hostname
HouseOps
hudi
Hudi
hudiCluster
HudiCluster
HyperLogLog
Hypot
IANA
icebergCluster
IcebergCluster
IDE
IDEs
IDNA
@ -409,6 +404,7 @@ IPTrie
IProcessor
IPv
ITION
IcebergCluster
Identifiant
IdentifierQuotingRule
IdentifierQuotingStyle
@ -1233,6 +1229,7 @@ argMin
argmax
argmin
arrayAUC
arrayAUCPR
arrayAll
arrayAvg
arrayCompact
@ -1272,10 +1269,10 @@ arrayPartialShuffle
arrayPartialSort
arrayPopBack
arrayPopFront
arrayPrAUC
arrayProduct
arrayPushBack
arrayPushFront
arrayROCAUC
arrayRandomSample
arrayReduce
arrayReduceInRanges
@ -1617,9 +1614,11 @@ defaultValueOfArgumentType
defaultValueOfTypeName
delim
deltaLake
deltaLakeCluster
deltaSum
deltaSumTimestamp
deltalake
deltalakeCluster
deltasum
deltasumtimestamp
demangle
@ -1939,10 +1938,13 @@ html
http
https
hudi
hudi
hudiCluster
hyperscan
hypot
hyvor
iTerm
icebergCluster
icosahedron
icudata
idempotency
@ -3167,4 +3169,3 @@ znode
znodes
zookeeperSessionUptime
zstd
BFloat