mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #6343 from dimarub2000/benchmark_comparison_mode
[WIP] Clickhouse-benchmark comparison mode
This commit is contained in:
commit
44424651b0
@ -32,6 +32,7 @@
|
||||
#include <Client/Connection.h>
|
||||
#include <Common/InterruptListener.h>
|
||||
#include <Common/Config/configReadClient.h>
|
||||
#include <Common/T_test.h>
|
||||
|
||||
|
||||
/** A tool for evaluating ClickHouse performance.
|
||||
@ -41,6 +42,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Ports = std::vector<UInt16>;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
@ -50,17 +53,34 @@ namespace ErrorCodes
|
||||
class Benchmark : public Poco::Util::Application
|
||||
{
|
||||
public:
|
||||
Benchmark(unsigned concurrency_, double delay_,
|
||||
const String & host_, UInt16 port_, bool secure_, const String & default_database_,
|
||||
Benchmark(unsigned concurrency_, double delay_, Strings && hosts_, Ports && ports_,
|
||||
bool cumulative_, bool secure_, const String & default_database_,
|
||||
const String & user_, const String & password_, const String & stage,
|
||||
bool randomize_, size_t max_iterations_, double max_time_,
|
||||
const String & json_path_, const Settings & settings_)
|
||||
const String & json_path_, size_t confidence_, const Settings & settings_)
|
||||
:
|
||||
concurrency(concurrency_), delay(delay_), queue(concurrency),
|
||||
connections(concurrency, host_, port_, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable),
|
||||
randomize(randomize_), max_iterations(max_iterations_), max_time(max_time_),
|
||||
json_path(json_path_), settings(settings_), global_context(Context::createGlobal()), pool(concurrency)
|
||||
concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_),
|
||||
cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_),
|
||||
confidence(confidence_), json_path(json_path_), settings(settings_),
|
||||
global_context(Context::createGlobal()), pool(concurrency)
|
||||
{
|
||||
const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable;
|
||||
size_t connections_cnt = std::max(ports_.size(), hosts_.size());
|
||||
|
||||
connections.reserve(connections_cnt);
|
||||
comparison_info_total.reserve(connections_cnt);
|
||||
comparison_info_per_interval.reserve(connections_cnt);
|
||||
|
||||
for (size_t i = 0; i < connections_cnt; ++i)
|
||||
{
|
||||
UInt16 cur_port = i >= ports_.size() ? 9000 : ports_[i];
|
||||
std::string cur_host = i >= hosts_.size() ? "localhost" : hosts_[i];
|
||||
|
||||
connections.emplace_back(std::make_unique<ConnectionPool>(concurrency, cur_host, cur_port, default_database_, user_, password_, "benchmark", Protocol::Compression::Enable, secure));
|
||||
comparison_info_per_interval.emplace_back(std::make_shared<Stats>());
|
||||
comparison_info_total.emplace_back(std::make_shared<Stats>());
|
||||
}
|
||||
|
||||
global_context.makeGlobalContext();
|
||||
|
||||
std::cerr << std::fixed << std::setprecision(3);
|
||||
@ -101,21 +121,29 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
using Query = std::string;
|
||||
using Entry = ConnectionPool::Entry;
|
||||
using EntryPtr = std::shared_ptr<Entry>;
|
||||
using EntryPtrs = std::vector<EntryPtr>;
|
||||
|
||||
unsigned concurrency;
|
||||
double delay;
|
||||
|
||||
using Query = std::string;
|
||||
using Queries = std::vector<Query>;
|
||||
Queries queries;
|
||||
|
||||
using Queue = ConcurrentBoundedQueue<Query>;
|
||||
Queue queue;
|
||||
|
||||
ConnectionPool connections;
|
||||
using ConnectionPoolUniq = std::unique_ptr<ConnectionPool>;
|
||||
using ConnectionPoolUniqs = std::vector<ConnectionPoolUniq>;
|
||||
ConnectionPoolUniqs connections;
|
||||
|
||||
bool randomize;
|
||||
bool cumulative;
|
||||
size_t max_iterations;
|
||||
double max_time;
|
||||
size_t confidence;
|
||||
String json_path;
|
||||
Settings settings;
|
||||
Context global_context;
|
||||
@ -128,12 +156,12 @@ private:
|
||||
|
||||
struct Stats
|
||||
{
|
||||
Stopwatch watch;
|
||||
std::atomic<size_t> queries{0};
|
||||
size_t read_rows = 0;
|
||||
size_t read_bytes = 0;
|
||||
size_t result_rows = 0;
|
||||
size_t result_bytes = 0;
|
||||
double work_time = 0;
|
||||
|
||||
using Sampler = ReservoirSampler<double>;
|
||||
Sampler sampler {1 << 16};
|
||||
@ -141,6 +169,7 @@ private:
|
||||
void add(double seconds, size_t read_rows_inc, size_t read_bytes_inc, size_t result_rows_inc, size_t result_bytes_inc)
|
||||
{
|
||||
++queries;
|
||||
work_time += seconds;
|
||||
read_rows += read_rows_inc;
|
||||
read_bytes += read_bytes_inc;
|
||||
result_rows += result_rows_inc;
|
||||
@ -150,8 +179,8 @@ private:
|
||||
|
||||
void clear()
|
||||
{
|
||||
watch.restart();
|
||||
queries = 0;
|
||||
work_time = 0;
|
||||
read_rows = 0;
|
||||
read_bytes = 0;
|
||||
result_rows = 0;
|
||||
@ -160,15 +189,18 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
Stats info_per_interval;
|
||||
Stats info_total;
|
||||
using MultiStats = std::vector<std::shared_ptr<Stats>>;
|
||||
MultiStats comparison_info_per_interval;
|
||||
MultiStats comparison_info_total;
|
||||
T_test t_test;
|
||||
|
||||
Stopwatch total_watch;
|
||||
Stopwatch delay_watch;
|
||||
|
||||
std::mutex mutex;
|
||||
|
||||
ThreadPool pool;
|
||||
|
||||
|
||||
void readQueries()
|
||||
{
|
||||
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
@ -213,7 +245,7 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
if (max_time > 0 && info_total.watch.elapsedSeconds() >= max_time)
|
||||
if (max_time > 0 && total_watch.elapsedSeconds() >= max_time)
|
||||
{
|
||||
std::cout << "Stopping launch of queries. Requested time limit is exhausted.\n";
|
||||
return false;
|
||||
@ -227,8 +259,8 @@ private:
|
||||
|
||||
if (delay > 0 && delay_watch.elapsedSeconds() > delay)
|
||||
{
|
||||
printNumberOfQueriesExecuted(info_total.queries);
|
||||
report(info_per_interval);
|
||||
printNumberOfQueriesExecuted(queries_executed);
|
||||
cumulative ? report(comparison_info_total) : report(comparison_info_per_interval);
|
||||
delay_watch.restart();
|
||||
}
|
||||
}
|
||||
@ -242,11 +274,17 @@ private:
|
||||
std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1);
|
||||
|
||||
for (size_t i = 0; i < concurrency; ++i)
|
||||
pool.schedule(std::bind(&Benchmark::thread, this,
|
||||
connections.get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings))));
|
||||
{
|
||||
EntryPtrs connection_entries;
|
||||
connection_entries.reserve(connections.size());
|
||||
|
||||
for (const auto & connection : connections)
|
||||
connection_entries.emplace_back(std::make_shared<Entry>(connection->get(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings))));
|
||||
|
||||
pool.schedule(std::bind(&Benchmark::thread, this, connection_entries));
|
||||
}
|
||||
|
||||
InterruptListener interrupt_listener;
|
||||
info_per_interval.watch.restart();
|
||||
delay_watch.restart();
|
||||
|
||||
/// Push queries into queue
|
||||
@ -262,20 +300,24 @@ private:
|
||||
}
|
||||
|
||||
pool.wait();
|
||||
info_total.watch.stop();
|
||||
total_watch.stop();
|
||||
|
||||
if (!json_path.empty())
|
||||
reportJSON(info_total, json_path);
|
||||
reportJSON(comparison_info_total, json_path);
|
||||
|
||||
printNumberOfQueriesExecuted(info_total.queries);
|
||||
report(info_total);
|
||||
printNumberOfQueriesExecuted(queries_executed);
|
||||
report(comparison_info_total);
|
||||
}
|
||||
|
||||
|
||||
void thread(ConnectionPool::Entry connection)
|
||||
void thread(EntryPtrs & connection_entries)
|
||||
{
|
||||
Query query;
|
||||
|
||||
/// Randomly choosing connection index
|
||||
pcg64 generator(randomSeed());
|
||||
std::uniform_int_distribution<size_t> distribution(0, connection_entries.size() - 1);
|
||||
|
||||
try
|
||||
{
|
||||
/// In these threads we do not accept INT signal.
|
||||
@ -296,8 +338,7 @@ private:
|
||||
if (shutdown || (max_iterations && queries_executed == max_iterations))
|
||||
return;
|
||||
}
|
||||
|
||||
execute(connection, query);
|
||||
execute(connection_entries, query, distribution(generator));
|
||||
++queries_executed;
|
||||
}
|
||||
}
|
||||
@ -309,20 +350,19 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void execute(ConnectionPool::Entry & connection, Query & query)
|
||||
void execute(EntryPtrs & connection_entries, Query & query, size_t connection_index)
|
||||
{
|
||||
Stopwatch watch;
|
||||
RemoteBlockInputStream stream(
|
||||
*connection,
|
||||
*(*connection_entries[connection_index]),
|
||||
query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage);
|
||||
|
||||
Progress progress;
|
||||
stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });
|
||||
|
||||
stream.readPrefix();
|
||||
while (Block block = stream.read())
|
||||
;
|
||||
while (Block block = stream.read());
|
||||
|
||||
stream.readSuffix();
|
||||
|
||||
const BlockStreamProfileInfo & info = stream.getProfileInfo();
|
||||
@ -330,33 +370,47 @@ private:
|
||||
double seconds = watch.elapsedSeconds();
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
info_per_interval.add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes);
|
||||
info_total.add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes);
|
||||
|
||||
comparison_info_per_interval[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes);
|
||||
comparison_info_total[connection_index]->add(seconds, progress.read_rows, progress.read_bytes, info.rows, info.bytes);
|
||||
t_test.add(connection_index, seconds);
|
||||
}
|
||||
|
||||
|
||||
void report(Stats & info)
|
||||
void report(MultiStats & infos)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
/// Avoid zeros, nans or exceptions
|
||||
if (0 == info.queries)
|
||||
return;
|
||||
std::cerr << "\n";
|
||||
for (size_t i = 0; i < infos.size(); ++i)
|
||||
{
|
||||
const auto & info = infos[i];
|
||||
|
||||
double seconds = info.watch.elapsedSeconds();
|
||||
/// Avoid zeros, nans or exceptions
|
||||
if (0 == info->queries)
|
||||
return;
|
||||
|
||||
std::cerr
|
||||
<< "\n"
|
||||
<< "QPS: " << (info.queries / seconds) << ", "
|
||||
<< "RPS: " << (info.read_rows / seconds) << ", "
|
||||
<< "MiB/s: " << (info.read_bytes / seconds / 1048576) << ", "
|
||||
<< "result RPS: " << (info.result_rows / seconds) << ", "
|
||||
<< "result MiB/s: " << (info.result_bytes / seconds / 1048576) << "."
|
||||
<< "\n";
|
||||
double seconds = info->work_time / concurrency;
|
||||
|
||||
std::cerr
|
||||
<< connections[i]->getDescription() << ", "
|
||||
<< "queries " << info->queries << ", "
|
||||
<< "QPS: " << (info->queries / seconds) << ", "
|
||||
<< "RPS: " << (info->read_rows / seconds) << ", "
|
||||
<< "MiB/s: " << (info->read_bytes / seconds / 1048576) << ", "
|
||||
<< "result RPS: " << (info->result_rows / seconds) << ", "
|
||||
<< "result MiB/s: " << (info->result_bytes / seconds / 1048576) << "."
|
||||
<< "\n";
|
||||
}
|
||||
std::cerr << "\n";
|
||||
|
||||
auto print_percentile = [&](double percent)
|
||||
{
|
||||
std::cerr << percent << "%\t" << info.sampler.quantileInterpolated(percent / 100.0) << " sec." << std::endl;
|
||||
std::cerr << percent << "%\t\t";
|
||||
for (const auto & info : infos)
|
||||
{
|
||||
std::cerr << info->sampler.quantileInterpolated(percent / 100.0) << " sec." << "\t";
|
||||
}
|
||||
std::cerr << "\n";
|
||||
};
|
||||
|
||||
for (int percent = 0; percent <= 90; percent += 10)
|
||||
@ -367,10 +421,16 @@ private:
|
||||
print_percentile(99.9);
|
||||
print_percentile(99.99);
|
||||
|
||||
info.clear();
|
||||
std::cerr << "\n" << t_test.compareAndReport(confidence).second << "\n";
|
||||
|
||||
if (!cumulative)
|
||||
{
|
||||
for (auto & info : infos)
|
||||
info->clear();
|
||||
}
|
||||
}
|
||||
|
||||
void reportJSON(Stats & info, const std::string & filename)
|
||||
void reportJSON(MultiStats & infos, const std::string & filename)
|
||||
{
|
||||
WriteBufferFromFile json_out(filename);
|
||||
|
||||
@ -381,36 +441,41 @@ private:
|
||||
json_out << double_quote << key << ": " << value << (with_comma ? ",\n" : "\n");
|
||||
};
|
||||
|
||||
auto print_percentile = [&json_out, &info](auto percent, bool with_comma = true)
|
||||
auto print_percentile = [&json_out](Stats & info, auto percent, bool with_comma = true)
|
||||
{
|
||||
json_out << "\"" << percent << "\"" << ": " << info.sampler.quantileInterpolated(percent / 100.0) << (with_comma ? ",\n" : "\n");
|
||||
};
|
||||
|
||||
json_out << "{\n";
|
||||
|
||||
json_out << double_quote << "statistics" << ": {\n";
|
||||
for (size_t i = 0; i < infos.size(); ++i)
|
||||
{
|
||||
const auto & info = infos[i];
|
||||
|
||||
double seconds = info.watch.elapsedSeconds();
|
||||
print_key_value("QPS", info.queries / seconds);
|
||||
print_key_value("RPS", info.read_rows / seconds);
|
||||
print_key_value("MiBPS", info.read_bytes / seconds);
|
||||
print_key_value("RPS_result", info.result_rows / seconds);
|
||||
print_key_value("MiBPS_result", info.result_bytes / seconds);
|
||||
print_key_value("num_queries", info.queries.load(), false);
|
||||
json_out << double_quote << connections[i]->getDescription() << ": {\n";
|
||||
json_out << double_quote << "statistics" << ": {\n";
|
||||
|
||||
json_out << "},\n";
|
||||
print_key_value("QPS", info->queries / info->work_time);
|
||||
print_key_value("RPS", info->read_rows / info->work_time);
|
||||
print_key_value("MiBPS", info->read_bytes / info->work_time);
|
||||
print_key_value("RPS_result", info->result_rows / info->work_time);
|
||||
print_key_value("MiBPS_result", info->result_bytes / info->work_time);
|
||||
print_key_value("num_queries", info->queries.load(), false);
|
||||
|
||||
json_out << double_quote << "query_time_percentiles" << ": {\n";
|
||||
json_out << "},\n";
|
||||
json_out << double_quote << "query_time_percentiles" << ": {\n";
|
||||
|
||||
for (int percent = 0; percent <= 90; percent += 10)
|
||||
print_percentile(percent);
|
||||
for (int percent = 0; percent <= 90; percent += 10)
|
||||
print_percentile(*info, percent);
|
||||
|
||||
print_percentile(95);
|
||||
print_percentile(99);
|
||||
print_percentile(99.9);
|
||||
print_percentile(99.99, false);
|
||||
print_percentile(*info, 95);
|
||||
print_percentile(*info, 99);
|
||||
print_percentile(*info, 99.9);
|
||||
print_percentile(*info, 99.99, false);
|
||||
|
||||
json_out << "}\n";
|
||||
json_out << "}\n";
|
||||
json_out << (i == infos.size() - 1 ? "}\n" : "},\n");
|
||||
}
|
||||
|
||||
json_out << "}\n";
|
||||
}
|
||||
@ -449,13 +514,15 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
|
||||
("timelimit,t", value<double>()->default_value(0.), "stop launch of queries after specified time limit")
|
||||
("randomize,r", value<bool>()->default_value(false), "randomize order of execution")
|
||||
("json", value<std::string>()->default_value(""), "write final report to specified file in JSON format")
|
||||
("host,h", value<std::string>()->default_value("localhost"), "")
|
||||
("port", value<UInt16>()->default_value(9000), "")
|
||||
("host,h", value<Strings>()->multitoken(), "")
|
||||
("port,p", value<Ports>()->multitoken(), "")
|
||||
("cumulative", "prints cumulative data instead of data per interval")
|
||||
("secure,s", "Use TLS connection")
|
||||
("user", value<std::string>()->default_value("default"), "")
|
||||
("password", value<std::string>()->default_value(""), "")
|
||||
("database", value<std::string>()->default_value("default"), "")
|
||||
("stacktrace", "print stack traces of exceptions")
|
||||
("confidence", value<size_t>()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)")
|
||||
;
|
||||
|
||||
Settings settings;
|
||||
@ -475,12 +542,15 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
|
||||
print_stacktrace = options.count("stacktrace");
|
||||
|
||||
UseSSL use_ssl;
|
||||
Ports ports = options.count("port") ? options["port"].as<Ports>() : Ports({9000});
|
||||
Strings hosts = options.count("host") ? options["host"].as<Strings>() : Strings({"localhost"});
|
||||
|
||||
Benchmark benchmark(
|
||||
options["concurrency"].as<unsigned>(),
|
||||
options["delay"].as<double>(),
|
||||
options["host"].as<std::string>(),
|
||||
options["port"].as<UInt16>(),
|
||||
std::move(hosts),
|
||||
std::move(ports),
|
||||
options.count("cumulative"),
|
||||
options.count("secure"),
|
||||
options["database"].as<std::string>(),
|
||||
options["user"].as<std::string>(),
|
||||
@ -490,6 +560,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
|
||||
options["iterations"].as<size_t>(),
|
||||
options["timelimit"].as<double>(),
|
||||
options["json"].as<std::string>(),
|
||||
options["confidence"].as<size_t>(),
|
||||
settings);
|
||||
return benchmark.run();
|
||||
}
|
||||
|
@ -88,6 +88,10 @@ public:
|
||||
{
|
||||
return host;
|
||||
}
|
||||
std::string getDescription() const
|
||||
{
|
||||
return host + ":" + toString(port);
|
||||
}
|
||||
|
||||
protected:
|
||||
/** Creates a new object to put in the pool. */
|
||||
|
214
dbms/src/Common/T_test.h
Normal file
214
dbms/src/Common/T_test.h
Normal file
@ -0,0 +1,214 @@
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
#include <cmath>
|
||||
#include <iostream>
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
|
||||
/**
|
||||
* About:
|
||||
* This is implementation of Independent two-sample t-test
|
||||
* Read about it on https://en.wikipedia.org/wiki/Student%27s_t-test (Equal or unequal sample sizes, equal variance)
|
||||
*
|
||||
* Usage:
|
||||
* It's it used to assume with some level of confidence that two distributions don't differ.
|
||||
* Values can be added with T_test.add(0/1, value) and after compared and reported with compareAndReport().
|
||||
*/
|
||||
struct T_test
|
||||
{
|
||||
struct DistributionData
|
||||
{
|
||||
size_t size = 0;
|
||||
double sum = 0;
|
||||
double squares_sum = 0;
|
||||
|
||||
void add(double value)
|
||||
{
|
||||
++size;
|
||||
sum += value;
|
||||
squares_sum += value * value;
|
||||
}
|
||||
|
||||
double avg() const
|
||||
{
|
||||
return sum / size;
|
||||
}
|
||||
|
||||
double var() const
|
||||
{
|
||||
return (squares_sum - (sum * sum / size)) / static_cast<double>(size - 1);
|
||||
}
|
||||
|
||||
void clear()
|
||||
{
|
||||
size = 0;
|
||||
sum = 0;
|
||||
squares_sum = 0;
|
||||
}
|
||||
};
|
||||
|
||||
std::vector<DistributionData> data;
|
||||
|
||||
/// First row corresponds to infinity size of distributions case
|
||||
const double students_table[101][6] =
|
||||
{
|
||||
{ 1.282, 1.645, 1.960, 2.326, 2.576, 3.090 },
|
||||
{ 3.078, 6.314, 12.706, 31.821, 63.657, 318.313 },
|
||||
{ 1.886, 2.920, 4.303, 6.965, 9.925, 22.327 },
|
||||
{ 1.638, 2.353, 3.182, 4.541, 5.841, 10.215 },
|
||||
{ 1.533, 2.132, 2.776, 3.747, 4.604, 7.173 },
|
||||
{ 1.476, 2.015, 2.571, 3.365, 4.032, 5.893 },
|
||||
{ 1.440, 1.943, 2.447, 3.143, 3.707, 5.208 },
|
||||
{ 1.415, 1.895, 2.365, 2.998, 3.499, 4.782 },
|
||||
{ 1.397, 1.860, 2.306, 2.896, 3.355, 4.499 },
|
||||
{ 1.383, 1.833, 2.262, 2.821, 3.250, 4.296 },
|
||||
{ 1.372, 1.812, 2.228, 2.764, 3.169, 4.143 },
|
||||
{ 1.363, 1.796, 2.201, 2.718, 3.106, 4.024 },
|
||||
{ 1.356, 1.782, 2.179, 2.681, 3.055, 3.929 },
|
||||
{ 1.350, 1.771, 2.160, 2.650, 3.012, 3.852 },
|
||||
{ 1.345, 1.761, 2.145, 2.624, 2.977, 3.787 },
|
||||
{ 1.341, 1.753, 2.131, 2.602, 2.947, 3.733 },
|
||||
{ 1.337, 1.746, 2.120, 2.583, 2.921, 3.686 },
|
||||
{ 1.333, 1.740, 2.110, 2.567, 2.898, 3.646 },
|
||||
{ 1.330, 1.734, 2.101, 2.552, 2.878, 3.610 },
|
||||
{ 1.328, 1.729, 2.093, 2.539, 2.861, 3.579 },
|
||||
{ 1.325, 1.725, 2.086, 2.528, 2.845, 3.552 },
|
||||
{ 1.323, 1.721, 2.080, 2.518, 2.831, 3.527 },
|
||||
{ 1.321, 1.717, 2.074, 2.508, 2.819, 3.505 },
|
||||
{ 1.319, 1.714, 2.069, 2.500, 2.807, 3.485 },
|
||||
{ 1.318, 1.711, 2.064, 2.492, 2.797, 3.467 },
|
||||
{ 1.316, 1.708, 2.060, 2.485, 2.787, 3.450 },
|
||||
{ 1.315, 1.706, 2.056, 2.479, 2.779, 3.435 },
|
||||
{ 1.314, 1.703, 2.052, 2.473, 2.771, 3.421 },
|
||||
{ 1.313, 1.701, 2.048, 2.467, 2.763, 3.408 },
|
||||
{ 1.311, 1.699, 2.045, 2.462, 2.756, 3.396 },
|
||||
{ 1.310, 1.697, 2.042, 2.457, 2.750, 3.385 },
|
||||
{ 1.309, 1.696, 2.040, 2.453, 2.744, 3.375 },
|
||||
{ 1.309, 1.694, 2.037, 2.449, 2.738, 3.365 },
|
||||
{ 1.308, 1.692, 2.035, 2.445, 2.733, 3.356 },
|
||||
{ 1.307, 1.691, 2.032, 2.441, 2.728, 3.348 },
|
||||
{ 1.306, 1.690, 2.030, 2.438, 2.724, 3.340 },
|
||||
{ 1.306, 1.688, 2.028, 2.434, 2.719, 3.333 },
|
||||
{ 1.305, 1.687, 2.026, 2.431, 2.715, 3.326 },
|
||||
{ 1.304, 1.686, 2.024, 2.429, 2.712, 3.319 },
|
||||
{ 1.304, 1.685, 2.023, 2.426, 2.708, 3.313 },
|
||||
{ 1.303, 1.684, 2.021, 2.423, 2.704, 3.307 },
|
||||
{ 1.303, 1.683, 2.020, 2.421, 2.701, 3.301 },
|
||||
{ 1.302, 1.682, 2.018, 2.418, 2.698, 3.296 },
|
||||
{ 1.302, 1.681, 2.017, 2.416, 2.695, 3.291 },
|
||||
{ 1.301, 1.680, 2.015, 2.414, 2.692, 3.286 },
|
||||
{ 1.301, 1.679, 2.014, 2.412, 2.690, 3.281 },
|
||||
{ 1.300, 1.679, 2.013, 2.410, 2.687, 3.277 },
|
||||
{ 1.300, 1.678, 2.012, 2.408, 2.685, 3.273 },
|
||||
{ 1.299, 1.677, 2.011, 2.407, 2.682, 3.269 },
|
||||
{ 1.299, 1.677, 2.010, 2.405, 2.680, 3.265 },
|
||||
{ 1.299, 1.676, 2.009, 2.403, 2.678, 3.261 },
|
||||
{ 1.298, 1.675, 2.008, 2.402, 2.676, 3.258 },
|
||||
{ 1.298, 1.675, 2.007, 2.400, 2.674, 3.255 },
|
||||
{ 1.298, 1.674, 2.006, 2.399, 2.672, 3.251 },
|
||||
{ 1.297, 1.674, 2.005, 2.397, 2.670, 3.248 },
|
||||
{ 1.297, 1.673, 2.004, 2.396, 2.668, 3.245 },
|
||||
{ 1.297, 1.673, 2.003, 2.395, 2.667, 3.242 },
|
||||
{ 1.297, 1.672, 2.002, 2.394, 2.665, 3.239 },
|
||||
{ 1.296, 1.672, 2.002, 2.392, 2.663, 3.237 },
|
||||
{ 1.296, 1.671, 2.001, 2.391, 2.662, 3.234 },
|
||||
{ 1.296, 1.671, 2.000, 2.390, 2.660, 3.232 },
|
||||
{ 1.296, 1.670, 2.000, 2.389, 2.659, 3.229 },
|
||||
{ 1.295, 1.670, 1.999, 2.388, 2.657, 3.227 },
|
||||
{ 1.295, 1.669, 1.998, 2.387, 2.656, 3.225 },
|
||||
{ 1.295, 1.669, 1.998, 2.386, 2.655, 3.223 },
|
||||
{ 1.295, 1.669, 1.997, 2.385, 2.654, 3.220 },
|
||||
{ 1.295, 1.668, 1.997, 2.384, 2.652, 3.218 },
|
||||
{ 1.294, 1.668, 1.996, 2.383, 2.651, 3.216 },
|
||||
{ 1.294, 1.668, 1.995, 2.382, 2.650, 3.214 },
|
||||
{ 1.294, 1.667, 1.995, 2.382, 2.649, 3.213 },
|
||||
{ 1.294, 1.667, 1.994, 2.381, 2.648, 3.211 },
|
||||
{ 1.294, 1.667, 1.994, 2.380, 2.647, 3.209 },
|
||||
{ 1.293, 1.666, 1.993, 2.379, 2.646, 3.207 },
|
||||
{ 1.293, 1.666, 1.993, 2.379, 2.645, 3.206 },
|
||||
{ 1.293, 1.666, 1.993, 2.378, 2.644, 3.204 },
|
||||
{ 1.293, 1.665, 1.992, 2.377, 2.643, 3.202 },
|
||||
{ 1.293, 1.665, 1.992, 2.376, 2.642, 3.201 },
|
||||
{ 1.293, 1.665, 1.991, 2.376, 2.641, 3.199 },
|
||||
{ 1.292, 1.665, 1.991, 2.375, 2.640, 3.198 },
|
||||
{ 1.292, 1.664, 1.990, 2.374, 2.640, 3.197 },
|
||||
{ 1.292, 1.664, 1.990, 2.374, 2.639, 3.195 },
|
||||
{ 1.292, 1.664, 1.990, 2.373, 2.638, 3.194 },
|
||||
{ 1.292, 1.664, 1.989, 2.373, 2.637, 3.193 },
|
||||
{ 1.292, 1.663, 1.989, 2.372, 2.636, 3.191 },
|
||||
{ 1.292, 1.663, 1.989, 2.372, 2.636, 3.190 },
|
||||
{ 1.292, 1.663, 1.988, 2.371, 2.635, 3.189 },
|
||||
{ 1.291, 1.663, 1.988, 2.370, 2.634, 3.188 },
|
||||
{ 1.291, 1.663, 1.988, 2.370, 2.634, 3.187 },
|
||||
{ 1.291, 1.662, 1.987, 2.369, 2.633, 3.185 },
|
||||
{ 1.291, 1.662, 1.987, 2.369, 2.632, 3.184 },
|
||||
{ 1.291, 1.662, 1.987, 2.368, 2.632, 3.183 },
|
||||
{ 1.291, 1.662, 1.986, 2.368, 2.631, 3.182 },
|
||||
{ 1.291, 1.662, 1.986, 2.368, 2.630, 3.181 },
|
||||
{ 1.291, 1.661, 1.986, 2.367, 2.630, 3.180 },
|
||||
{ 1.291, 1.661, 1.986, 2.367, 2.629, 3.179 },
|
||||
{ 1.291, 1.661, 1.985, 2.366, 2.629, 3.178 },
|
||||
{ 1.290, 1.661, 1.985, 2.366, 2.628, 3.177 },
|
||||
{ 1.290, 1.661, 1.985, 2.365, 2.627, 3.176 },
|
||||
{ 1.290, 1.661, 1.984, 2.365, 2.627, 3.175 },
|
||||
{ 1.290, 1.660, 1.984, 2.365, 2.626, 3.175 },
|
||||
{ 1.290, 1.660, 1.984, 2.364, 2.626, 3.174 },
|
||||
};
|
||||
|
||||
const std::vector<double> confidence_level = { 80, 90, 95, 98, 99, 99.5 };
|
||||
|
||||
T_test()
|
||||
{
|
||||
data.resize(2);
|
||||
}
|
||||
|
||||
void clear()
|
||||
{
|
||||
data[0].clear();
|
||||
data[1].clear();
|
||||
}
|
||||
|
||||
void add(size_t distribution, double value)
|
||||
{
|
||||
if (distribution > 1)
|
||||
return;
|
||||
data[distribution].add(value);
|
||||
}
|
||||
|
||||
/// Confidence_level_index can be set in range [0, 5]. Corresponding values can be found above.
|
||||
std::pair<bool, std::string> compareAndReport(size_t confidence_level_index = 5) const
|
||||
{
|
||||
if (confidence_level_index > 5)
|
||||
confidence_level_index = 5;
|
||||
|
||||
if (data[0].size == 0 || data[1].size == 0)
|
||||
return {true, ""};
|
||||
|
||||
size_t degrees_of_freedom = (data[0].size - 1) + (data[1].size - 1);
|
||||
|
||||
double table_value = students_table[degrees_of_freedom > 100 ? 0 : degrees_of_freedom][confidence_level_index];
|
||||
|
||||
double pooled_standard_deviation = sqrt(((data[0].size - 1) * data[0].var() + (data[1].size - 1) * data[1].var()) / degrees_of_freedom);
|
||||
|
||||
double t_statistic = pooled_standard_deviation * sqrt(1.0 / data[0].size + 1.0 / data[1].size);
|
||||
|
||||
double mean_difference = fabs(data[0].avg() - data[1].avg());
|
||||
|
||||
double mean_confidence_interval = table_value * t_statistic;
|
||||
|
||||
std::stringstream ss;
|
||||
if (mean_difference > mean_confidence_interval && (mean_difference - mean_confidence_interval > 0.0001)) /// difference must be more than 0.0001, to take into account connection latency.
|
||||
{
|
||||
ss << "Difference at " << confidence_level[confidence_level_index] << "% confidence : ";
|
||||
ss << std::fixed << std::setprecision(8) << "mean difference is " << mean_difference << ", but confidence interval is " << mean_confidence_interval;
|
||||
return {false, ss.str()};
|
||||
}
|
||||
else
|
||||
{
|
||||
ss << "No difference proven at " << confidence_level[confidence_level_index] << "% confidence";
|
||||
return {true, ss.str()};
|
||||
}
|
||||
}
|
||||
|
||||
};
|
Loading…
Reference in New Issue
Block a user