diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 498f01db938..22f5da6af8e 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -3222,6 +3222,41 @@ Result: └─────────┘ ``` +## arrayNormalizedGini + +Calculates the normalized Gini coefficient. + +**Syntax** + +```sql +arrayNormalizedGini(predicted, label) +``` + +**Arguments** + +- `predicted` — Predicted values ([Array(T)](../data-types/array.md)) +- `label` — Actual values ([Array(T)](../data-types/array.md)) + +**Returned Value** + +- A tuple containing the Gini coefficients of the predicted values, the Gini coefficient of the normalized values, and the normalized Gini coefficient (= the ratio of the former two Gini coefficients). + +**Examples** + +Query: + +```sql +SELECT arrayNormalizedGini([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2]); +``` + +Result: + +``` +┌─arrayNormalizedGini([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2])──────────┐ +│ (0.18055555555555558,0.2638888888888889,0.6842105263157896) │ +└─────────────────────────────────────────────────────────────┘ +``` + ## Distance functions All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md). diff --git a/src/Functions/array/arrayNormalizedGini.cpp b/src/Functions/array/arrayNormalizedGini.cpp new file mode 100644 index 00000000000..4b0ef2070d1 --- /dev/null +++ b/src/Functions/array/arrayNormalizedGini.cpp @@ -0,0 +1,415 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +static constexpr size_t MAX_ARRAY_SIZE = 1 << 20; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; +} + +struct Impl +{ + template + static void vectorConst( + const PaddedPODArray & array_predicted_data, + const ColumnArray::Offsets & array_predicted_offsets, + const PaddedPODArray & array_labels_const, + PaddedPODArray & col_gini_predicted, + PaddedPODArray & col_gini_labels, + PaddedPODArray & col_gini_normalized) + { + size_t size = col_gini_predicted.size(); + size_t array_size = array_labels_const.size(); + + if (array_size > MAX_ARRAY_SIZE) + throw Exception( + ErrorCodes::TOO_LARGE_ARRAY_SIZE, + "Too large array size in function arrayNormalizedGini: {}, maximum: {}", + array_size, + MAX_ARRAY_SIZE); + + for (size_t i = 0; i < size; ++i) + { + size_t array1_size = array_predicted_offsets[i] - array_predicted_offsets[i - 1]; + if (array1_size != array_size) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "All arrays in function arrayNormalizedGini should have same size"); + + // Why we need to create a new array here every loop, because array2 will be sorted in calculateNormalizedGini. + PODArrayWithStackMemory array2(array_labels_const.begin(), array_labels_const.end()); + + auto [gini_predicted, gini_labels, gini_normalized] = calculateNormalizedGini(array_predicted_data, array_predicted_offsets[i - 1], array2, array_size); + + col_gini_predicted[i] = gini_predicted; + col_gini_labels[i] = gini_labels; + col_gini_normalized[i] = gini_normalized; + } + } + + template + static void vectorVector( + const PaddedPODArray & array_predicted_data, + const ColumnArray::Offsets & array_predicted_offsets, + const PaddedPODArray & array_labels_data, + const ColumnArray::Offsets & array_labels_offsets, + PaddedPODArray & col_gini_predicted, + PaddedPODArray & col_gini_labels, + PaddedPODArray & col_gini_normalized) + { + size_t size = col_gini_predicted.size(); + size_t array_size = size > 0 ? array_predicted_offsets[0] - array_predicted_offsets[-1] : 0; + + if (array_size > MAX_ARRAY_SIZE) + throw Exception( + ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in arrayNormalizedGini: {}, maximum: {}", array_size, MAX_ARRAY_SIZE); + + for (size_t i = 0; i < size; ++i) + { + size_t array1_size = array_predicted_offsets[i] - array_predicted_offsets[i - 1]; + size_t array2_size = array_labels_offsets[i] - array_labels_offsets[i - 1]; + if (array1_size != array_size || array2_size != array_size) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "All arrays in function arrayNormalizedGini should have same size"); + + PODArrayWithStackMemory array2(array_labels_data.data() + array_labels_offsets[i - 1], array_labels_data.data() + array_labels_offsets[i]); + + auto [gini_predicted, gini_labels, gini_normalized] = calculateNormalizedGini(array_predicted_data, array_predicted_offsets[i - 1], array2, array_size); + + col_gini_predicted[i] = gini_predicted; + col_gini_labels[i] = gini_labels; + col_gini_normalized[i] = gini_normalized; + } + } + + template + static void constVector( + const PaddedPODArray & array_predicted_const, + const PaddedPODArray & array_labels_data, + const ColumnArray::Offsets & array_labels_offsets, + PaddedPODArray & col_gini_predicted, + PaddedPODArray & col_gini_labels, + PaddedPODArray & col_gini_normalized) + { + size_t size = col_gini_predicted.size(); + size_t array_size = array_predicted_const.size(); + + if (array_size > MAX_ARRAY_SIZE) + throw Exception( + ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in arrayNormalizedGini: {}, maximum: {}", array_size, MAX_ARRAY_SIZE); + + for (size_t i = 0; i < size; ++i) + { + size_t array1_size = array_labels_offsets[i] - array_labels_offsets[i - 1]; + if (array1_size != array_size) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "All arrays in function arrayNormalizedGini should have same size"); + + PODArrayWithStackMemory array2(array_labels_data.data() + array_labels_offsets[i - 1], array_labels_data.data() + array_labels_offsets[i]); + + auto [gini_predicted, gini_labels, gini_normalized] = calculateNormalizedGini(array_predicted_const, 0, array2, array_size); + + col_gini_predicted[i] = gini_predicted; + col_gini_labels[i] = gini_labels; + col_gini_normalized[i] = gini_normalized; + } + } + +private: + template + static std::tuple calculateNormalizedGini( + const PaddedPODArray & array1, size_t offset, + PODArrayWithStackMemory & array2, size_t array_size) + { + auto sort_idx = sortIndexes(array1, offset, array_size); + + PODArrayWithStackMemory sorted_array2(array_size); + for (size_t i = 0; i < array_size; ++i) + sorted_array2[i] = array2[sort_idx[i]]; + + Float64 total_sum = std::accumulate(array2.begin(), array2.end(), 0.0); + + PODArrayWithStackMemory pred_cumsum_ratio(array_size); + + Float64 pred_cumsum = 0; + for (size_t i = 0; i < array_size; ++i) + { + pred_cumsum += sorted_array2[i] / total_sum; + pred_cumsum_ratio[i] = pred_cumsum; + } + + pdqsort(array2.begin(), array2.end()); + PODArrayWithStackMemory ltv_cumsum_ratio(array_size); + + Float64 ltv_cumsum = 0; + for (size_t i = 0; i < array_size; ++i) + { + ltv_cumsum += array2[i] / total_sum; + ltv_cumsum_ratio[i] = ltv_cumsum; + } + + Float64 random_gain_cumsum_ratio = 0.5 * (array_size + 1); + Float64 accumulate_pred_ratio = std::accumulate(pred_cumsum_ratio.begin(), pred_cumsum_ratio.end(), 0.0); + Float64 accumulate_ltv_ratio = std::accumulate(ltv_cumsum_ratio.begin(), ltv_cumsum_ratio.end(), 0.0); + + Float64 pred_gini = (random_gain_cumsum_ratio - accumulate_pred_ratio) / array_size; + Float64 gini_labels = (random_gain_cumsum_ratio - accumulate_ltv_ratio) / array_size; + + return std::make_tuple(pred_gini, gini_labels, pred_gini / gini_labels); + } + + template + static PODArrayWithStackMemory sortIndexes(const PaddedPODArray & array, size_t offset, size_t array_size) + { + PODArrayWithStackMemory idx(array_size); + std::iota(idx.begin(), idx.end(), 0); + pdqsort(idx.begin(), idx.end(), [&array, offset](size_t i1, size_t i2) { return array[i1 + offset] < array[i2 + offset]; }); + return idx; + } +}; + +/** + * Calculate the normalized Gini coefficient. See https://arxiv.org/pdf/1912.07753 + */ +class FunctionArrayNormalizedGini : public IFunction +{ +public: + static constexpr auto name = "arrayNormalizedGini"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & 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()); + + const DataTypeArray * arg1_type = checkAndGetDataType(arguments[0].get()); + if (arg1_type == nullptr || !isNumber(arg1_type->getNestedType())) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be an Array of numeric type, got {}", + getName(), + arguments[0]->getName()); + + const DataTypeArray * arg2_type = checkAndGetDataType(arguments[1].get()); + if (arg2_type == nullptr || !isNumber(arg2_type->getNestedType())) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument for function {} must be an Array of numeric typegot {}", + getName(), + arguments[1]->getName()); + + return std::make_shared( + DataTypes{std::make_shared(), std::make_shared(), std::make_shared()}); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto & col_predicted = arguments[0].column; + const auto & col_labels = arguments[1].column; + + Columns result(3); + for (size_t i = 0; i < 3; ++i) + result[i] = DataTypeFloat64().createColumn(); + + if (const ColumnArray * array_predicted = checkAndGetColumn(col_predicted.get())) + { + const auto & array_predicted_offsets = array_predicted->getOffsets(); + const auto & array_predicted_type = typeid_cast(arguments[0].type.get())->getNestedType(); + + if (const ColumnConst * array_labels_const = checkAndGetColumn(col_labels.get())) + { + const ColumnArray * column_array_const_internal_array = checkAndGetColumn(array_labels_const->getDataColumnPtr().get()); + const auto & array_labels_type = typeid_cast(arguments[1].type.get())->getNestedType(); + + if (castBothTypes( + array_predicted_type.get(), + array_labels_type.get(), + [&](const auto & type_predicted, const auto & type_labels) + { + using TypePredicted = typename std::decay_t::FieldType; + const ColumnVector * array_predicted_data = checkAndGetColumn>(array_predicted->getDataPtr().get()); + + using TypeLabels = typename std::decay_t::FieldType; + const ColumnVector * col_labels_data = checkAndGetColumn>(column_array_const_internal_array->getDataPtr().get()); + + auto col_gini_predicted = ColumnFloat64::create(input_rows_count); + auto col_gini_labels = ColumnFloat64::create(input_rows_count); + auto col_gini_normalized = ColumnFloat64::create(input_rows_count); + + Impl::vectorConst( + array_predicted_data->getData(), + array_predicted_offsets, + col_labels_data->getData(), + col_gini_predicted->getData(), + col_gini_labels->getData(), + col_gini_normalized->getData()); + + result[0] = std::move(col_gini_predicted); + result[1] = std::move(col_gini_labels); + result[2] = std::move(col_gini_normalized); + + return true; + })) + { + return ColumnTuple::create(result); + } + } + else + { + const ColumnArray * array_labels = checkAndGetColumn(col_labels.get()); + const auto & array_label_offsets = array_labels->getOffsets(); + const auto & array_labels_type = typeid_cast(arguments[1].type.get())->getNestedType(); + + if (castBothTypes( + array_predicted_type.get(), + array_labels_type.get(), + [&](const auto & type_predicted, const auto & type_labels) + { + using TypePredicted = typename std::decay_t::FieldType; + const ColumnVector * array_predicted_data = checkAndGetColumn>(array_predicted->getDataPtr().get()); + + using TypeLabels = typename std::decay_t::FieldType; + const ColumnVector * col_labels_data = checkAndGetColumn>(array_labels->getDataPtr().get()); + + auto col_gini_predicted = ColumnFloat64::create(input_rows_count); + auto col_gini_labels = ColumnFloat64::create(input_rows_count); + auto col_gini_normalized = ColumnFloat64::create(input_rows_count); + + Impl::vectorVector( + array_predicted_data->getData(), + array_predicted_offsets, + col_labels_data->getData(), + array_label_offsets, + col_gini_predicted->getData(), + col_gini_labels->getData(), + col_gini_normalized->getData()); + + result[0] = std::move(col_gini_predicted); + result[1] = std::move(col_gini_labels); + result[2] = std::move(col_gini_normalized); + + return true; + })) + { + return ColumnTuple::create(result); + } + } + } + else if (const ColumnConst * array_predicted_const = checkAndGetColumn(col_predicted.get())) + { + /// Note that const-const case is handled by useDefaultImplementationForConstants. + + const ColumnArray * column_array_const = checkAndGetColumn(array_predicted_const->getDataColumnPtr().get()); + const auto & array_predicted_type = typeid_cast(arguments[0].type.get())->getNestedType(); + + const ColumnArray * array_labels = checkAndGetColumn(col_labels.get()); + const auto & array_label_offsets = array_labels->getOffsets(); + const auto & array_labels_type = typeid_cast(arguments[1].type.get())->getNestedType(); + + if (castBothTypes( + array_predicted_type.get(), + array_labels_type.get(), + [&](const auto & type_predicted, const auto & type_labels) + { + using TypePredicted = typename std::decay_t::FieldType; + const ColumnVector * array_predicted_data = checkAndGetColumn>(column_array_const->getDataPtr().get()); + + using TypeLabels = typename std::decay_t::FieldType; + const ColumnVector * col_labels_data = checkAndGetColumn>(array_labels->getDataPtr().get()); + + auto col_gini_predicted = ColumnFloat64::create(input_rows_count); + auto col_gini_labels = ColumnFloat64::create(input_rows_count); + auto col_gini_normalized = ColumnFloat64::create(input_rows_count); + + Impl::constVector( + array_predicted_data->getData(), + col_labels_data->getData(), + array_label_offsets, + col_gini_predicted->getData(), + col_gini_labels->getData(), + col_gini_normalized->getData()); + + result[0] = std::move(col_gini_predicted); + result[1] = std::move(col_gini_labels); + result[2] = std::move(col_gini_normalized); + + return true; + })) + { + return ColumnTuple::create(result); + } + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column of argument of function {}", getName()); + } + +private: + template + static bool castType(const IDataType * type, F && f) + { + return castTypeToEither< + DataTypeInt8, + DataTypeInt16, + DataTypeInt32, + DataTypeInt64, + DataTypeUInt8, + DataTypeUInt16, + DataTypeUInt32, + DataTypeUInt64, + DataTypeFloat32, + DataTypeFloat64>(type, std::forward(f)); + } + + template + static bool castBothTypes(const IDataType * left, const IDataType * right, F && f) + { + return castType(left, [&](const auto & left_) + { + return castType(right, [&](const auto & right_) + { + return f(left_, right_); + }); + }); + } + +}; + +REGISTER_FUNCTION(NormalizedGini) +{ + FunctionDocumentation::Description doc_description = "Calculates the normalized Gini coefficient."; + FunctionDocumentation::Syntax doc_syntax = "arrayNormalizedGini(predicted, label)"; + FunctionDocumentation::Arguments doc_arguments = {{"predicted", "Predicted value (Array(T))."}, {"label", "Actual value (Array(T))."}}; + FunctionDocumentation::ReturnedValue doc_returned_value = "A tuple containing the Gini coefficients of the predicted values, the Gini coefficient of the normalized values, and the normalized Gini coefficient (= the ratio of the former two Gini coefficients)."; + FunctionDocumentation::Examples doc_examples + = {{"Example", + "SELECT arrayNormalizedGini([0.9, 0.3, 0.8, 0.7],[6, 1, 0, 2]);", + "(0.18055555555555558,0.2638888888888889,0.6842105263157896)"}}; + FunctionDocumentation::Categories doc_categories = {"Array"}; + + factory.registerFunction( + {doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::Case::Sensitive); +} + +} diff --git a/tests/queries/0_stateless/03279_array_normalized_gini.reference b/tests/queries/0_stateless/03279_array_normalized_gini.reference new file mode 100644 index 00000000000..0a1d4eb7fa5 --- /dev/null +++ b/tests/queries/0_stateless/03279_array_normalized_gini.reference @@ -0,0 +1,14 @@ +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18888888888888894,0.3,0.6296296296296299) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) +(0.18055555555555558,0.2638888888888889,0.6842105263157896) diff --git a/tests/queries/0_stateless/03279_array_normalized_gini.sql b/tests/queries/0_stateless/03279_array_normalized_gini.sql new file mode 100644 index 00000000000..e8e6554181b --- /dev/null +++ b/tests/queries/0_stateless/03279_array_normalized_gini.sql @@ -0,0 +1,24 @@ +SELECT arrayNormalizedGini([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2]); +SELECT arrayNormalizedGini([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2, 1]); -- { serverError ILLEGAL_COLUMN } + +SELECT arrayNormalizedGini([0.9, 0.3, 0.8, 0.75, 0.65, 0.6, 0.78, 0.7, 0.05, 0.4, 0.4, 0.05, 0.5, 0.1, 0.1], [1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0]); + +SELECT arrayNormalizedGini(arrayResize([1], 2000000), arrayResize([1], 2000000)); -- { serverError TOO_LARGE_ARRAY_SIZE } + +DROP TABLE IF EXISTS t; +CREATE TABLE t +( + `a1` Array(Float32), + `a2` Array(UInt32) +) +ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO t VALUES ([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2]), ([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2]), ([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2]), ([0.9, 0.3, 0.8, 0.7], [6, 1, 0, 2]); + +SELECT arrayNormalizedGini(a1, a2) FROM t; + +SELECT arrayNormalizedGini(a1, [6, 1, 0, 2]) FROM t; +SELECT arrayNormalizedGini([0.9, 0.3, 0.8, 0.7], a2) FROM t; + +DROP TABLE t; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 832aafbb3da..63ca9f0d8b0 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -50,6 +50,7 @@ Autocompletion AvroConfluent AzureQueue Azurite +arrayNormalizedGini BFloat BIGINT BIGSERIAL @@ -350,6 +351,7 @@ GetResolution GetUnidirectionalEdge GetUnidirectionalEdgeBoundary GetUnidirectionalEdgesFromHexagon +Gini GitLab GlobalThread GlobalThreadActive