mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Merge
This commit is contained in:
parent
9dc8900ee0
commit
f93141355b
@ -20,6 +20,7 @@
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
#include <DB/AggregateFunctions/UniqCombinedBiasData.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -29,7 +30,7 @@ namespace DB
|
||||
|
||||
struct AggregateFunctionUniqUniquesHashSetData
|
||||
{
|
||||
typedef UniquesHashSet<DefaultHash<UInt64>> Set;
|
||||
typedef UniquesHashSet<DefaultHash<UInt64> > Set;
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniq"; }
|
||||
@ -94,83 +95,79 @@ struct AggregateFunctionUniqExactData<String>
|
||||
static String getName() { return "uniqExact"; }
|
||||
};
|
||||
|
||||
template <typename T, HyperLogLogMode mode>
|
||||
struct BaseUniqCombinedData
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, DefaultHash<Key>, HashTableGrower<> >,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
DefaultHash<Key>,
|
||||
UInt64,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
mode
|
||||
>;
|
||||
|
||||
Set set;
|
||||
};
|
||||
|
||||
template <HyperLogLogMode mode>
|
||||
struct BaseUniqCombinedData<String, mode>
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<
|
||||
Key,
|
||||
HashSet<Key, TrivialHash, HashTableGrower<> >,
|
||||
16,
|
||||
14,
|
||||
17,
|
||||
TrivialHash,
|
||||
UInt64,
|
||||
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
|
||||
mode
|
||||
>;
|
||||
|
||||
Set set;
|
||||
};
|
||||
|
||||
/// Агрегатные функции uniqCombinedRaw, uniqCombinedLinearCounting, и uniqCombinedBiasCorrected
|
||||
/// предназначены для разработки новых версий функции uniqCombined.
|
||||
/// Пользователи должны использовать только uniqCombined.
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedRawData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::Raw>
|
||||
{
|
||||
static String getName() { return "uniqCombinedRaw"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedLinearCountingData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::LinearCounting>
|
||||
{
|
||||
static String getName() { return "uniqCombinedLinearCounting"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedBiasCorrectedData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::BiasCorrected>
|
||||
{
|
||||
static String getName() { return "uniqCombinedBiasCorrected"; }
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct AggregateFunctionUniqCombinedData
|
||||
: public BaseUniqCombinedData<T, HyperLogLogMode::FullFeatured>
|
||||
{
|
||||
using Key = UInt32;
|
||||
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<> >, 16, 14, 17, TrivialHash>;
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct AggregateFunctionUniqCombinedData<String>
|
||||
{
|
||||
using Key = UInt64;
|
||||
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, HashTableGrower<> >, 16, 14, 17, TrivialHash>;
|
||||
Set set;
|
||||
|
||||
static String getName() { return "uniqCombined"; }
|
||||
};
|
||||
|
||||
namespace detail
|
||||
{
|
||||
|
||||
/** Хэш-функция для uniqCombined.
|
||||
*/
|
||||
template<typename T, typename Enable = void>
|
||||
struct CombinedCardinalityTraits
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
return key;
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Int64>::value>::type>
|
||||
{
|
||||
using U = typename std::make_unsigned<T>::type;
|
||||
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
return intHash32<0>(static_cast<U>(key));
|
||||
};
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, UInt64>::value>::type>
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
return intHash32<0>(key);
|
||||
};
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Float64>::value>::type>
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
UInt64 res = 0;
|
||||
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&key), sizeof(key));
|
||||
return intHash32<0>(res);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct CombinedCardinalityTraits<T, typename std::enable_if<std::is_same<T, Float32>::value>::type>
|
||||
{
|
||||
static UInt32 hash(T key)
|
||||
{
|
||||
UInt32 res = 0;
|
||||
memcpy(reinterpret_cast<char *>(&res), reinterpret_cast<char *>(&key), sizeof(key));
|
||||
return res;
|
||||
}
|
||||
};
|
||||
|
||||
/** Хэш-функция для uniq.
|
||||
*/
|
||||
template <typename T> struct AggregateFunctionUniqTraits
|
||||
@ -199,41 +196,51 @@ template <> struct AggregateFunctionUniqTraits<Float64>
|
||||
};
|
||||
|
||||
/** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq.
|
||||
* Используется для частичной специализации для добавления строк.
|
||||
*/
|
||||
template<typename T, typename Data>
|
||||
struct OneAdder
|
||||
{
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num)
|
||||
{
|
||||
data.set.insert(AggregateFunctionUniqTraits<T>::hash(static_cast<const ColumnVector<T> &>(column).getData()[row_num]));
|
||||
}
|
||||
};
|
||||
* Используется для частичной специализации для добавления строк.
|
||||
*/
|
||||
template <typename T, typename Data, typename Enable = void>
|
||||
struct OneAdder;
|
||||
|
||||
template<typename Data>
|
||||
struct OneAdder<String, Data>
|
||||
template <typename T, typename Data>
|
||||
struct OneAdder<T, Data, typename std::enable_if<
|
||||
std::is_same<Data, AggregateFunctionUniqUniquesHashSetData>::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqHLL12Data<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedRawData<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedLinearCountingData<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedBiasCorrectedData<T> >::value ||
|
||||
std::is_same<Data, AggregateFunctionUniqCombinedData<T> >::value>::type>
|
||||
{
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num)
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T2> &>(column).getData()[row_num];
|
||||
data.set.insert(AggregateFunctionUniqTraits<T2>::hash(value));
|
||||
}
|
||||
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
/// Имейте ввиду, что вычисление приближённое.
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash64(value.data, value.size));
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct OneAdder<T, AggregateFunctionUniqExactData<T> >
|
||||
template <typename T, typename Data>
|
||||
struct OneAdder<T, Data, typename std::enable_if<
|
||||
std::is_same<Data, AggregateFunctionUniqExactData<T>>::value>::type>
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqExactData<T> & data, const IColumn & column, size_t row_num)
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
data.set.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
|
||||
data.set.insert(static_cast<const ColumnVector<T2> &>(column).getData()[row_num]);
|
||||
}
|
||||
};
|
||||
|
||||
template<>
|
||||
struct OneAdder<String, AggregateFunctionUniqExactData<String> >
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqExactData<String> & data, const IColumn & column, size_t row_num)
|
||||
template <typename T2 = T>
|
||||
static void addOne(Data & data, const IColumn & column, size_t row_num,
|
||||
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
|
||||
@ -246,26 +253,6 @@ struct OneAdder<String, AggregateFunctionUniqExactData<String> >
|
||||
}
|
||||
};
|
||||
|
||||
template<typename T>
|
||||
struct OneAdder<T, AggregateFunctionUniqCombinedData<T> >
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqCombinedData<T> & data, const IColumn & column, size_t row_num)
|
||||
{
|
||||
const auto & value = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
data.set.insert(CombinedCardinalityTraits<T>::hash(value));
|
||||
}
|
||||
};
|
||||
|
||||
template<>
|
||||
struct OneAdder<String, AggregateFunctionUniqCombinedData<String> >
|
||||
{
|
||||
static void addOne(AggregateFunctionUniqCombinedData<String> & data, const IColumn & column, size_t row_num)
|
||||
{
|
||||
StringRef value = column.getDataAt(row_num);
|
||||
data.set.insert(CityHash64(value.data, value.size));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
39
dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h
Normal file
39
dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
|
||||
#include <array>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Данные для HyperLogLogBiasEstimator в функции uniqCombined.
|
||||
* Схема разработки следующая:
|
||||
* 1. Собрать clickhouse.
|
||||
* 2. Запустить скрипт src/dbms/scripts/gen-bias-data.py, который возвращает один массив для getRawEstimates()
|
||||
* и другой массив для getBiases().
|
||||
* 3. Обновить массивы raw_estimates и biases. Также обновить размер массивов в InterpolatedData.
|
||||
* 4. Собрать clickhouse.
|
||||
* 5. Запустить скрипт src/dbms/scripts/linear-counting-threshold.py, который создаёт 3 файла:
|
||||
* - raw_graph.txt (1-й столбец: настоящее количество уникальных значений;
|
||||
* 2-й столбец: относительная погрешность в случае HyperLogLog без применения каких-либо поправок)
|
||||
* - linear_counting_graph.txt (1-й столбец: настоящее количество уникальных значений;
|
||||
* 2-й столбец: относительная погрешность в случае HyperLogLog с применением LinearCounting)
|
||||
* - bias_corrected_graph.txt (1-й столбец: настоящее количество уникальных значений;
|
||||
* 2-й столбец: относительная погрешность в случае HyperLogLog с применением поправок из алгортима HyperLogLog++)
|
||||
* 6. Сгенерить график с gnuplot на основе этих данных.
|
||||
* 7. Определить минимальное количество уникальных значений, при котором лучше исправить погрешность
|
||||
* с помощью её оценки (т.е. по алгоритму HyperLogLog++), чем применить алгоритм LinearCounting.
|
||||
* 7. Соответственно обновить константу в функции getThreshold()
|
||||
* 8. Собрать clickhouse.
|
||||
*/
|
||||
struct UniqCombinedBiasData
|
||||
{
|
||||
using InterpolatedData = std::array<double, 178>;
|
||||
|
||||
static double getThreshold();
|
||||
/// Оценки количества уникальных значений по алгоритму HyperLogLog без применения каких-либо поправок.
|
||||
static const InterpolatedData & getRawEstimates();
|
||||
/// Соответствующие оценки погрешности.
|
||||
static const InterpolatedData & getBiases();
|
||||
};
|
||||
|
||||
}
|
@ -34,17 +34,32 @@ template
|
||||
UInt8 medium_set_power2_max,
|
||||
UInt8 K,
|
||||
typename Hash = IntHash32<Key>,
|
||||
typename HashValueType = UInt32,
|
||||
typename BiasEstimator = TrivialBiasEstimator,
|
||||
HyperLogLogMode mode = HyperLogLogMode::FullFeatured,
|
||||
typename DenominatorType = double
|
||||
>
|
||||
class CombinedCardinalityEstimator
|
||||
{
|
||||
public:
|
||||
using Self = CombinedCardinalityEstimator<Key, HashContainer, small_set_size_max, medium_set_power2_max, K, Hash, DenominatorType>;
|
||||
using Self = CombinedCardinalityEstimator
|
||||
<
|
||||
Key,
|
||||
HashContainer,
|
||||
small_set_size_max,
|
||||
medium_set_power2_max,
|
||||
K,
|
||||
Hash,
|
||||
HashValueType,
|
||||
BiasEstimator,
|
||||
mode,
|
||||
DenominatorType
|
||||
>;
|
||||
|
||||
private:
|
||||
using Small = SmallSet<Key, small_set_size_max>;
|
||||
using Medium = HashContainer;
|
||||
using Large = HyperLogLogCounter<K, Hash, DenominatorType>;
|
||||
using Large = HyperLogLogCounter<K, Hash, HashValueType, DenominatorType, BiasEstimator, mode>;
|
||||
|
||||
public:
|
||||
CombinedCardinalityEstimator()
|
||||
|
@ -21,7 +21,7 @@ class HyperLogLogWithSmallSetOptimization
|
||||
{
|
||||
private:
|
||||
using Small = SmallSet<Key, small_set_size>;
|
||||
using Large = HyperLogLogCounter<K, Hash, DenominatorType>;
|
||||
using Large = HyperLogLogCounter<K, Hash, UInt32, DenominatorType>;
|
||||
|
||||
Small small;
|
||||
Large * large = nullptr;
|
||||
|
252
dbms/scripts/gen-bias-data.py
Executable file
252
dbms/scripts/gen-bias-data.py
Executable file
@ -0,0 +1,252 @@
|
||||
#!/usr/bin/python3.4
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
import sys
|
||||
import argparse
|
||||
import tempfile
|
||||
import random
|
||||
import subprocess
|
||||
import bisect
|
||||
from copy import deepcopy
|
||||
|
||||
# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/
|
||||
class UniqueRandomGenerator:
|
||||
prime = 4294967291
|
||||
|
||||
def __init__(self, seed_base, seed_offset):
|
||||
self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161)
|
||||
self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905)
|
||||
|
||||
def next(self):
|
||||
val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635)
|
||||
self.index = self.index + 1
|
||||
return val
|
||||
|
||||
def permutePQR(self, x):
|
||||
if x >=self.prime:
|
||||
return x
|
||||
else:
|
||||
residue = (x * x) % self.prime
|
||||
if x <= self.prime/2:
|
||||
return residue
|
||||
else:
|
||||
return self.prime - residue
|
||||
|
||||
def generate_data_source(host, port, http_port, begin, end, count):
|
||||
chunk_size = round((end - begin) / float(count))
|
||||
used_values = 0
|
||||
|
||||
cur_count = 0
|
||||
next_size = 0
|
||||
|
||||
sup = 32768
|
||||
n1 = random.randrange(0, sup)
|
||||
n2 = random.randrange(0, sup)
|
||||
urng = UniqueRandomGenerator(n1, n2)
|
||||
|
||||
with tempfile.TemporaryDirectory() as tmp_dir:
|
||||
filename = tmp_dir + '/table.txt'
|
||||
file_handle = open(filename, 'w+b')
|
||||
|
||||
while cur_count < count:
|
||||
next_size += chunk_size
|
||||
|
||||
while used_values < next_size:
|
||||
h = urng.next()
|
||||
used_values = used_values + 1
|
||||
outstr = str(h) + "\t" + str(cur_count) + "\n";
|
||||
file_handle.write(bytes(outstr, 'UTF-8'));
|
||||
|
||||
cur_count = cur_count + 1
|
||||
|
||||
file_handle.close()
|
||||
|
||||
query = 'DROP TABLE IF EXISTS data_source'
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
query = 'CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog'
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
|
||||
cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE)
|
||||
subprocess.check_output(("POST", "http://localhost:{0}/?query=INSERT INTO data_source FORMAT TabSeparated".format(http_port)), stdin=cat.stdout)
|
||||
cat.wait()
|
||||
|
||||
def perform_query(host, port):
|
||||
query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, "
|
||||
query += "runningAccumulate(uniqCombinedRawState(UserID)) AS approx "
|
||||
query += "FROM data_source GROUP BY KeyID"
|
||||
return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query])
|
||||
|
||||
def parse_clickhouse_response(output):
|
||||
parsed = []
|
||||
lines = output.decode().split("\n")
|
||||
for cur_line in lines:
|
||||
rows = cur_line.split("\t")
|
||||
if len(rows) == 2:
|
||||
parsed.append([float(rows[0]), float(rows[1])])
|
||||
return parsed
|
||||
|
||||
def accumulate_data(stats, data):
|
||||
if not stats:
|
||||
stats = deepcopy(data)
|
||||
else:
|
||||
for row1, row2 in zip(stats, data):
|
||||
row1[1] += row2[1];
|
||||
return stats
|
||||
|
||||
def generate_raw_result(stats, count):
|
||||
expected_tab = []
|
||||
bias_tab = []
|
||||
for row in stats:
|
||||
exact = row[0]
|
||||
expected = row[1] / count
|
||||
bias = expected - exact
|
||||
|
||||
expected_tab.append(expected)
|
||||
bias_tab.append(bias)
|
||||
return [ expected_tab, bias_tab ]
|
||||
|
||||
def generate_sample(raw_estimates, biases, n_samples):
|
||||
result = []
|
||||
|
||||
min_card = raw_estimates[0]
|
||||
max_card = raw_estimates[len(raw_estimates) - 1]
|
||||
step = (max_card - min_card) / n_samples
|
||||
|
||||
for i in range(0, n_samples + 1):
|
||||
x = min_card + i * step
|
||||
j = bisect.bisect_left(raw_estimates, x)
|
||||
|
||||
if j == len(raw_estimates):
|
||||
result.append((raw_estimates[j - 1], biases[j - 1]))
|
||||
elif raw_estimates[j] == x:
|
||||
result.append((raw_estimates[j], biases[j]))
|
||||
else:
|
||||
# Найти 6 ближайших соседей. Вычислить среднее арифметическое.
|
||||
|
||||
# 6 точек слева x [j-6 j-5 j-4 j-3 j-2 j-1]
|
||||
|
||||
begin = max(j - 6, 0) - 1
|
||||
end = j - 1
|
||||
|
||||
T = []
|
||||
for k in range(end, begin, -1):
|
||||
T.append(x - raw_estimates[k])
|
||||
|
||||
# 6 точек справа x [j j+1 j+2 j+3 j+4 j+5]
|
||||
|
||||
begin = j
|
||||
end = min(j + 5, len(raw_estimates) - 1) + 1
|
||||
|
||||
U = []
|
||||
for k in range(begin, end):
|
||||
U.append(raw_estimates[k] - x)
|
||||
|
||||
# Сливаем расстояния.
|
||||
|
||||
V = []
|
||||
|
||||
lim = min(len(T), len(U))
|
||||
k1 = 0
|
||||
k2 = 0
|
||||
|
||||
while k1 < lim and k2 < lim:
|
||||
if T[k1] == U[k2]:
|
||||
V.append(j - k1 - 1)
|
||||
V.append(j + k2)
|
||||
k1 = k1 + 1
|
||||
k2 = k2 + 1
|
||||
elif T[k1] < U[k2]:
|
||||
V.append(j - k1 - 1)
|
||||
k1 = k1 + 1
|
||||
else:
|
||||
V.append(j + k2)
|
||||
k2 = k2 + 1
|
||||
|
||||
if k1 < len(T):
|
||||
while k1 < len(T):
|
||||
V.append(j - k1 - 1)
|
||||
k1 = k1 + 1
|
||||
elif k2 < len(U):
|
||||
while k2 < len(U):
|
||||
V.append(j + k2)
|
||||
k2 = k2 + 1
|
||||
|
||||
# Выбираем 6 ближайших точек.
|
||||
# Вычисляем средние.
|
||||
|
||||
begin = 0
|
||||
end = min(len(V), 6)
|
||||
|
||||
sum = 0
|
||||
bias = 0
|
||||
for k in range(begin, end):
|
||||
sum += raw_estimates[V[k]]
|
||||
bias += biases[V[k]]
|
||||
sum /= float(end)
|
||||
bias /= float(end)
|
||||
|
||||
result.append((sum, bias))
|
||||
|
||||
# Пропустить последовательные результаты, чьи оценки одинаковые.
|
||||
final_result = []
|
||||
last = -1
|
||||
for entry in result:
|
||||
if entry[0] != last:
|
||||
final_result.append((entry[0], entry[1]))
|
||||
last = entry[0]
|
||||
|
||||
return final_result
|
||||
|
||||
def dump_arrays(stats):
|
||||
|
||||
print("Size of each array: {0}\n".format(len(stats)))
|
||||
|
||||
is_first = True
|
||||
sep = ''
|
||||
|
||||
print("// For UniqCombinedBiasData::getRawEstimates():")
|
||||
print("{")
|
||||
for row in stats:
|
||||
print("\t{0}{1}".format(sep, row[0]))
|
||||
if is_first == True:
|
||||
is_first = False
|
||||
sep = ","
|
||||
print("}")
|
||||
|
||||
is_first = True
|
||||
sep = ""
|
||||
|
||||
print("\n// For UniqCombinedBiasData::getBiases():")
|
||||
print("{")
|
||||
for row in stats:
|
||||
print("\t{0}{1}".format(sep, row[1]))
|
||||
if is_first == True:
|
||||
is_first = False
|
||||
sep = ","
|
||||
print("}")
|
||||
|
||||
def start():
|
||||
parser = argparse.ArgumentParser(description = "Generate bias correction tables.")
|
||||
parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name");
|
||||
parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port");
|
||||
parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port");
|
||||
parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations");
|
||||
parser.add_argument("-s", "--generated", type=int, default=700000, help="number of generated values");
|
||||
parser.add_argument("-g", "--samples", type=int, default=200, help="number of sampled values");
|
||||
args = parser.parse_args()
|
||||
|
||||
stats = []
|
||||
|
||||
for i in range(0, args.iterations):
|
||||
print(i + 1)
|
||||
sys.stdout.flush()
|
||||
generate_data_source(args.host, str(args.port), str(args.http_port), 0, args.generated, 1000)
|
||||
output = perform_query(args.host, str(args.port))
|
||||
data = parse_clickhouse_response(output)
|
||||
stats = accumulate_data(stats, data)
|
||||
|
||||
result = generate_raw_result(stats, args.iterations)
|
||||
sample = generate_sample(result[0], result[1], args.samples)
|
||||
dump_arrays(sample)
|
||||
|
||||
if __name__ == "__main__": start()
|
143
dbms/scripts/linear-counting-threshold.py
Executable file
143
dbms/scripts/linear-counting-threshold.py
Executable file
@ -0,0 +1,143 @@
|
||||
#!/usr/bin/python3.4
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
import sys
|
||||
import argparse
|
||||
import tempfile
|
||||
import random
|
||||
import subprocess
|
||||
import bisect
|
||||
from copy import deepcopy
|
||||
|
||||
# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/
|
||||
class UniqueRandomGenerator:
|
||||
prime = 4294967291
|
||||
|
||||
def __init__(self, seed_base, seed_offset):
|
||||
self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161)
|
||||
self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905)
|
||||
|
||||
def next(self):
|
||||
val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635)
|
||||
self.index = self.index + 1
|
||||
return val
|
||||
|
||||
def permutePQR(self, x):
|
||||
if x >=self.prime:
|
||||
return x
|
||||
else:
|
||||
residue = (x * x) % self.prime
|
||||
if x <= self.prime/2:
|
||||
return residue
|
||||
else:
|
||||
return self.prime - residue
|
||||
|
||||
def generate_data_source(host, port, http_port, begin, end, count):
|
||||
chunk_size = round((end - begin) / float(count))
|
||||
used_values = 0
|
||||
|
||||
cur_count = 0
|
||||
next_size = 0
|
||||
|
||||
sup = 32768
|
||||
n1 = random.randrange(0, sup)
|
||||
n2 = random.randrange(0, sup)
|
||||
urng = UniqueRandomGenerator(n1, n2)
|
||||
|
||||
with tempfile.TemporaryDirectory() as tmp_dir:
|
||||
filename = tmp_dir + '/table.txt'
|
||||
file_handle = open(filename, 'w+b')
|
||||
|
||||
while cur_count < count:
|
||||
next_size += chunk_size
|
||||
|
||||
while used_values < next_size:
|
||||
h = urng.next()
|
||||
used_values = used_values + 1
|
||||
outstr = str(h) + "\t" + str(cur_count) + "\n";
|
||||
file_handle.write(bytes(outstr, 'UTF-8'));
|
||||
|
||||
cur_count = cur_count + 1
|
||||
|
||||
file_handle.close()
|
||||
|
||||
query = 'DROP TABLE IF EXISTS data_source'
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
query = 'CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog'
|
||||
subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query])
|
||||
|
||||
cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE)
|
||||
subprocess.check_output(("POST", "http://localhost:{0}/?query=INSERT INTO data_source FORMAT TabSeparated".format(http_port)), stdin=cat.stdout)
|
||||
cat.wait()
|
||||
|
||||
def perform_query(host, port):
|
||||
query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, "
|
||||
query += "runningAccumulate(uniqCombinedRawState(UserID)) AS raw, "
|
||||
query += "runningAccumulate(uniqCombinedLinearCountingState(UserID)) AS linear_counting, "
|
||||
query += "runningAccumulate(uniqCombinedBiasCorrectedState(UserID)) AS bias_corrected "
|
||||
query += "FROM data_source GROUP BY KeyID"
|
||||
return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query])
|
||||
|
||||
def parse_clickhouse_response(output):
|
||||
parsed = []
|
||||
lines = output.decode().split("\n")
|
||||
for cur_line in lines:
|
||||
rows = cur_line.split("\t")
|
||||
if len(rows) == 4:
|
||||
parsed.append([float(rows[0]), float(rows[1]), float(rows[2]), float(rows[3])])
|
||||
return parsed
|
||||
|
||||
def accumulate_data(stats, data):
|
||||
if not stats:
|
||||
stats = deepcopy(data)
|
||||
else:
|
||||
for row1, row2 in zip(stats, data):
|
||||
row1[1] += row2[1];
|
||||
row1[2] += row2[2];
|
||||
row1[3] += row2[3];
|
||||
return stats
|
||||
|
||||
def dump_graphs(stats, count):
|
||||
fh1 = open("raw_graph.txt", "w+b")
|
||||
fh2 = open("linear_counting_graph.txt", "w+b")
|
||||
fh3 = open("bias_corrected_graph.txt", "w+b")
|
||||
|
||||
expected_tab = []
|
||||
bias_tab = []
|
||||
for row in stats:
|
||||
exact = row[0]
|
||||
raw = row[1] / count;
|
||||
linear_counting = row[2] / count;
|
||||
bias_corrected = row[3] / count;
|
||||
|
||||
outstr = "{0}\t{1}\n".format(exact, abs(raw - exact) / exact)
|
||||
fh1.write(bytes(outstr, 'UTF-8'))
|
||||
|
||||
outstr = "{0}\t{1}\n".format(exact, abs(linear_counting - exact) / exact)
|
||||
fh2.write(bytes(outstr, 'UTF-8'))
|
||||
|
||||
outstr = "{0}\t{1}\n".format(exact, abs(bias_corrected - exact) / exact)
|
||||
fh3.write(bytes(outstr, 'UTF-8'))
|
||||
|
||||
def start():
|
||||
parser = argparse.ArgumentParser(description = "Generate bias correction tables.")
|
||||
parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name");
|
||||
parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port");
|
||||
parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port");
|
||||
parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations");
|
||||
parser.add_argument("-s", "--generated", type=int, default=700000, help="number of generated values");
|
||||
args = parser.parse_args()
|
||||
|
||||
stats = []
|
||||
|
||||
for i in range(0, args.iterations):
|
||||
print(i + 1)
|
||||
sys.stdout.flush()
|
||||
generate_data_source(args.host, str(args.port), str(args.http_port), 0, args.generated, 1000)
|
||||
output = perform_query(args.host, str(args.port))
|
||||
data = parse_clickhouse_response(output)
|
||||
stats = accumulate_data(stats, data)
|
||||
|
||||
dump_graphs(stats, args.iterations)
|
||||
|
||||
if __name__ == "__main__": start()
|
@ -351,6 +351,72 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombinedRaw")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqCombinedRawData>(*argument_types[0]);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDate::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqCombinedRawData<DataTypeDateTime::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedRawData<String>>;
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombinedLinearCounting")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
|
||||
AggregateFunctionUniqCombinedLinearCountingData>(*argument_types[0]);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDate::FieldType,
|
||||
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDate::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
|
||||
AggregateFunctionUniqCombinedLinearCountingData<DataTypeDateTime::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedLinearCountingData<String>>;
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombinedBiasCorrected")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const IDataType & argument_type = *argument_types[0];
|
||||
|
||||
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq,
|
||||
AggregateFunctionUniqCombinedBiasCorrectedData>(*argument_types[0]);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDate::FieldType,
|
||||
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDate::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionUniq<DataTypeDateTime::FieldType,
|
||||
AggregateFunctionUniqCombinedBiasCorrectedData<DataTypeDateTime::FieldType>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
return new AggregateFunctionUniq<String, AggregateFunctionUniqCombinedBiasCorrectedData<String>>;
|
||||
else
|
||||
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (name == "uniqCombined")
|
||||
{
|
||||
if (argument_types.size() != 1)
|
||||
@ -733,6 +799,9 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun
|
||||
"uniq",
|
||||
"uniqHLL12",
|
||||
"uniqExact",
|
||||
"uniqCombinedRaw",
|
||||
"uniqCombinedLinearCounting",
|
||||
"uniqCombinedBiasCorrected",
|
||||
"uniqCombined",
|
||||
"uniqUpTo",
|
||||
"groupArray",
|
||||
|
390
dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp
Normal file
390
dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp
Normal file
@ -0,0 +1,390 @@
|
||||
#include <DB/AggregateFunctions/UniqCombinedBiasData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData raw_estimates =
|
||||
{
|
||||
700.0
|
||||
,3850.0
|
||||
,7350.0
|
||||
,10850.0
|
||||
,14350.0
|
||||
,89003.5714
|
||||
,103764.30343333333
|
||||
,105572.1915
|
||||
,109252.46533333334
|
||||
,112638.20573333332
|
||||
,116094.29566666669
|
||||
,119619.81926666666
|
||||
,123214.92233333334
|
||||
,126469.06656666666
|
||||
,130196.15093333334
|
||||
,133566.85673333335
|
||||
,136991.63890000002
|
||||
,140470.0118666667
|
||||
,144000.91686666667
|
||||
,147585.44463333333
|
||||
,151222.7466
|
||||
,154447.75893333333
|
||||
,158181.68399999998
|
||||
,161492.41386666667
|
||||
,164840.6352
|
||||
,168713.9904
|
||||
,172143.82656666666
|
||||
,175611.2078
|
||||
,179116.94873333335
|
||||
,182658.0355
|
||||
,186236.36723333332
|
||||
,189332.1009
|
||||
,192976.1847
|
||||
,196654.62706666664
|
||||
,199835.39103333335
|
||||
,203575.92429999998
|
||||
,206808.87086666666
|
||||
,210611.72886666664
|
||||
,213896.25913333334
|
||||
,217759.63066666664
|
||||
,221096.10933333333
|
||||
,224456.31466666667
|
||||
,227839.0366333333
|
||||
,231242.72576666667
|
||||
,235239.98256666667
|
||||
,238688.95070000002
|
||||
,242158.17593333332
|
||||
,245649.42926666664
|
||||
,249158.9859666667
|
||||
,252689.67179999998
|
||||
,256241.95376666667
|
||||
,259214.9391666667
|
||||
,262798.3925666667
|
||||
,266399.8345666667
|
||||
,270018.35863333335
|
||||
,273653.1149
|
||||
,276696.7119
|
||||
,280366.51476666663
|
||||
,284051.95540000004
|
||||
,287133.5254333333
|
||||
,290847.31173333334
|
||||
,294579.5226
|
||||
,297698.64109999995
|
||||
,301454.39253333333
|
||||
,305223.59123333334
|
||||
,308375.3184666667
|
||||
,312170.06
|
||||
,315342.02996666665
|
||||
,319162.8188666667
|
||||
,322356.3565666666
|
||||
,326199.5866
|
||||
,329412.83396666666
|
||||
,332634.3235666667
|
||||
,336510.7596333333
|
||||
,339747.7330333333
|
||||
,343643.0385666667
|
||||
,346896.77420000004
|
||||
,350157.6729666667
|
||||
,354079.3932333334
|
||||
,357354.5196333334
|
||||
,360638.3034333333
|
||||
,364588.47873333335
|
||||
,367886.05706666666
|
||||
,371189.98006666667
|
||||
,375161.95876666665
|
||||
,378478.6737666666
|
||||
,381801.6619
|
||||
,385130.9645
|
||||
,389131.7460333333
|
||||
,392471.6233333333
|
||||
,395817.1175
|
||||
,399165.1003333333
|
||||
,402518.7819333333
|
||||
,406549.7624333333
|
||||
,409916.016
|
||||
,413289.0218666666
|
||||
,416661.9977333333
|
||||
,420040.4257333334
|
||||
,424099.3186333333
|
||||
,427485.4292000001
|
||||
,430876.4814666666
|
||||
,434269.4718
|
||||
,437665.82826666674
|
||||
,441066.7185
|
||||
,444469.97226666665
|
||||
,448561.9376666667
|
||||
,451974.73750000005
|
||||
,455389.1112
|
||||
,458808.5816666667
|
||||
,462230.8184666667
|
||||
,465656.9889
|
||||
,469081.3269
|
||||
,472512.4878
|
||||
,475944.4204333333
|
||||
,480065.7132666667
|
||||
,483502.04110000003
|
||||
,486939.5075666667
|
||||
,490379.7868333334
|
||||
,493818.5365333333
|
||||
,497259.08013333334
|
||||
,500705.3513
|
||||
,504155.6234666666
|
||||
,507606.65499999997
|
||||
,511060.7448666667
|
||||
,514517.4004
|
||||
,517973.35829999996
|
||||
,521431.3761666666
|
||||
,524891.7097333333
|
||||
,529044.7593
|
||||
,532507.0878999999
|
||||
,535971.5070333333
|
||||
,539436.2416999999
|
||||
,542903.1470333333
|
||||
,546370.3423
|
||||
,549837.6947999999
|
||||
,553307.0003666667
|
||||
,556775.3770333333
|
||||
,560247.6308333334
|
||||
,563721.0700333334
|
||||
,567196.7586333333
|
||||
,570669.8439666666
|
||||
,574146.018
|
||||
,577622.2794666667
|
||||
,581098.3862333334
|
||||
,584575.8826666666
|
||||
,588055.1468000001
|
||||
,591538.0234
|
||||
,595018.0103000001
|
||||
,598504.5469333333
|
||||
,601992.5697666666
|
||||
,605475.5452
|
||||
,608959.4645
|
||||
,612444.0261
|
||||
,615929.6436
|
||||
,619412.3877333334
|
||||
,622903.4263999999
|
||||
,626391.3657333333
|
||||
,629876.7359333333
|
||||
,633364.2825999999
|
||||
,636855.2673666667
|
||||
,640344.4321000001
|
||||
,643836.5543666667
|
||||
,647327.3073999999
|
||||
,650818.3525666667
|
||||
,654312.2421666667
|
||||
,657807.0899666668
|
||||
,661301.4443666666
|
||||
,664794.1040333334
|
||||
,668288.1969666666
|
||||
,671781.0196666667
|
||||
,675272.7522333333
|
||||
,678766.9045999999
|
||||
,682259.3583666667
|
||||
,685747.8148333334
|
||||
,689238.7994666666
|
||||
,692732.0478333334
|
||||
,696224.6407
|
||||
,700069.9224
|
||||
};
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData biases =
|
||||
{
|
||||
0.0
|
||||
,0.0
|
||||
,0.0
|
||||
,0.0
|
||||
,0.0
|
||||
,71153.5714
|
||||
,85214.30343333333
|
||||
,83522.1915
|
||||
,80202.46533333334
|
||||
,77288.20573333332
|
||||
,74444.29566666667
|
||||
,71669.81926666667
|
||||
,68964.92233333334
|
||||
,66619.06656666666
|
||||
,64046.15093333333
|
||||
,61816.85673333333
|
||||
,59641.6389
|
||||
,57520.01186666667
|
||||
,55450.91686666667
|
||||
,53435.44463333334
|
||||
,51472.74659999999
|
||||
,49797.75893333333
|
||||
,47931.68399999999
|
||||
,46342.41386666667
|
||||
,44790.6352
|
||||
,43063.9904
|
||||
,41593.82656666667
|
||||
,40161.2078
|
||||
,38766.94873333333
|
||||
,37408.035500000005
|
||||
,36086.36723333333
|
||||
,34982.1009
|
||||
,33726.184700000005
|
||||
,32504.627066666664
|
||||
,31485.391033333333
|
||||
,30325.924299999995
|
||||
,29358.870866666668
|
||||
,28261.72886666667
|
||||
,27346.259133333337
|
||||
,26309.630666666668
|
||||
,25446.109333333337
|
||||
,24606.31466666666
|
||||
,23789.036633333333
|
||||
,22992.725766666666
|
||||
,22089.98256666667
|
||||
,21338.9507
|
||||
,20608.175933333332
|
||||
,19899.429266666673
|
||||
,19208.985966666663
|
||||
,18539.6718
|
||||
,17891.95376666667
|
||||
,17364.939166666667
|
||||
,16748.392566666666
|
||||
,16149.834566666666
|
||||
,15568.358633333331
|
||||
,15003.114899999995
|
||||
,14546.711900000004
|
||||
,14016.51476666668
|
||||
,13501.955399999997
|
||||
,13083.52543333332
|
||||
,12597.311733333336
|
||||
,12129.522600000006
|
||||
,11748.641100000008
|
||||
,11304.392533333332
|
||||
,10873.59123333334
|
||||
,10525.318466666678
|
||||
,10120.059999999998
|
||||
,9792.029966666674
|
||||
,9412.818866666668
|
||||
,9106.356566666664
|
||||
,8749.58660000001
|
||||
,8462.833966666678
|
||||
,8184.323566666659
|
||||
,7860.759633333325
|
||||
,7597.733033333323
|
||||
,7293.038566666665
|
||||
,7046.774200000004
|
||||
,6807.672966666675
|
||||
,6529.393233333336
|
||||
,6304.519633333344
|
||||
,6088.30343333332
|
||||
,5838.4787333333325
|
||||
,5636.057066666661
|
||||
,5439.980066666671
|
||||
,5211.958766666658
|
||||
,5028.673766666664
|
||||
,4851.661899999996
|
||||
,4680.964499999992
|
||||
,4481.746033333319
|
||||
,4321.623333333322
|
||||
,4167.117500000012
|
||||
,4015.1003333333356
|
||||
,3868.781933333337
|
||||
,3699.762433333332
|
||||
,3566.0159999999937
|
||||
,3439.021866666648
|
||||
,3311.9977333333422
|
||||
,3190.4257333333276
|
||||
,3049.3186333333238
|
||||
,2935.4291999999937
|
||||
,2826.4814666666593
|
||||
,2719.4717999999993
|
||||
,2615.8282666666782
|
||||
,2516.7184999999977
|
||||
,2419.972266666669
|
||||
,2311.9376666666744
|
||||
,2224.7374999999884
|
||||
,2139.1111999999944
|
||||
,2058.581666666665
|
||||
,1980.8184666666687
|
||||
,1906.9888999999966
|
||||
,1831.3268999999952
|
||||
,1762.4878000000026
|
||||
,1694.420433333328
|
||||
,1615.7132666666682
|
||||
,1552.0410999999924
|
||||
,1489.507566666677
|
||||
,1429.7868333333365
|
||||
,1368.536533333332
|
||||
,1309.0801333333268
|
||||
,1255.35129999999
|
||||
,1205.6234666666617
|
||||
,1156.6549999999988
|
||||
,1110.744866666675
|
||||
,1067.4004000000034
|
||||
,1023.3583000000023
|
||||
,981.3761666666638
|
||||
,941.7097333333513
|
||||
,894.7593000000148
|
||||
,857.0879000000035
|
||||
,821.5070333333375
|
||||
,786.2416999999745
|
||||
,753.1470333333127
|
||||
,720.3422999999797
|
||||
,687.6947999999975
|
||||
,657.0003666666647
|
||||
,625.3770333333329
|
||||
,597.6308333333387
|
||||
,571.0700333333225
|
||||
,546.7586333333165
|
||||
,519.8439666666478
|
||||
,496.0180000000012
|
||||
,472.2794666666693
|
||||
,448.386233333343
|
||||
,425.8826666666816
|
||||
,405.1468000000071
|
||||
,388.0233999999861
|
||||
,368.01030000002356
|
||||
,354.54693333333125
|
||||
,342.5697666666626
|
||||
,325.5452000000126
|
||||
,309.4644999999825
|
||||
,294.0261000000173
|
||||
,279.64360000001034
|
||||
,262.38773333333666
|
||||
,253.42639999999665
|
||||
,241.36573333333945
|
||||
,226.7359333333443
|
||||
,214.28259999999622
|
||||
,205.26736666667662
|
||||
,194.43210000001514
|
||||
,186.55436666666841
|
||||
,177.30740000001
|
||||
,168.35256666666828
|
||||
,162.24216666668266
|
||||
,157.0899666666713
|
||||
,151.44436666666297
|
||||
,144.1040333333464
|
||||
,138.19696666668946
|
||||
,131.01966666666945
|
||||
,122.7522333333424
|
||||
,116.90459999998954
|
||||
,109.35836666667213
|
||||
,97.81483333332774
|
||||
,88.7994666666491
|
||||
,82.04783333333519
|
||||
,74.64070000000841
|
||||
,69.92240000003949
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
double UniqCombinedBiasData::getThreshold()
|
||||
{
|
||||
return 176000;
|
||||
}
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData & UniqCombinedBiasData::getRawEstimates()
|
||||
{
|
||||
return raw_estimates;
|
||||
}
|
||||
|
||||
const UniqCombinedBiasData::InterpolatedData & UniqCombinedBiasData::getBiases()
|
||||
{
|
||||
return biases;
|
||||
}
|
||||
|
||||
}
|
@ -239,25 +239,25 @@
|
||||
31 162
|
||||
35 162
|
||||
36 162
|
||||
0 54226
|
||||
1 54034
|
||||
3 54016
|
||||
6 53982
|
||||
7 54076
|
||||
9 54218
|
||||
10 27075
|
||||
11 54093
|
||||
13 54108
|
||||
14 54096
|
||||
17 54294
|
||||
19 54070
|
||||
20 54028
|
||||
21 54170
|
||||
22 54106
|
||||
26 54103
|
||||
31 54050
|
||||
35 54130
|
||||
36 53868
|
||||
0 53901
|
||||
1 54056
|
||||
3 53999
|
||||
6 54129
|
||||
7 54213
|
||||
9 53853
|
||||
10 26975
|
||||
11 54064
|
||||
13 53963
|
||||
14 53997
|
||||
17 54129
|
||||
19 53923
|
||||
20 53958
|
||||
21 54117
|
||||
22 54150
|
||||
26 54047
|
||||
31 53948
|
||||
35 53931
|
||||
36 53982
|
||||
0.125 1
|
||||
0.5 1
|
||||
0.05 1
|
||||
@ -291,25 +291,25 @@
|
||||
0.043 162
|
||||
0.037 162
|
||||
0.071 162
|
||||
0.045 54170
|
||||
0.125 54076
|
||||
0.5 54034
|
||||
0.05 54070
|
||||
0.143 53982
|
||||
0.091 27075
|
||||
0.056 54294
|
||||
0.048 54028
|
||||
0.083 54093
|
||||
0.25 54016
|
||||
1 54226
|
||||
0.1 54218
|
||||
0.028 54130
|
||||
0.027 53868
|
||||
0.031 54050
|
||||
0.067 54096
|
||||
0.043 54106
|
||||
0.037 54103
|
||||
0.071 54108
|
||||
0.045 54117
|
||||
0.125 54213
|
||||
0.5 54056
|
||||
0.05 53923
|
||||
0.143 54129
|
||||
0.091 26975
|
||||
0.056 54129
|
||||
0.048 53958
|
||||
0.083 54064
|
||||
0.25 53999
|
||||
1 53901
|
||||
0.1 53853
|
||||
0.028 53931
|
||||
0.027 53982
|
||||
0.031 53948
|
||||
0.067 53997
|
||||
0.043 54150
|
||||
0.037 54047
|
||||
0.071 53963
|
||||
0.5 1
|
||||
0.05 1
|
||||
0.25 1
|
||||
@ -343,25 +343,25 @@
|
||||
0.037 162
|
||||
0.1 163
|
||||
1 162
|
||||
0.5 54034
|
||||
0.05 54070
|
||||
0.25 54016
|
||||
0.048 54028
|
||||
0.091 27075
|
||||
0.043 54106
|
||||
0.071 54108
|
||||
0.083 54093
|
||||
0.125 54076
|
||||
0.031 54050
|
||||
0.143 53982
|
||||
0.028 54130
|
||||
0.067 54096
|
||||
0.045 54170
|
||||
0.027 53868
|
||||
0.056 54294
|
||||
0.037 54103
|
||||
0.1 54218
|
||||
1 54226
|
||||
0.5 54056
|
||||
0.05 53923
|
||||
0.25 53999
|
||||
0.048 53958
|
||||
0.091 26975
|
||||
0.043 54150
|
||||
0.071 53963
|
||||
0.083 54064
|
||||
0.125 54213
|
||||
0.031 53948
|
||||
0.143 54129
|
||||
0.028 53931
|
||||
0.067 53997
|
||||
0.045 54117
|
||||
0.027 53982
|
||||
0.056 54129
|
||||
0.037 54047
|
||||
0.1 53853
|
||||
1 53901
|
||||
1 1
|
||||
3 1
|
||||
6 1
|
||||
|
Loading…
Reference in New Issue
Block a user