mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
small refactor
This commit is contained in:
parent
532cd4beaa
commit
9afe579225
@ -1,11 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/StatSample.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/PODArray_fwd.h>
|
||||
#include <common/types.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
@ -21,40 +23,23 @@
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
#include <iostream>
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <template <typename> class Comparator>
|
||||
struct ComparePairFirst final
|
||||
{
|
||||
template <typename X, typename Y>
|
||||
bool operator()(const std::pair<X, Y> & lhs, const std::pair<X, Y> & rhs) const
|
||||
{
|
||||
return Comparator<X>{}(lhs.first, rhs.first);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <template <typename> class Comparator>
|
||||
struct ComparePairSecond final
|
||||
{
|
||||
template <typename X, typename Y>
|
||||
bool operator()(const std::pair<X, Y> & lhs, const std::pair<X, Y> & rhs) const
|
||||
{
|
||||
return Comparator<Y>{}(lhs.second, rhs.second);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename X = Float64, typename Y = Float64>
|
||||
struct AggregateFunctionRankCorrelationData final
|
||||
{
|
||||
size_t size_x = 0;
|
||||
|
||||
using Allocator = MixedAlignedArenaAllocator<alignof(std::pair<X, Y>), 4096>;
|
||||
using Array = PODArray<std::pair<X, Y>, 32, Allocator>;
|
||||
using AllocatorFirstSample = MixedAlignedArenaAllocator<alignof(X), 4096>;
|
||||
using FirstSample = PODArray<X, 32, AllocatorFirstSample>;
|
||||
|
||||
Array values;
|
||||
using AllocatorSecondSample = MixedAlignedArenaAllocator<alignof(Y), 4096>;
|
||||
using SecondSample = PODArray<Y, 32, AllocatorSecondSample>;
|
||||
|
||||
FirstSample first;
|
||||
SecondSample second;
|
||||
};
|
||||
|
||||
template <typename X, typename Y>
|
||||
@ -62,8 +47,8 @@ class AggregateFunctionRankCorrelation :
|
||||
public IAggregateFunctionDataHelper<AggregateFunctionRankCorrelationData<X, Y>, AggregateFunctionRankCorrelation<X, Y>>
|
||||
{
|
||||
using Data = AggregateFunctionRankCorrelationData<X, Y>;
|
||||
using Allocator = MixedAlignedArenaAllocator<alignof(std::pair<Float64, Float64>), 4096>;
|
||||
using Array = PODArray<std::pair<Float64, Float64>, 32, Allocator>;
|
||||
using FirstSample = typename Data::FirstSample;
|
||||
using SecondSample = typename Data::SecondSample;
|
||||
|
||||
public:
|
||||
explicit AggregateFunctionRankCorrelation(const DataTypes & arguments)
|
||||
@ -83,7 +68,8 @@ public:
|
||||
void insert(Data & a, const std::pair<X, Y> & x, Arena * arena) const
|
||||
{
|
||||
++a.size_x;
|
||||
a.values.push_back(x, arena);
|
||||
a.first.push_back(x.first, arena);
|
||||
a.second.push_back(x.second, arena);
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
|
||||
@ -93,11 +79,10 @@ public:
|
||||
auto new_x = assert_cast<const ColumnVector<X> &>(*columns[0]).getData()[row_num];
|
||||
auto new_y = assert_cast<const ColumnVector<Y> &>(*columns[1]).getData()[row_num];
|
||||
|
||||
auto new_arg = std::make_pair(new_x, new_y);
|
||||
|
||||
a.size_x += 1;
|
||||
|
||||
a.values.push_back(new_arg, arena);
|
||||
a.first.push_back(new_x, arena);
|
||||
a.second.push_back(new_y, arena);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
|
||||
@ -107,15 +92,17 @@ public:
|
||||
|
||||
if (b.size_x)
|
||||
for (size_t i = 0; i < b.size_x; ++i)
|
||||
insert(a, b.values[i], arena);
|
||||
insert(a, std::make_pair(b.first[i], b.second[i]), arena);
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
const auto & value = this->data(place).values;
|
||||
const auto & first = this->data(place).first;
|
||||
const auto & second = this->data(place).second;
|
||||
size_t size = this->data(place).size_x;
|
||||
writeVarUInt(size, buf);
|
||||
buf.write(reinterpret_cast<const char *>(value.data()), size * sizeof(value[0]));
|
||||
buf.write(reinterpret_cast<const char *>(first.data()), size * sizeof(first[0]));
|
||||
buf.write(reinterpret_cast<const char *>(second.data()), size * sizeof(second[0]));
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
|
||||
@ -123,94 +110,38 @@ public:
|
||||
size_t size = 0;
|
||||
readVarUInt(size, buf);
|
||||
|
||||
auto & value = this->data(place).values;
|
||||
auto & first = this->data(place).first;
|
||||
|
||||
value.resize(size, arena);
|
||||
buf.read(reinterpret_cast<char *>(value.data()), size * sizeof(value[0]));
|
||||
first.resize(size, arena);
|
||||
buf.read(reinterpret_cast<char *>(first.data()), size * sizeof(first[0]));
|
||||
|
||||
auto & second = this->data(place).second;
|
||||
|
||||
second.resize(size, arena);
|
||||
buf.read(reinterpret_cast<char *>(second.data()), size * sizeof(second[0]));
|
||||
}
|
||||
|
||||
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * /*arena*/) const override
|
||||
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override
|
||||
{
|
||||
const auto & value = this->data(place).values;
|
||||
/// Because ranks are adjusted, we have to store each of them in Float type.
|
||||
using RanksArray = PODArrayWithStackMemory<Float64, 32>;
|
||||
|
||||
const auto & first = this->data(place).first;
|
||||
const auto & second = this->data(place).second;
|
||||
size_t size = this->data(place).size_x;
|
||||
|
||||
// create a copy of values not to format data
|
||||
PODArrayWithStackMemory<std::pair<Float64, Float64>, 32> tmp_values;
|
||||
tmp_values.resize(size);
|
||||
for (size_t j = 0; j < size; ++ j)
|
||||
tmp_values[j] = static_cast<std::pair<Float64, Float64>>(value[j]);
|
||||
RanksArray first_ranks;
|
||||
first_ranks.resize(first.size());
|
||||
computeRanks<FirstSample, RanksArray>(first, first_ranks);
|
||||
|
||||
// sort x_values
|
||||
std::sort(std::begin(tmp_values), std::end(tmp_values), ComparePairFirst<std::greater>{});
|
||||
|
||||
for (size_t j = 0; j < size;)
|
||||
{
|
||||
// replace x_values with their ranks
|
||||
size_t rank = j + 1;
|
||||
size_t same = 1;
|
||||
size_t cur_sum = rank;
|
||||
size_t cur_start = j;
|
||||
|
||||
while (j < size - 1)
|
||||
{
|
||||
if (tmp_values[j].first == tmp_values[j + 1].first)
|
||||
{
|
||||
// rank of (j + 1)th number
|
||||
rank += 1;
|
||||
++same;
|
||||
cur_sum += rank;
|
||||
++j;
|
||||
}
|
||||
else
|
||||
break;
|
||||
}
|
||||
|
||||
// insert rank is calculated as average of ranks of equal values
|
||||
Float64 insert_rank = static_cast<Float64>(cur_sum) / same;
|
||||
for (size_t i = cur_start; i <= j; ++i)
|
||||
tmp_values[i].first = insert_rank;
|
||||
++j;
|
||||
}
|
||||
|
||||
// sort y_values
|
||||
std::sort(std::begin(tmp_values), std::end(tmp_values), ComparePairSecond<std::greater>{});
|
||||
|
||||
// replace y_values with their ranks
|
||||
for (size_t j = 0; j < size;)
|
||||
{
|
||||
// replace x_values with their ranks
|
||||
size_t rank = j + 1;
|
||||
size_t same = 1;
|
||||
size_t cur_sum = rank;
|
||||
size_t cur_start = j;
|
||||
|
||||
while (j < size - 1)
|
||||
{
|
||||
if (tmp_values[j].second == tmp_values[j + 1].second)
|
||||
{
|
||||
// rank of (j + 1)th number
|
||||
rank += 1;
|
||||
++same;
|
||||
cur_sum += rank;
|
||||
++j;
|
||||
}
|
||||
else
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// insert rank is calculated as average of ranks of equal values
|
||||
Float64 insert_rank = static_cast<Float64>(cur_sum) / same;
|
||||
for (size_t i = cur_start; i <= j; ++i)
|
||||
tmp_values[i].second = insert_rank;
|
||||
++j;
|
||||
}
|
||||
RanksArray second_ranks;
|
||||
second_ranks.resize(second.size());
|
||||
computeRanks<SecondSample, RanksArray>(second, second_ranks);
|
||||
|
||||
// count d^2 sum
|
||||
Float64 answer = static_cast<Float64>(0);
|
||||
for (size_t j = 0; j < size; ++ j)
|
||||
answer += (tmp_values[j].first - tmp_values[j].second) * (tmp_values[j].first - tmp_values[j].second);
|
||||
answer += (first_ranks[j] - second_ranks[j]) * (first_ranks[j] - second_ranks[j]);
|
||||
|
||||
answer *= 6;
|
||||
answer /= size * (size * size - 1);
|
||||
|
68
src/AggregateFunctions/StatSample.h
Normal file
68
src/AggregateFunctions/StatSample.h
Normal file
@ -0,0 +1,68 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/ArenaAllocator.h>
|
||||
|
||||
#include <numeric>
|
||||
#include <algorithm>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename Values, typename Ranks>
|
||||
void computeRanks(const Values & values, Ranks & out) {
|
||||
std::vector<size_t> indexes(values.size());
|
||||
std::iota(indexes.begin(), indexes.end(), 0);
|
||||
std::sort(indexes.begin(), indexes.end(),
|
||||
[&] (size_t lhs, size_t rhs) { return values[lhs] < values[rhs]; });
|
||||
|
||||
size_t left = 0;
|
||||
while (left < indexes.size()) {
|
||||
size_t right = left;
|
||||
while (right < indexes.size() && values[indexes[left]] == values[indexes[right]]) {
|
||||
++right;
|
||||
}
|
||||
auto adjusted = (left + right + 1.) / 2.;
|
||||
for (size_t iter = left; iter < right; ++iter) {
|
||||
out[indexes[iter]] = adjusted;
|
||||
}
|
||||
left = right;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename T = double>
|
||||
struct StatisticalSampleData
|
||||
{
|
||||
using Allocator = MixedAlignedArenaAllocator<alignof(T), 4096>;
|
||||
using SampleArray = PODArray<T, 32, Allocator>;
|
||||
|
||||
SampleArray first;
|
||||
SampleArray second;
|
||||
|
||||
void add(T x, T y)
|
||||
{
|
||||
first.push_back(x);
|
||||
second.push_back(y);
|
||||
}
|
||||
|
||||
void merge(const StatisticalSampleData & rhs)
|
||||
{
|
||||
first.insert(first.end(), rhs.first.begin(), rhs.first.end());
|
||||
second.insert(second.end(), rhs.second.begin(), rhs.second.end());
|
||||
}
|
||||
|
||||
void write(WriteBuffer & buf) const
|
||||
{
|
||||
writePODBinary(*this, buf);
|
||||
}
|
||||
|
||||
void read(ReadBuffer & buf) const
|
||||
{
|
||||
readPODBinary(*this, buf);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -7,7 +7,7 @@ SELECT '1';
|
||||
SELECT rankCorr(number, number) FROM numbers(100);
|
||||
|
||||
SELECT '-1';
|
||||
SELECT rankCorr(number, -1 * number) FROM numbers(100);
|
||||
SELECT rankCorr(number, -1 * CAST(number AS Int64)) FROM numbers(100);
|
||||
|
||||
SELECT '-0.037';
|
||||
SELECT roundBankers(rankCorr(exp(number), sin(number)), 3) FROM numbers(100);
|
||||
|
Loading…
Reference in New Issue
Block a user